OSDir


[Date Prev][Date Next][Thread Prev][Thread Next][Date Index][Thread Index]

[GitHub] niketh closed pull request #5999: Fix versionedinterval timeline to handle new data for the same version


niketh closed pull request #5999: Fix versionedinterval timeline to handle new data for the same version
URL: https://github.com/apache/incubator-druid/pull/5999
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/api/pom.xml b/api/pom.xml
index d4bcddd4370..d9ecd50af11 100644
--- a/api/pom.xml
+++ b/api/pom.xml
@@ -28,7 +28,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
     </parent>
 
     <dependencies>
diff --git a/api/src/main/java/io/druid/guice/JsonConfigurator.java b/api/src/main/java/io/druid/guice/JsonConfigurator.java
index f6d766dd955..09721a932d5 100644
--- a/api/src/main/java/io/druid/guice/JsonConfigurator.java
+++ b/api/src/main/java/io/druid/guice/JsonConfigurator.java
@@ -93,7 +93,6 @@ public JsonConfigurator(
           log.info(e, "Unable to parse [%s]=[%s] as a json object, using as is.", prop, propValue);
           value = propValue;
         }
-
         hieraricalPutValue(propertyPrefix, prop, prop.substring(propertyBase.length()), value, jsonMap);
       }
     }
@@ -175,8 +174,11 @@ private static void hieraricalPutValue(
   )
   {
     int dotIndex = property.indexOf('.');
+    // Always put property with name even if it is of form a.b. This will make sure the property is available for classes
+    // where JsonProperty names are of the form a.b
+    // Note:- this will cause more than required properties to be present in the jsonMap.
+    targetMap.put(property, value);
     if (dotIndex < 0) {
-      targetMap.put(property, value);
       return;
     }
     if (dotIndex == 0) {
diff --git a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java
index a45a9a7865a..0560206f213 100644
--- a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java
+++ b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java
@@ -25,6 +25,7 @@
 import org.joda.time.DateTime;
 
 import javax.annotation.Nullable;
+import java.util.Objects;
 
 public class TaskStatusPlus
 {
@@ -40,7 +41,7 @@ public TaskStatusPlus(
       @JsonProperty("id") String id,
       @JsonProperty("createdTime") DateTime createdTime,
       @JsonProperty("queueInsertionTime") DateTime queueInsertionTime,
-      @JsonProperty("state") @Nullable TaskState state,
+      @JsonProperty("statusCode") @Nullable TaskState state,
       @JsonProperty("duration") @Nullable Long duration,
       @JsonProperty("location") TaskLocation location
   )
@@ -74,7 +75,8 @@ public DateTime getQueueInsertionTime()
     return queueInsertionTime;
   }
 
-  @JsonProperty
+  @Nullable
+  @JsonProperty("statusCode")
   public TaskState getState()
   {
     return state;
@@ -91,4 +93,40 @@ public TaskLocation getLocation()
   {
     return location;
   }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    final TaskStatusPlus that = (TaskStatusPlus) o;
+    if (!id.equals(that.id)) {
+      return false;
+    }
+    if (!createdTime.equals(that.createdTime)) {
+      return false;
+    }
+    if (!queueInsertionTime.equals(that.queueInsertionTime)) {
+      return false;
+    }
+    if (!Objects.equals(state, that.state)) {
+      return false;
+    }
+    if (!Objects.equals(duration, that.duration)) {
+      return false;
+    }
+    return location.equals(that.location);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(id, createdTime, queueInsertionTime, state, duration, location);
+  }
 }
diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentFinder.java b/api/src/main/java/io/druid/segment/loading/DataSegmentFinder.java
index 937a42e72c2..66af8353daf 100644
--- a/api/src/main/java/io/druid/segment/loading/DataSegmentFinder.java
+++ b/api/src/main/java/io/druid/segment/loading/DataSegmentFinder.java
@@ -20,8 +20,11 @@
 package io.druid.segment.loading;
 
 import io.druid.guice.annotations.ExtensionPoint;
+import io.druid.java.util.common.Pair;
+import io.druid.java.util.common.logger.Logger;
 import io.druid.timeline.DataSegment;
 
+import java.util.Map;
 import java.util.Set;
 
 /**
@@ -31,6 +34,8 @@
 @ExtensionPoint
 public interface DataSegmentFinder
 {
+  Logger log = new Logger(DataSegmentFinder.class);
+
   /**
    * This method should first recursively look for descriptor.json (partitionNum_descriptor.json for HDFS data storage) underneath
    * workingDirPath and then verify that index.zip (partitionNum_index.zip for HDFS data storage) exists in the same folder.
@@ -46,4 +51,26 @@
    * @return a set of segments that were found underneath workingDirPath
    */
   Set<DataSegment> findSegments(String workingDirPath, boolean updateDescriptor) throws SegmentLoadingException;
+
+  /**
+   * Adds dataSegment if it does not exist in timestampedSegments. If it exists, replaces entry if segmentModifiedAt is
+   * newer than stored timestamp.
+   *
+   * @param timestampedSegments map of <segmentID, Pair<segment, modifiedAt>> containing segments with modified time
+   * @param dataSegment         segment to add
+   * @param segmentModifiedAt   segment modified timestamp
+   */
+  static void putInMapRetainingNewest(
+      Map<String, Pair<DataSegment, Long>> timestampedSegments, DataSegment dataSegment, long segmentModifiedAt
+  )
+  {
+    timestampedSegments.merge(
+        dataSegment.getIdentifier(),
+        Pair.of(dataSegment, segmentModifiedAt),
+        (previous, current) -> {
+          log.warn("Multiple copies of segmentId [%s] found, using newest version", current.lhs.getIdentifier());
+          return previous.rhs > current.rhs ? previous : current;
+        }
+    );
+  }
 }
diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentKiller.java b/api/src/main/java/io/druid/segment/loading/DataSegmentKiller.java
index c26a73daeb1..96bbeb9362d 100644
--- a/api/src/main/java/io/druid/segment/loading/DataSegmentKiller.java
+++ b/api/src/main/java/io/druid/segment/loading/DataSegmentKiller.java
@@ -20,16 +20,41 @@
 package io.druid.segment.loading;
 
 import io.druid.guice.annotations.ExtensionPoint;
+import io.druid.java.util.common.logger.Logger;
 import io.druid.timeline.DataSegment;
 
 import java.io.IOException;
 
-/**
- */
 @ExtensionPoint
 public interface DataSegmentKiller
 {
-  void kill(DataSegment segments) throws SegmentLoadingException;
-  void killAll() throws IOException;
+  Logger log = new Logger(DataSegmentKiller.class);
+
+  /**
+   * Removes segment files (index and metadata) from deep storage.
+   * @param segment the segment to kill
+   * @throws SegmentLoadingException if the segment could not be completely removed
+   */
+  void kill(DataSegment segment) throws SegmentLoadingException;
 
+  /**
+   * A more stoic killer who doesn't throw a tantrum if things get messy. Use when killing segments for best-effort
+   * cleanup.
+   * @param segment the segment to kill
+   */
+  default void killQuietly(DataSegment segment)
+  {
+    try {
+      kill(segment);
+    }
+    catch (Exception e) {
+      log.debug(e, "Failed to kill segment %s", segment);
+    }
+  }
+
+  /**
+   * Like a nuke. Use wisely. Used by the 'reset-cluster' command, and of the built-in deep storage implementations, it
+   * is only implemented by local and HDFS.
+   */
+  void killAll() throws IOException;
 }
diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentPusher.java b/api/src/main/java/io/druid/segment/loading/DataSegmentPusher.java
index b9bf810f72c..7c4cead40cb 100644
--- a/api/src/main/java/io/druid/segment/loading/DataSegmentPusher.java
+++ b/api/src/main/java/io/druid/segment/loading/DataSegmentPusher.java
@@ -30,6 +30,7 @@
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 
 @ExtensionPoint
 public interface DataSegmentPusher
@@ -39,18 +40,53 @@
   @Deprecated
   String getPathForHadoop(String dataSource);
   String getPathForHadoop();
-  DataSegment push(File file, DataSegment segment) throws IOException;
+
+  /**
+   * Pushes index files and segment descriptor to deep storage.
+   * @param file directory containing index files
+   * @param segment segment descriptor
+   * @param useUniquePath if true, pushes to a unique file path. This prevents situations where task failures or replica
+   *                      tasks can either overwrite or fail to overwrite existing segments leading to the possibility
+   *                      of different versions of the same segment ID containing different data. As an example, a Kafka
+   *                      indexing task starting at offset A and ending at offset B may push a segment to deep storage
+   *                      and then fail before writing the loadSpec to the metadata table, resulting in a replacement
+   *                      task being spawned. This replacement will also start at offset A but will read to offset C and
+   *                      will then push a segment to deep storage and write the loadSpec metadata. Without unique file
+   *                      paths, this can only work correctly if new segments overwrite existing segments. Suppose that
+   *                      at this point the task then fails so that the supervisor retries again from offset A. This 3rd
+   *                      attempt will overwrite the segments in deep storage before failing to write the loadSpec
+   *                      metadata, resulting in inconsistencies in the segment data now in deep storage and copies of
+   *                      the segment already loaded by historicals.
+   *
+   *                      If unique paths are used, caller is responsible for cleaning up segments that were pushed but
+   *                      were not written to the metadata table (for example when using replica tasks).
+   * @return segment descriptor
+   * @throws IOException
+   */
+  DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException;
+
   //use map instead of LoadSpec class to avoid dependency pollution.
   Map<String, Object> makeLoadSpec(URI finalIndexZipFilePath);
 
+  /**
+   * @deprecated backward-compatibiliy shim that should be removed on next major release;
+   * use {@link #getStorageDir(DataSegment, boolean)} instead.
+   */
+  @Deprecated
   default String getStorageDir(DataSegment dataSegment)
   {
-    return getDefaultStorageDir(dataSegment);
+    return getStorageDir(dataSegment, false);
+  }
+
+  default String getStorageDir(DataSegment dataSegment, boolean useUniquePath)
+  {
+    return getDefaultStorageDir(dataSegment, useUniquePath);
   }
 
   default String makeIndexPathName(DataSegment dataSegment, String indexName)
   {
-    return StringUtils.format("./%s/%s", getStorageDir(dataSegment), indexName);
+    // This is only called from Hadoop batch which doesn't require unique segment paths so set useUniquePath=false
+    return StringUtils.format("./%s/%s", getStorageDir(dataSegment, false), indexName);
   }
 
   /**
@@ -66,13 +102,19 @@ default String makeIndexPathName(DataSegment dataSegment, String indexName)
   // If above format is ever changed, make sure to change it appropriately in other places
   // e.g. HDFSDataSegmentKiller uses this information to clean the version, interval and dataSource directories
   // on segment deletion if segment being deleted was the only segment
-  static String getDefaultStorageDir(DataSegment segment)
+  static String getDefaultStorageDir(DataSegment segment, boolean useUniquePath)
   {
     return JOINER.join(
         segment.getDataSource(),
         StringUtils.format("%s_%s", segment.getInterval().getStart(), segment.getInterval().getEnd()),
         segment.getVersion(),
-        segment.getShardSpec().getPartitionNum()
+        segment.getShardSpec().getPartitionNum(),
+        useUniquePath ? generateUniquePath() : null
     );
   }
+
+  static String generateUniquePath()
+  {
+    return UUID.randomUUID().toString();
+  }
 }
diff --git a/api/src/main/java/io/druid/utils/CompressionUtils.java b/api/src/main/java/io/druid/utils/CompressionUtils.java
index 6a551e319e0..2b25186b25f 100644
--- a/api/src/main/java/io/druid/utils/CompressionUtils.java
+++ b/api/src/main/java/io/druid/utils/CompressionUtils.java
@@ -36,26 +36,26 @@
   private static final Logger log = new Logger(CompressionUtils.class);
 
 
-  @Deprecated // Use com.metamx.common.CompressionUtils.zip
+  @Deprecated // Use io.druid.java.util.common.CompressionUtils.zip
   public static long zip(File directory, File outputZipFile) throws IOException
   {
     return io.druid.java.util.common.CompressionUtils.zip(directory, outputZipFile);
   }
 
 
-  @Deprecated // Use com.metamx.common.CompressionUtils.zip
+  @Deprecated // Use io.druid.java.util.common.CompressionUtils.zip
   public static long zip(File directory, OutputStream out) throws IOException
   {
     return io.druid.java.util.common.CompressionUtils.zip(directory, out);
   }
 
-  @Deprecated // Use com.metamx.common.CompressionUtils.unzip
+  @Deprecated // Use io.druid.java.util.common.CompressionUtils.unzip
   public static void unzip(File pulledFile, File outDir) throws IOException
   {
     io.druid.java.util.common.CompressionUtils.unzip(pulledFile, outDir);
   }
 
-  @Deprecated // Use com.metamx.common.CompressionUtils.unzip
+  @Deprecated // Use io.druid.java.util.common.CompressionUtils.unzip
   public static void unzip(InputStream in, File outDir) throws IOException
   {
     io.druid.java.util.common.CompressionUtils.unzip(in, outDir);
@@ -63,8 +63,8 @@ public static void unzip(InputStream in, File outDir) throws IOException
 
   /**
    * Uncompress using a gzip uncompress algorithm from the `pulledFile` to the `outDir`.
-   * Unlike `com.metamx.common.CompressionUtils.gunzip`, this function takes an output *DIRECTORY* and tries to guess the file name.
-   * It is recommended that the caller use `com.metamx.common.CompressionUtils.gunzip` and specify the output file themselves to ensure names are as expected
+   * Unlike `io.druid.java.util.common.CompressionUtils.gunzip`, this function takes an output *DIRECTORY* and tries to guess the file name.
+   * It is recommended that the caller use `io.druid.java.util.common.CompressionUtils.gunzip` and specify the output file themselves to ensure names are as expected
    *
    * @param pulledFile The source file
    * @param outDir     The destination directory to put the resulting file
diff --git a/api/src/test/java/io/druid/guice/JsonConfiguratorTest.java b/api/src/test/java/io/druid/guice/JsonConfiguratorTest.java
index 0ce4f77a79a..acfadf57131 100644
--- a/api/src/test/java/io/druid/guice/JsonConfiguratorTest.java
+++ b/api/src/test/java/io/druid/guice/JsonConfiguratorTest.java
@@ -94,10 +94,13 @@ public ExecutableValidator forExecutables()
   public void testTest()
   {
     Assert.assertEquals(
-        new MappableObject("p1", ImmutableList.<String>of("p2")),
-        new MappableObject("p1", ImmutableList.<String>of("p2"))
+        new MappableObject("p1", ImmutableList.<String>of("p2"), "p2"),
+        new MappableObject("p1", ImmutableList.<String>of("p2"), "p2")
+    );
+    Assert.assertEquals(
+        new MappableObject("p1", null, null),
+        new MappableObject("p1", ImmutableList.<String>of(), null)
     );
-    Assert.assertEquals(new MappableObject("p1", null), new MappableObject("p1", ImmutableList.<String>of()));
   }
 
   @Test
@@ -140,6 +143,19 @@ public void testQuotedConfig()
     Assert.assertEquals("testing \"prop1\"", obj.prop1);
     Assert.assertEquals(ImmutableList.of(), obj.prop1List);
   }
+
+  @Test
+  public void testPropertyWithDot()
+  {
+    final JsonConfigurator configurator = new JsonConfigurator(mapper, validator);
+    properties.setProperty(PROP_PREFIX + "prop2.prop.2", "testing");
+    properties.setProperty(PROP_PREFIX + "prop1", "prop1");
+    final MappableObject obj = configurator.configurate(properties, PROP_PREFIX, MappableObject.class);
+    Assert.assertEquals("testing", obj.prop2);
+    Assert.assertEquals(ImmutableList.of(), obj.prop1List);
+    Assert.assertEquals("prop1", obj.prop1);
+
+  }
 }
 
 class MappableObject
@@ -148,15 +164,19 @@ public void testQuotedConfig()
   final String prop1;
   @JsonProperty("prop1List")
   final List<String> prop1List;
+  @JsonProperty("prop2.prop.2")
+  final String prop2;
 
   @JsonCreator
   protected MappableObject(
       @JsonProperty("prop1") final String prop1,
-      @JsonProperty("prop1List") final List<String> prop1List
+      @JsonProperty("prop1List") final List<String> prop1List,
+      @JsonProperty("prop2.prop.2") final String prop2
   )
   {
     this.prop1 = prop1;
     this.prop1List = prop1List == null ? ImmutableList.<String>of() : prop1List;
+    this.prop2 = prop2;
   }
 
 
@@ -172,6 +192,12 @@ public String getProp1()
     return prop1;
   }
 
+  @JsonProperty
+  public String getProp2()
+  {
+    return prop2;
+  }
+
   @Override
   public boolean equals(Object o)
   {
diff --git a/api/src/test/java/io/druid/indexer/TaskStatusPlusTest.java b/api/src/test/java/io/druid/indexer/TaskStatusPlusTest.java
new file mode 100644
index 00000000000..a4f3460e65a
--- /dev/null
+++ b/api/src/test/java/io/druid/indexer/TaskStatusPlusTest.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.indexer;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
+import io.druid.java.util.common.DateTimes;
+import org.joda.time.DateTime;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TaskStatusPlusTest
+{
+  @Test
+  public void testSerde() throws IOException
+  {
+    final ObjectMapper mapper = new ObjectMapper();
+    mapper.registerModule(
+        new SimpleModule()
+            .addDeserializer(DateTime.class, new DateTimeDeserializer())
+            .addSerializer(DateTime.class, ToStringSerializer.instance)
+    );
+    final TaskStatusPlus status = new TaskStatusPlus(
+        "testId",
+        DateTimes.nowUtc(),
+        DateTimes.nowUtc(),
+        TaskState.RUNNING,
+        1000L,
+        TaskLocation.create("testHost", 1010, -1)
+    );
+    final String json = mapper.writeValueAsString(status);
+    Assert.assertEquals(status, mapper.readValue(json, TaskStatusPlus.class));
+  }
+
+  // Copied from io.druid.jackson.JodaStuff
+  private static class DateTimeDeserializer extends StdDeserializer<DateTime>
+  {
+    public DateTimeDeserializer()
+    {
+      super(DateTime.class);
+    }
+
+    @Override
+    public DateTime deserialize(JsonParser jp, DeserializationContext ctxt)
+        throws IOException, JsonProcessingException
+    {
+      JsonToken t = jp.getCurrentToken();
+      if (t == JsonToken.VALUE_NUMBER_INT) {
+        return DateTimes.utc(jp.getLongValue());
+      }
+      if (t == JsonToken.VALUE_STRING) {
+        String str = jp.getText().trim();
+        if (str.length() == 0) { // [JACKSON-360]
+          return null;
+        }
+        // make sure to preserve time zone information when parsing timestamps
+        return DateTimes.ISO_DATE_OR_TIME_WITH_OFFSET.parse(str);
+      }
+      throw ctxt.mappingException(getValueClass());
+    }
+  }
+}
diff --git a/aws-common/pom.xml b/aws-common/pom.xml
index 608552f98f5..53457ad7588 100644
--- a/aws-common/pom.xml
+++ b/aws-common/pom.xml
@@ -26,7 +26,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
     </parent>
 
     <dependencies>
diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml
index 596931c1439..61fbd627cbf 100644
--- a/benchmarks/pom.xml
+++ b/benchmarks/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
   </parent>
 
   <prerequisites>
diff --git a/benchmarks/src/main/java/io/druid/benchmark/LoadStatusBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/LoadStatusBenchmark.java
new file mode 100644
index 00000000000..e1b877d481d
--- /dev/null
+++ b/benchmarks/src/main/java/io/druid/benchmark/LoadStatusBenchmark.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.benchmark;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.StringUtils;
+import io.druid.timeline.DataSegment;
+import io.druid.timeline.partition.NoneShardSpec;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 15)
+@Measurement(iterations = 30)
+public class LoadStatusBenchmark
+{
+  // Number of total data segments
+  @Param({"10000"})
+  int totalSegmentsCount;
+
+  @Param({"true", "false"})
+  private boolean serverHasAllSegments;
+
+  private Set<DataSegment> datasourceSegments;
+  private Collection<DataSegment> serverSegments;
+
+  @Setup(Level.Invocation)
+  public void setup()
+  {
+    Map<String, DataSegment> immutableDatasourceSegmentsMap;
+    ConcurrentHashMap<String, DataSegment> serverSegmentsMap;
+
+    HashMap<String, DataSegment> datasourceSegmentsMap = Maps.newHashMap();
+    serverSegmentsMap = new ConcurrentHashMap<>();
+
+    for (int i = 0; i < totalSegmentsCount; i++) {
+      DataSegment segment = new DataSegment(
+          "benchmarkDatasource",
+          Intervals.of(StringUtils.format("%s-01-01/%s-12-31", i + 1970, i + 1970)),
+          "1",
+          null,
+          null,
+          null,
+          NoneShardSpec.instance(),
+          1,
+          1
+      );
+
+      datasourceSegmentsMap.put(segment.getIdentifier(), segment);
+
+      if (serverHasAllSegments || i % 2 == 0) {
+        serverSegmentsMap.put(segment.getIdentifier(), segment);
+      }
+    }
+
+    immutableDatasourceSegmentsMap = ImmutableMap.copyOf(datasourceSegmentsMap);
+
+    datasourceSegments = Sets.newHashSet(immutableDatasourceSegmentsMap.values());
+    serverSegments = Collections.unmodifiableCollection(serverSegmentsMap.values());
+  }
+
+  @Benchmark
+  @BenchmarkMode(Mode.AverageTime)
+  @OutputTimeUnit(TimeUnit.MICROSECONDS)
+  public void oldVersion(Blackhole blackhole)
+  {
+    datasourceSegments.removeAll(serverSegments);
+    blackhole.consume(datasourceSegments);
+  }
+
+  @Benchmark
+  @BenchmarkMode(Mode.AverageTime)
+  @OutputTimeUnit(TimeUnit.MICROSECONDS)
+  public void newVersion(Blackhole blackhole)
+  {
+    for (DataSegment segment : serverSegments) {
+      datasourceSegments.remove(segment);
+    }
+    blackhole.consume(datasourceSegments);
+  }
+}
diff --git a/bytebuffer-collections/pom.xml b/bytebuffer-collections/pom.xml
deleted file mode 100755
index 4b8b231d5ea..00000000000
--- a/bytebuffer-collections/pom.xml
+++ /dev/null
@@ -1,138 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ Licensed to Metamarkets Group Inc. (Metamarkets) under one
-  ~ or more contributor license agreements. See the NOTICE file
-  ~ distributed with this work for additional information
-  ~ regarding copyright ownership. Metamarkets 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:xsi="http://www.w3.org/2001/XMLSchema-instance"; xmlns="http://maven.apache.org/POM/4.0.0";
-         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>io.druid</groupId>
-    <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
-  </parent>
-
-  <artifactId>bytebuffer-collections</artifactId>
-  <name>bytebuffer-collections</name>
-  <description>ByteBuffer Collections</description>
-
-  <dependencies>
-    <dependency>
-      <groupId>io.druid</groupId>
-      <artifactId>extendedset</artifactId>
-      <version>${project.parent.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-      <version>16.0.1</version>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-annotations</artifactId>
-      <version>2.4.6</version>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-core</artifactId>
-      <version>2.4.6</version>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-      <version>2.4.6</version>
-    </dependency>
-    <dependency>
-      <groupId>org.roaringbitmap</groupId>
-      <artifactId>RoaringBitmap</artifactId>
-    </dependency>
-
-    <!-- Tests -->
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.easymock</groupId>
-      <artifactId>easymock</artifactId>
-      <version>3.0</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.carrotsearch</groupId>
-      <artifactId>junit-benchmarks</artifactId>
-      <version>0.7.2</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.h2database</groupId>
-      <artifactId>h2</artifactId>
-      <version>1.4.182</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava-testlib</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-        <version>2.4</version>
-        <executions>
-          <execution>
-            <goals>
-              <goal>test-jar</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <excludedGroups>io.druid.test.annotation.Benchmark</excludedGroups>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-
-  <profiles>
-    <profile>
-      <id>benchmark</id>
-      <build>
-        <plugins>
-          <plugin>
-            <artifactId>maven-surefire-plugin</artifactId>
-            <configuration>
-              <argLine>-server -Xms3G -Xmx3G -Djub.consumers=CONSOLE,H2 -Djub.db.file=benchmarks/benchmarks</argLine>
-              <groups>io.druid.test.annotation.Benchmark</groups>
-              <excludedGroups>io.druid.test.annotation.Dummy</excludedGroups>
-            </configuration>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-</project>
diff --git a/common/pom.xml b/common/pom.xml
index 58379d33864..8a453a950b1 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -26,7 +26,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
     </parent>
 
     <dependencies>
diff --git a/common/src/main/java/io/druid/concurrent/LifecycleLock.java b/common/src/main/java/io/druid/concurrent/LifecycleLock.java
index a94e3cd31a3..7cc8e64a113 100644
--- a/common/src/main/java/io/druid/concurrent/LifecycleLock.java
+++ b/common/src/main/java/io/druid/concurrent/LifecycleLock.java
@@ -175,9 +175,9 @@ void exitStop()
       }
     }
 
-    void reset()
+    void exitStopAndReset()
     {
-      if (!compareAndSetState(STOPPED, NOT_STARTED)) {
+      if (!compareAndSetState(STOPPING, NOT_STARTED)) {
         throw new IllegalMonitorStateException("Not called exitStop() before reset()");
       }
     }
@@ -187,7 +187,7 @@ void reset()
 
   /**
    * Start latch, only one canStart() call in any thread on this LifecycleLock object could return true, if {@link
-   * #reset()} is not called in between.
+   * #exitStopAndReset()} is not called in between.
    */
   public boolean canStart()
   {
@@ -257,8 +257,8 @@ public boolean canStop()
   }
 
   /**
-   * If this LifecycleLock is used in a restartable object, which uses {@link #reset()}, exitStop() must be called
-   * before exit from stop() on this object, usually in a finally block.
+   * Finalizes stopping the the LifecycleLock. This method must be called before exit from stop() on this object,
+   * usually in a finally block. If you're using a restartable object, use {@link #exitStopAndReset()} instead.
    *
    * @throws IllegalMonitorStateException if {@link #canStop()} is not yet called on this LifecycleLock
    */
@@ -268,12 +268,14 @@ public void exitStop()
   }
 
   /**
-   * Resets the LifecycleLock after {@link #exitStop()}, so that {@link #canStart()} could be called again.
+   * Finalizes stopping the LifecycleLock and resets it, so that {@link #canStart()} could be called again. If this
+   * LifecycleLock is used in a restartable object, this method must be called before exit from stop() on this object,
+   * usually in a finally block.
    *
-   * @throws IllegalMonitorStateException if {@link #exitStop()} is not yet called on this LifecycleLock
+   * @throws IllegalMonitorStateException if {@link #canStop()} is not yet called on this LifecycleLock
    */
-  public void reset()
+  public void exitStopAndReset()
   {
-    sync.reset();
+    sync.exitStopAndReset();
   }
 }
diff --git a/common/src/main/java/io/druid/timeline/partition/PartitionHolder.java b/common/src/main/java/io/druid/timeline/partition/PartitionHolder.java
index ace26479cb0..13101b56eb1 100644
--- a/common/src/main/java/io/druid/timeline/partition/PartitionHolder.java
+++ b/common/src/main/java/io/druid/timeline/partition/PartitionHolder.java
@@ -57,6 +57,12 @@ public PartitionHolder(PartitionHolder partitionHolder)
 
   public void add(PartitionChunk<T> chunk)
   {
+    // Treeset will ignore add if it already contains the chunk.
+    // Removing and readding will ensure that it adds the latest chunk
+    if (holderSet.contains(chunk)) {
+      holderSet.remove(chunk);
+    }
+
     holderSet.add(chunk);
   }
 
diff --git a/common/src/test/java/io/druid/concurrent/LifecycleLockTest.java b/common/src/test/java/io/druid/concurrent/LifecycleLockTest.java
index e1d9f8adbbf..afb55bf2c2c 100644
--- a/common/src/test/java/io/druid/concurrent/LifecycleLockTest.java
+++ b/common/src/test/java/io/druid/concurrent/LifecycleLockTest.java
@@ -138,8 +138,7 @@ public void testRestart()
     lifecycleLock.started();
     lifecycleLock.exitStart();
     Assert.assertTrue(lifecycleLock.canStop());
-    lifecycleLock.exitStop();
-    lifecycleLock.reset();
+    lifecycleLock.exitStopAndReset();
     Assert.assertTrue(lifecycleLock.canStart());
   }
 
diff --git a/common/src/test/java/io/druid/timeline/VersionedIntervalTimelineTest.java b/common/src/test/java/io/druid/timeline/VersionedIntervalTimelineTest.java
index 9fbd97a946b..b2a18f7e3f4 100644
--- a/common/src/test/java/io/druid/timeline/VersionedIntervalTimelineTest.java
+++ b/common/src/test/java/io/druid/timeline/VersionedIntervalTimelineTest.java
@@ -63,6 +63,7 @@ public void setUp() throws Exception
     add("2011-04-01/2011-04-09", "2", 1);
     add("2011-04-06/2011-04-09", "3", 4);
     add("2011-04-01/2011-04-02", "3", 5);
+    add("2011-04-06/2011-04-09", "3", 100);
 
     add("2011-05-01/2011-05-02", "1", 6);
     add("2011-05-01/2011-05-05", "2", 7);
@@ -84,7 +85,7 @@ public void testApril() throws Exception
         Arrays.asList(
             createExpected("2011-04-01/2011-04-02", "3", 5),
             createExpected("2011-04-02/2011-04-06", "2", 1),
-            createExpected("2011-04-06/2011-04-09", "3", 4)
+            createExpected("2011-04-06/2011-04-09", "3", 100)
         ),
         timeline.lookup(Intervals.of("2011-04-01/2011-04-09"))
     );
@@ -102,7 +103,7 @@ public void testApril2() throws Exception
             createExpected("2011-04-01/2011-04-02", "3", 5),
             createExpected("2011-04-02/2011-04-03", "1", 2),
             createExpected("2011-04-03/2011-04-06", "1", 3),
-            createExpected("2011-04-06/2011-04-09", "3", 4)
+            createExpected("2011-04-06/2011-04-09", "3", 100)
         ),
         timeline.lookup(Intervals.of("2011-04-01/2011-04-09"))
     );
@@ -123,7 +124,7 @@ public void testApril3() throws Exception
         Arrays.asList(
             createExpected("2011-04-01/2011-04-02", "3", 5),
             createExpected("2011-04-03/2011-04-06", "1", 3),
-            createExpected("2011-04-06/2011-04-09", "3", 4)
+            createExpected("2011-04-06/2011-04-09", "3", 100)
         ),
         timeline.lookup(Intervals.of("2011-04-01/2011-04-09"))
     );
@@ -443,7 +444,7 @@ public void testOverlapSameVersionIsOkay() throws Exception
 
     assertValues(
         Collections.singletonList(
-            createExpected("2011-01-01/2011-01-10", "2", 2)
+            createExpected("2011-01-01/2011-01-10", "2", 3)
         ),
         timeline.lookup(Intervals.of("2011-01-01/2011-01-10"))
     );
@@ -560,8 +561,8 @@ public void testOverlapSecondAfter() throws Exception
 
     assertValues(
         Arrays.asList(
-            createExpected("2011-01-01/2011-01-10", "2", 1),
-            createExpected("2011-01-10/2011-01-15", "1", 3)
+            createExpected("2011-01-01/2011-01-10", "2", 3),
+            createExpected("2011-01-10/2011-01-15", "1", 1)
         ),
         timeline.lookup(Intervals.of("2011-01-01/2011-01-15"))
     );
@@ -930,7 +931,7 @@ public void testOverlapV1Large() throws Exception
             createExpected("2011-01-01/2011-01-03", "1", 1),
             createExpected("2011-01-03/2011-01-05", "2", 2),
             createExpected("2011-01-05/2011-01-13", "1", 1),
-            createExpected("2011-01-13/2011-01-20", "2", 2)
+            createExpected("2011-01-13/2011-01-20", "2", 3)
         ),
         timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
     );
@@ -947,10 +948,12 @@ public void testOverlapV2Large() throws Exception
     add("2011-01-03/2011-01-05", "1", 2);
     add("2011-01-13/2011-01-20", "1", 3);
 
+    System.out.println(timeline.lookup(Intervals.of("2011-01-01/2011-01-20")));
+
     assertValues(
         Arrays.asList(
-            createExpected("2011-01-01/2011-01-15", "2", 2),
-            createExpected("2011-01-15/2011-01-20", "1", 1)
+            createExpected("2011-01-01/2011-01-15", "2", 1),
+            createExpected("2011-01-15/2011-01-20", "1", 3)
         ),
         timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
     );
@@ -1775,7 +1778,7 @@ private void assertValues(
 
       Assert.assertEquals(pair.lhs, holder.getInterval());
       Assert.assertEquals(pair.rhs.lhs, holder.getVersion());
-      Assert.assertEquals(pair.rhs.rhs, holder.getObject());
+      Assert.assertEquals(pair.rhs.rhs.toString(), holder.getObject().toString());
     }
   }
 
diff --git a/distribution/pom.xml b/distribution/pom.xml
index 365ee356c46..15d9f898607 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -16,8 +16,7 @@
   ~ 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";>
+<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>
 
     <packaging>pom</packaging>
@@ -29,7 +28,7 @@
     <parent>
         <artifactId>druid</artifactId>
         <groupId>io.druid</groupId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
     </parent>
 
     <dependencies>
@@ -60,7 +59,7 @@
                             <executable>java</executable>
                             <arguments>
                                 <argument>-classpath</argument>
-                                <classpath/>
+                                <classpath />
                                 <argument>-Ddruid.extensions.loadList=[]</argument>
                                 <argument>-Ddruid.extensions.directory=${project.build.directory}/extensions</argument>
                                 <argument>
@@ -190,7 +189,7 @@
                                     <executable>java</executable>
                                     <arguments>
                                         <argument>-classpath</argument>
-                                        <classpath/>
+                                        <classpath />
                                         <argument>-Ddruid.extensions.loadList=[]</argument>
                                         <argument>-Ddruid.extensions.directory=${project.build.directory}/extensions
                                         </argument>
diff --git a/docs/content/configuration/auth.md b/docs/content/configuration/auth.md
index 358a54431b8..2fecfc1a86c 100644
--- a/docs/content/configuration/auth.md
+++ b/docs/content/configuration/auth.md
@@ -9,6 +9,7 @@ layout: doc_page
 |`druid.auth.authenticationChain`|JSON List of Strings|List of Authenticator type names|["allowAll"]|no|
 |`druid.escalator.type`|String|Type of the Escalator that should be used for internal Druid communications. This Escalator must use an authentication scheme that is supported by an Authenticator in `druid.auth.authenticationChain`.|"noop"|no|
 |`druid.auth.authorizers`|JSON List of Strings|List of Authorizer type names |["allowAll"]|no|
+|`druid.auth.allowUnauthenticatedHttpOptions`|Boolean|If true, skip authentication checks for HTTP OPTIONS requests. This is needed for certain use cases, such as supporting CORS pre-flight requests. Note that disabling authentication checks for OPTIONS requests will allow unauthenticated users to determine what Druid endpoints are valid (by checking if the OPTIONS request returns a 200 instead of 404), so enabling this option may reveal information about server configuration, including information about what extensions are loaded (if those extensions add endpoints).|false|no|
 
 ## Enabling Authentication/Authorization
 
diff --git a/docs/content/configuration/coordinator.md b/docs/content/configuration/coordinator.md
index 2dfd97098df..867a3043ac2 100644
--- a/docs/content/configuration/coordinator.md
+++ b/docs/content/configuration/coordinator.md
@@ -29,6 +29,7 @@ The coordinator node uses several of the global configs in [Configuration](../co
 |`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|false|
 |`druid.coordinator.conversion.on`|Boolean flag for converting old segment indexing versions to the latest segment indexing version.|false|
 |`druid.coordinator.load.timeout`|The timeout duration for when the coordinator assigns a segment to a historical node.|PT15M|
+|`druid.coordinator.kill.pendingSegments.on`|Boolean flag for whether or not the coordinator clean up old entries in the `pendingSegments` table of metadata store. If set to true, coordinator will check the created time of most recently complete task. If it doesn't exist, it finds the created time of the earlist running/pending/waiting tasks. Once the created time is found, then for all dataSources not in the `killPendingSegmentsSkipList` (see [Dynamic configuration](#dynamic-configuration)), coordinator will ask the overlord to clean up the entries 1 day or more older than the found created time in the `pendingSegments` table. This will be done periodically based on `druid.coordinator.period` specified.|false|
 |`druid.coordinator.kill.on`|Boolean flag for whether or not the coordinator should submit kill task for unused segments, that is, hard delete them from metadata store and deep storage. If set to true, then for all whitelisted dataSources (or optionally all), coordinator will submit tasks periodically based on `period` specified. These kill tasks will delete all segments except for the last `durationToRetain` period. Whitelist or All can be set via dynamic configuration `killAllDataSources` and `killDataSourceWhitelist` described later.|false|
 |`druid.coordinator.kill.period`|How often to send kill tasks to the indexing service. Value must be greater than `druid.coordinator.period.indexingPeriod`. Only applies if kill is turned on.|P1D (1 Day)|
 |`druid.coordinator.kill.durationToRetain`| Do not kill segments in last `durationToRetain`, must be greater or equal to 0. Only applies and MUST be specified if kill is turned on. Note that default value is invalid.|PT-1S (-1 seconds)|
@@ -103,8 +104,9 @@ Issuing a GET request at the same URL will return the spec that is currently in
 |`replicantLifetime`|The maximum number of coordinator runs for a segment to be replicated before we start alerting.|15|
 |`replicationThrottleLimit`|The maximum number of segments that can be replicated at one time.|10|
 |`emitBalancingStats`|Boolean flag for whether or not we should emit balancing stats. This is an expensive operation.|false|
-|`killDataSourceWhitelist`|List of dataSources for which kill tasks are sent if property `druid.coordinator.kill.on` is true.|none|
+|`killDataSourceWhitelist`|List of dataSources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none|
 |`killAllDataSources`|Send kill tasks for ALL dataSources if property `druid.coordinator.kill.on` is true. If this is set to true then `killDataSourceWhitelist` must not be specified or be empty list.|false|
+|`killPendingSegmentsSkipList`|List of dataSources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none|
 |`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 0 (loading queue is unbounded) |0|
 
 To view the audit history of coordinator dynamic config issue a GET request to the URL -
diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md
index 66b8ca41f76..02a0d20e9de 100644
--- a/docs/content/configuration/index.md
+++ b/docs/content/configuration/index.md
@@ -180,19 +180,19 @@ Druid nodes periodically emit metrics and different metrics monitors can be incl
 |Property|Description|Default|
 |--------|-----------|-------|
 |`druid.monitoring.emissionPeriod`|How often metrics are emitted.|PT1m|
-|`druid.monitoring.monitors`|Sets list of Druid monitors used by a node. See below for names and more information. For example, you can specify monitors for a Broker with `druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"]`.|none (no monitors)|
+|`druid.monitoring.monitors`|Sets list of Druid monitors used by a node. See below for names and more information. For example, you can specify monitors for a Broker with `druid.monitoring.monitors=["io.druid.java.util.metrics.SysMonitor","io.druid.java.util.metrics.JvmMonitor"]`.|none (no monitors)|
 
 The following monitors are available:
 
 |Name|Description|
 |----|-----------|
 |`io.druid.client.cache.CacheMonitor`|Emits metrics (to logs) about the segment results cache for Historical and Broker nodes. Reports typical cache statistics include hits, misses, rates, and size (bytes and number of entries), as well as timeouts and and errors.|
-|`com.metamx.metrics.SysMonitor`|This uses the [SIGAR library](http://www.hyperic.com/products/sigar) to report on various system activities and statuses.|
+|`io.druid.java.util.metrics.SysMonitor`|This uses the [SIGAR library](http://www.hyperic.com/products/sigar) to report on various system activities and statuses.|
 |`io.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical nodes.|
-|`com.metamx.metrics.JvmMonitor`|Reports various JVM-related statistics.|
-|`com.metamx.metrics.JvmCpuMonitor`|Reports statistics of CPU consumption by the JVM.|
-|`com.metamx.metrics.CpuAcctDeltaMonitor`|Reports consumed CPU as per the cpuacct cgroup.|
-|`com.metamx.metrics.JvmThreadsMonitor`|Reports Thread statistics in the JVM, like numbers of total, daemon, started, died threads.|
+|`io.druid.java.util.metrics.JvmMonitor`|Reports various JVM-related statistics.|
+|`io.druid.java.util.metrics.JvmCpuMonitor`|Reports statistics of CPU consumption by the JVM.|
+|`io.druid.java.util.metrics.CpuAcctDeltaMonitor`|Reports consumed CPU as per the cpuacct cgroup.|
+|`io.druid.java.util.metrics.JvmThreadsMonitor`|Reports Thread statistics in the JVM, like numbers of total, daemon, started, died threads.|
 |`io.druid.segment.realtime.RealtimeMetricsMonitor`|Reports statistics on Realtime nodes.|
 |`io.druid.server.metrics.EventReceiverFirehoseMonitor`|Reports how many events have been queued in the EventReceiverFirehose.|
 |`io.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted.|
@@ -222,8 +222,8 @@ The Druid servers [emit various metrics](../operations/metrics.html) and alerts
 |`druid.emitter.http.basicAuthentication`|Login and password for authentification in "login:password" form, e. g. `druid.emitter.http.basicAuthentication=admin:adminpassword`|not specified = no authentification|
 |`druid.emitter.http.flushTimeOut|The timeout after which an event should be sent to the endpoint, even if internal buffers are not filled, in milliseconds.|not specified = no timeout|
 |`druid.emitter.http.batchingStrategy`|The strategy of how the batch is formatted. "ARRAY" means `[event1,event2]`, "NEWLINES" means `event1\nevent2`, ONLY_EVENTS means `event1event2`.|ARRAY|
-|`druid.emitter.http.maxBatchSize`|The maximum batch size, in bytes.|5191680 (i. e. 5 MB)|
-|`druid.emitter.http.batchQueueSizeLimit`|The maximum number of batches in emitter queue, if there are problems with emitting.|50|
+|`druid.emitter.http.maxBatchSize`|The maximum batch size, in bytes.|the minimum of (10% of JVM heap size divided by 2) or (5191680 (i. e. 5 MB))|
+|`druid.emitter.http.batchQueueSizeLimit`|The maximum number of batches in emitter queue, if there are problems with emitting.|the maximum of (2) or (10% of the JVM heap size divided by 5MB)|
 |`druid.emitter.http.minHttpTimeoutMillis`|If the speed of filling batches imposes timeout smaller than that, not even trying to send batch to endpoint, because it will likely fail, not being able to send the data that fast. Configure this depending based on emitter/successfulSending/minTimeMs metric. Reasonable values are 10ms..100ms.|0|
 |`druid.emitter.http.recipientBaseUrl`|The base URL to emit messages to. Druid will POST JSON to be consumed at the HTTP endpoint specified by this property.|none, required config|
 
diff --git a/docs/content/development/extensions-core/druid-kerberos.md b/docs/content/development/extensions-core/druid-kerberos.md
index 1ded0e0f5a5..dc0d38a7b5a 100644
--- a/docs/content/development/extensions-core/druid-kerberos.md
+++ b/docs/content/development/extensions-core/druid-kerberos.md
@@ -31,8 +31,6 @@ The configuration examples in the rest of this document will use "kerberos" as t
 ### Properties
 |Property|Possible Values|Description|Default|required|
 |--------|---------------|-----------|-------|--------|
-|`druid.auth.authenticator.kerberos.internalClientPrincipal`|`druid@xxxxxxxxxxx`| Principal user name, used for internal node communication|empty|Yes|
-|`druid.auth.authenticator.kerberos.internalClientKeytab`|`/etc/security/keytabs/druid.keytab`|Path to keytab file used for internal node communication|empty|Yes|
 |`druid.auth.authenticator.kerberos.serverPrincipal`|`HTTP/_HOST@xxxxxxxxxxx`| SPNego service principal used by druid nodes|empty|Yes|
 |`druid.auth.authenticator.kerberos.serverKeytab`|`/etc/security/keytabs/spnego.service.keytab`|SPNego service keytab used by druid nodes|empty|Yes|
 |`druid.auth.authenticator.kerberos.authToLocal`|`RULE:[1:$1@$0](druid@xxxxxxxxxxx)s/.*/druid DEFAULT`|It allows you to set a general rule for mapping principal names to local user names. It will be used if there is not an explicit mapping for the principal name that is being translated.|DEFAULT|No|
@@ -54,6 +52,17 @@ In Active Directory environment, SPNEGO token in the Authorization header includ
 which includes all security groups for the user. In some cases when the user belongs to many security groups the header to grow beyond what druid can handle by default.
 In such cases, max request header size that druid can handle can be increased by setting `druid.server.http.maxRequestHeaderSize` (default 8Kb) and `druid.router.http.maxRequestBufferSize` (default 8Kb).
 
+## Configuring Kerberos Escalated Client 
+
+Druid internal nodes communicate with each other using an escalated http Client. A Kerberos enabled escalated HTTP Client can be configured by following properties -  
+
+
+|Property|Example Values|Description|Default|required|
+|--------|---------------|-----------|-------|--------|
+|`druid.escalator.type`|`kerberos`| Type of Escalator client used for internal node communication.|n/a|Yes|
+|`druid.escalator.internalClientPrincipal`|`druid@xxxxxxxxxxx`| Principal user name, used for internal node communication|n/a|Yes|
+|`druid.escalator.internalClientKeytab`|`/etc/security/keytabs/druid.keytab`|Path to keytab file used for internal node communication|n/a|Yes|
+|`druid.escalator.authorizerName`|`MyBasicAuthorizer`|Authorizer that requests should be directed to.|n/a|Yes|
 
 ## Accessing Druid HTTP end points when kerberos security is enabled 
 1. To access druid HTTP endpoints via curl user will need to first login using `kinit` command as follows -  
diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md
index b32c479fe7b..c254aad114e 100644
--- a/docs/content/development/extensions-core/kafka-ingestion.md
+++ b/docs/content/development/extensions-core/kafka-ingestion.md
@@ -311,40 +311,17 @@ In this way, configuration changes can be applied without requiring any pause in
 
 ### On the Subject of Segments
 
-The Kafka indexing service may generate a significantly large number of segments which over time will cause query
-performance issues if not properly managed. One important characteristic to understand is that the Kafka indexing task
-will generate a Druid partition in each segment granularity interval for each partition in the Kafka topic. As an
-example, if you are ingesting realtime data and your segment granularity is 15 minutes with 10 partitions in the Kafka
-topic, you would generate a minimum of 40 segments an hour. This is a limitation imposed by the Kafka architecture which
-guarantees delivery order within a partition but not across partitions. Therefore as a consumer of Kafka, in order to
-generate segments deterministically (and be able to provide exactly-once ingestion semantics) partitions need to be
-handled separately.
-
-Compounding this, if your taskDuration was also set to 15 minutes, you would actually generate 80 segments an hour since
-any given 15 minute interval would be handled by two tasks. For an example of this behavior, let's say we started the
-supervisor at 9:05 with a 15 minute segment granularity. The first task would create a segment for 9:00-9:15 and a
-segment for 9:15-9:30 before stopping at 9:20. A second task would be created at 9:20 which would create another segment
-for 9:15-9:30 and a segment for 9:30-9:45 before stopping at 9:35. Hence, if taskDuration and segmentGranularity are the
-same duration, you will get two tasks generating a segment for each segment granularity interval.
-
-Understanding this behavior is the first step to managing the number of segments produced. Some recommendations for
-keeping the number of segments low are:
-
-  * Keep the number of Kafka partitions to the minimum required to sustain the required throughput for your event streams.
-  * Increase segment granularity and task duration so that more events are written into the same segment. One
-    consideration here is that segments are only handed off to historical nodes after the task duration has elapsed.
-    Since workers tend to be configured with less query-serving resources than historical nodes, query performance may
-    suffer if tasks run excessively long without handing off segments.
-
-In many production installations which have been ingesting events for a long period of time, these suggestions alone
-will not be sufficient to keep the number of segments at an optimal level. It is recommended that scheduled re-indexing
-tasks be run to merge segments together into new segments of an ideal size (in the range of ~500-700 MB per segment).
-Currently, the recommended way of doing this is by running periodic Hadoop batch ingestion jobs and using a `dataSource`
-inputSpec to read from the segments generated by the Kafka indexing tasks. Details on how to do this can be found under
-['Updating Existing Data'](../../ingestion/update-existing-data.html). Note that the Merge Task and Append Task described
-[here](../../ingestion/tasks.html) will not work as they require unsharded segments while Kafka indexing tasks always
-generated sharded segments.
-
-There is ongoing work to support automatic segment compaction of sharded segments as well as compaction not requiring
-Hadoop (see [here](https://github.com/druid-io/druid/pull/1998) and [here](https://github.com/druid-io/druid/pull/3611)
-for related PRs).
+Each Kafka Indexing Task puts events consumed from Kafka partitions assigned to it in a single segment for each segment
+granular interval until maxRowsPerSegment limit is reached, at this point a new partition for this segment granularity is
+created for further events. Kafka Indexing Task also does incremental hand-offs which means that all the segments created by a
+task will not be held up till the task duration is over. As soon as maxRowsPerSegment limit is hit, all the segments held
+by the task at that point in time will be handed-off and new set of segments will be created for further events.
+This means that the task can run for longer durations of time without accumulating old segments locally on Middle Manager
+nodes and it is encouraged to do so.
+
+Kafka Indexing Service may still produce some small segments. Lets say the task duration is 4 hours, segment granularity
+is set to an HOUR and Supervisor was started at 9:10 then after 4 hours at 13:10, new set of tasks will be started and
+events for the interval 13:00 - 14:00 may be split across previous and new set of tasks. If you see it becoming a problem then
+one can schedule re-indexing tasks be run to merge segments together into new segments of an ideal size (in the range of ~500-700 MB per segment).
+There is also ongoing work to support automatic segment compaction of sharded segments as well as compaction not requiring
+Hadoop (see [here](https://github.com/druid-io/druid/pull/5102)).
diff --git a/docs/content/development/extensions-core/lookups-cached-global.md b/docs/content/development/extensions-core/lookups-cached-global.md
index 7e4b11af843..a5e7d5eb04f 100644
--- a/docs/content/development/extensions-core/lookups-cached-global.md
+++ b/docs/content/development/extensions-core/lookups-cached-global.md
@@ -71,7 +71,7 @@ The parameters are as follows
 |--------|-----------|--------|-------|
 |`extractionNamespace`|Specifies how to populate the local cache. See below|Yes|-|
 |`firstCacheTimeout`|How long to wait (in ms) for the first run of the cache to populate. 0 indicates to not wait|No|`0` (do not wait)|
-|`injective`|If the underlying map is injective (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`|
+|`injective`|If the underlying map is [injective](../../querying/lookups.html#query-execution) (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`|
 
 If `firstCacheTimeout` is set to a non-zero value, it should be less than `druid.manager.lookups.hostUpdateTimeout`. If `firstCacheTimeout` is NOT set, then management is essentially asynchronous and does not know if a lookup succeeded or failed in starting. In such a case logs from the lookup nodes should be monitored for repeated failures.
 
diff --git a/docs/content/development/extensions-core/mysql.md b/docs/content/development/extensions-core/mysql.md
index eb03af6af86..5314c07eef5 100644
--- a/docs/content/development/extensions-core/mysql.md
+++ b/docs/content/development/extensions-core/mysql.md
@@ -53,3 +53,23 @@ Make sure to [include](../../operations/including-extensions.html) `mysql-metada
   packaged in a separate tarball that can be downloaded from [here](http://druid.io/downloads.html).
   You can also get it using [pull-deps](../../operations/pull-deps.html), or you can build
   it from source code; see [Build from Source](../build.html).
+
+
+## Encrypting MySQL connections
+  This extension provides support for encrypting MySQL connections. To get more information about encrypting MySQL connections using TLS/SSL in general, please refer to this [guide](https://dev.mysql.com/doc/refman/5.7/en/using-encrypted-connections.html).
+
+## Configuration
+
+|Property|Description|Default|Required|
+|--------|-----------|-------|--------|
+|`druid.metadata.mysql.ssl.useSSL`|Enable SSL|`false`|no|
+|`druid.metadata.mysql.ssl.clientCertificateKeyStoreUrl`|The file path URL to the client certificate key store.|none|no|
+|`druid.metadata.mysql.ssl.clientCertificateKeyStoreType`|The type of the key store where the client certificate is stored.|none|no|
+|`druid.metadata.mysql.ssl.clientCertificateKeyStorePassword`|The [Password Provider](../operations/password-provider.html) or String password for the client key store.|none|no|
+|`druid.metadata.mysql.ssl.verifyServerCertificate`|Enables server certificate verification.|false|no|
+|`druid.metadata.mysql.ssl.trustCertificateKeyStoreUrl`|The file path to the trusted root certificate key store.|Default trust store provided by MySQL|yes if `verifyServerCertificate` is set to true and a custom trust store is used|
+|`druid.metadata.mysql.ssl.trustCertificateKeyStoreType`|The type of the key store where trusted root certificates are stored.|JKS|yes if `verifyServerCertificate` is set to true and keystore type is not JKS|
+|`druid.metadata.mysql.ssl.trustCertificateKeyStorePassword`|The [Password Provider](../operations/password-provider.html) or String password for the trust store.|none|yes if `verifyServerCertificate` is set to true and password is not null|
+|`druid.metadata.mysql.ssl.enabledSSLCipherSuites`|Overrides the existing cipher suites with these cipher suites.|none|no|
+|`druid.metadata.mysql.ssl.enabledTLSProtocols`|Overrides the TLS protocols with these protocols.|none|no|
+
diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md
index bc728306042..a9e1d1ee2f0 100644
--- a/docs/content/development/extensions.md
+++ b/docs/content/development/extensions.md
@@ -22,6 +22,7 @@ Core extensions are maintained by Druid committers.
 |Name|Description|Docs|
 |----|-----------|----|
 |druid-avro-extensions|Support for data in Apache Avro data format.|[link](../development/extensions-core/avro.html)|
+|druid-basic-security|Support for Basic HTTP authentication and role-based access control.|[link](../development/extensions-core/druid-basic-security.html)|
 |druid-caffeine-cache|A local cache implementation backed by Caffeine.|[link](../development/extensions-core/caffeine-cache.html)|
 |druid-datasketches|Support for approximate counts and set operations with [DataSketches](http://datasketches.github.io/).|[link](../development/extensions-core/datasketches-aggregators.html)|
 |druid-hdfs-storage|HDFS deep storage.|[link](../development/extensions-core/hdfs.html)|
diff --git a/docs/content/development/modules.md b/docs/content/development/modules.md
index be3773bfaf1..db529f3029a 100644
--- a/docs/content/development/modules.md
+++ b/docs/content/development/modules.md
@@ -110,7 +110,7 @@ The following example was retrieved from a historical node configured to use Azu
 00Z_2015-04-14T02:41:09.484Z
 2015-04-14T02:42:33,463 INFO [ZkCoordinator-0] io.druid.guice.JsonConfigurator - Loaded class[class io.druid.storage.azure.AzureAccountConfig] from props[drui
 d.azure.] as [io.druid.storage.azure.AzureAccountConfig@759c9ad9]
-2015-04-14T02:49:08,275 INFO [ZkCoordinator-0] com.metamx.common.CompressionUtils - Unzipping file[/opt/druid/tmp/compressionUtilZipCache1263964429587449785.z
+2015-04-14T02:49:08,275 INFO [ZkCoordinator-0] io.druid.java.util.common.CompressionUtils - Unzipping file[/opt/druid/tmp/compressionUtilZipCache1263964429587449785.z
 ip] to [/opt/druid/zk_druid/dde/2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z/2015-04-14T02:41:09.484Z/0]
 2015-04-14T02:49:08,276 INFO [ZkCoordinator-0] io.druid.storage.azure.AzureDataSegmentPuller - Loaded 1196 bytes from [dde/2015-01-02T00:00:00.000Z_2015-01-03
 T00:00:00.000Z/2015-04-14T02:41:09.484Z/0/index.zip] to [/opt/druid/zk_druid/dde/2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z/2015-04-14T02:41:09.484Z/0]
diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md
index b5b1bc1c543..4d424efcadb 100644
--- a/docs/content/ingestion/tasks.md
+++ b/docs/content/ingestion/tasks.md
@@ -276,8 +276,10 @@ An example of compaction task is
 }
 ```
 
-This compaction task merges _all segments_ of the interval `2017-01-01/2018-01-01` into a _single segment_.
-To merge each day's worth of data into a separate segment, you can submit multiple `compact` tasks, one for each day. They will run in parallel.
+This compaction task reads _all segments_ of the interval `2017-01-01/2018-01-01` and results in new segments.
+Note that intervals of the input segments are merged into a single interval of `2017-01-01/2018-01-01` no matter what the segmentGranularity was.
+To controll the number of result segments, you can set `targetPartitionSize` or `numShards`. See [indexTuningConfig](#tuningconfig) for more details.
+To merge each day's worth of data into separate segments, you can submit multiple `compact` tasks, one for each day. They will run in parallel.
 
 A compaction task internally generates an `index` task spec for performing compaction work with some fixed parameters.
 For example, its `firehose` is always the [ingestSegmentSpec](./firehose.html), and `dimensionsSpec` and `metricsSpec`
diff --git a/docs/content/misc/math-expr.md b/docs/content/misc/math-expr.md
index 07dcc501808..abcebdd3b5e 100644
--- a/docs/content/misc/math-expr.md
+++ b/docs/content/misc/math-expr.md
@@ -62,7 +62,7 @@ The following built-in functions are available.
 |timestamp_ceil|timestamp_ceil(expr, period, \[origin, \[timezone\]\]) rounds up a timestamp, returning it as a new timestamp. Period can be any ISO8601 period, like P3M (quarters) or PT12H (half-days). The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|
 |timestamp_floor|timestamp_floor(expr, period, \[origin, [timezone\]\]) rounds down a timestamp, returning it as a new timestamp. Period can be any ISO8601 period, like P3M (quarters) or PT12H (half-days). The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|
 |timestamp_shift|timestamp_shift(expr, period, step, \[timezone\]) shifts a timestamp by a period (step times), returning it as a new timestamp. Period can be any ISO8601 period. Step may be negative. The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|
-|timestamp_extract|timestamp_extract(expr, unit, \[timezone\]) extracts a time part from expr, returning it as a number. Unit can be EPOCH, SECOND, MINUTE, HOUR, DAY (day of month), DOW (day of week), DOY (day of year), WEEK (week of [week year](https://en.wikipedia.org/wiki/ISO_week_date)), MONTH (1 through 12), QUARTER (1 through 4), or YEAR. The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00"|
+|timestamp_extract|timestamp_extract(expr, unit, \[timezone\]) extracts a time part from expr, returning it as a number. Unit can be EPOCH (number of seconds since 1970-01-01 00:00:00 UTC), SECOND, MINUTE, HOUR, DAY (day of month), DOW (day of week), DOY (day of year), WEEK (week of [week year](https://en.wikipedia.org/wiki/ISO_week_date)), MONTH (1 through 12), QUARTER (1 through 4), or YEAR. The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00"|
 |timestamp_parse|timestamp_parse(string expr, \[pattern, [timezone\]\]) parses a string into a timestamp using a given [Joda DateTimeFormat pattern](http://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html). If the pattern is not provided, this parses time strings in either ISO8601 or SQL format. The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00", and will be used as the time zone for strings that do not include a time zone offset. Pattern and time zone must be literals. Strings that cannot be parsed as timestamps will be returned as nulls.|
 |timestamp_format|timestamp_format(expr, \[pattern, \[timezone\]\]) formats a timestamp as a string with a given [Joda DateTimeFormat pattern](http://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html), or ISO8601 if the pattern is not provided. The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00". Pattern and time zone must be literals.|
 
diff --git a/docs/content/operations/insert-segment-to-db.md b/docs/content/operations/insert-segment-to-db.md
index 60549e69d6c..cd5ba2264ae 100644
--- a/docs/content/operations/insert-segment-to-db.md
+++ b/docs/content/operations/insert-segment-to-db.md
@@ -5,24 +5,43 @@ layout: doc_page
 
 `insert-segment-to-db` is a tool that can insert segments into Druid metadata storage. It is intended to be used
 to update the segment table in metadata storage after people manually migrate segments from one place to another.
-It can also be used to insert missing segment into Druid, or even recover metadata storage by telling it where the
+It can also be used to insert missing segments into Druid, or even recover metadata storage by telling it where the
 segments are stored.
 
-Note: This tool expects users to have Druid cluster running in a "safe" mode, where there are no active tasks to interfere
-the segments being inserted. Users can optionally bring down the cluster to make 100% sure nothing is interfering.
+**Note:** This tool simply scans the deep storage directory to reconstruct the metadata entries used to locate and
+identify each segment. It does not have any understanding about whether those segments _should actually_ be written to
+the metadata storage. In certain cases, this can lead to undesired or inconsistent results. Some examples of things to
+watch out for:
+  - Dropped datasources will be re-enabled.
+  - The latest version of each segment set will be loaded by Druid, which in some cases may not be the version you
+    actually want. An example of this is a bad compaction job that generates segments which need to be manually rolled
+    back by removing that version from the metadata table. If these segments are not also removed from deep storage,
+    they will be imported back into the metadata table and overshadow the correct version.
+  - Some indexers such as the Kafka indexing service have the potential to generate more than one set of segments that
+    have the same segment ID but different contents. When the metadata is first written, the correct set of segments is
+    referenced and the other set is normally deleted from deep storage. It is possible however that an unhandled
+    exception could result in multiple sets of segments with the same segment ID remaining in deep storage. Since this
+    tool does not know which one is the 'correct' one to use, it will simply select the newest segment set and ignore
+    the other versions. If the wrong segment set is picked, the exactly-once semantics of the Kafka indexing service
+    will no longer hold true and you may get duplicated or dropped events.
+
+With these considerations in mind, it is recommended that data migrations be done by exporting the original metadata
+storage directly, since that is the definitive cluster state. This tool should be used as a last resort when a direct
+export is not possible.
+
+**Note:** This tool expects users to have Druid cluster running in a "safe" mode, where there are no active tasks to interfere
+with the segments being inserted. Users can optionally bring down the cluster to make 100% sure nothing is interfering.
 
 In order to make it work, user will have to provide metadata storage credentials and deep storage type through Java JVM argument
-or runtime.properties file. Specifically, this tool needs to know
+or runtime.properties file. Specifically, this tool needs to know:
 
-`druid.metadata.storage.type`
-
-`druid.metadata.storage.connector.connectURI`
-
-`druid.metadata.storage.connector.user`
-
-`druid.metadata.storage.connector.password`
-
-`druid.storage.type`
+```
+druid.metadata.storage.type
+druid.metadata.storage.connector.connectURI
+druid.metadata.storage.connector.user
+druid.metadata.storage.connector.password
+druid.storage.type
+```
 
 Besides the properties above, you also need to specify the location where the segments are stored and whether you want to
 update descriptor.json (`partitionNum_descriptor.json` for HDFS data storage). These two can be provided through command line arguments.
diff --git a/docs/content/operations/performance-faq.md b/docs/content/operations/performance-faq.md
index edeaf77d8ae..1c27c0d3eb9 100644
--- a/docs/content/operations/performance-faq.md
+++ b/docs/content/operations/performance-faq.md
@@ -64,7 +64,7 @@ Yes, using a `log4j2.xml` similar to the following causes some of the more chatt
     <AsyncLogger name="io.druid.client.ServerInventoryView" level="debug" additivity="false">
       <AppenderRef ref="Console"/>
     </AsyncLogger>
-    <AsyncLogger name ="com.metamx.http.client.pool.ChannelResourceFactory" level="info" additivity="false">
+    <AsyncLogger name ="io.druid.java.util.http.client.pool.ChannelResourceFactory" level="info" additivity="false">
       <AppenderRef ref="Console"/>
     </AsyncLogger>
     <Root level="info">
diff --git a/docs/content/querying/dimensionspecs.md b/docs/content/querying/dimensionspecs.md
index 236e13b5a7a..19d080493fe 100644
--- a/docs/content/querying/dimensionspecs.md
+++ b/docs/content/querying/dimensionspecs.md
@@ -95,8 +95,6 @@ The default values are `replaceMissingValueWith = null` and `retainMissingValue
 
 It is illegal to set `retainMissingValue = true` and also specify a `replaceMissingValueWith`.
 
-A property of `injective` specifies if optimizations can be used which assume there is no combining of multiple names into one. For example: If ABC123 is the only key that maps to SomeCompany, that can be optimized since it is a unique lookup. But if both ABC123 and DEF456 BOTH map to SomeCompany, then that is NOT a unique lookup. Setting this value to true and setting `retainMissingValue` to FALSE (the default) may cause undesired behavior.
-
 A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`).
 
 The second kind where it is not possible to pass at query time due to their size, will be based on an external lookup table or resource that is already registered via configuration file or/and coordinator.
@@ -316,67 +314,38 @@ Example for the `__time` dimension:
 JavaScript-based functionality is disabled by default. Please refer to the Druid <a href="../development/javascript.html">JavaScript programming guide</a> for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
 </div>
 
-### Lookup extraction function
+### Registered lookup extraction function
 
-Lookups are a concept in Druid where dimension values are (optionally) replaced with new values. 
-For more documentation on using lookups, please see [here](../querying/lookups.html). 
-Explicit lookups allow you to specify a set of keys and values to use when performing the extraction.
+Lookups are a concept in Druid where dimension values are (optionally) replaced with new values.
+For more documentation on using lookups, please see [Lookups](../querying/lookups.html).
+The "registeredLookup" extraction function lets you refer to a lookup that has been registered in the cluster-wide
+configuration.
 
-```json
-{
-  "type":"lookup",
-  "lookup":{
-    "type":"map",
-    "map":{"foo":"bar", "baz":"bat"}
-  },
-  "retainMissingValue":true,
-  "injective":true
-}
-```
-
-```json
-{
-  "type":"lookup",
-  "lookup":{
-    "type":"map",
-    "map":{"foo":"bar", "baz":"bat"}
-  },
-  "retainMissingValue":false,
-  "injective":false,
-  "replaceMissingValueWith":"MISSING"
-}
-```
-
-```json
-{
-  "type":"lookup",
-  "lookup":{"type":"namespace","namespace":"some_lookup"},
-  "replaceMissingValueWith":"Unknown",
-  "injective":false
-}
-```
+An example:
 
 ```json
 {
-  "type":"lookup",
-  "lookup":{"type":"namespace","namespace":"some_lookup"},
-  "retainMissingValue":true,
-  "injective":false
+  "type":"registeredLookup",
+  "lookup":"some_lookup_name",
+  "retainMissingValue":true
 }
 ```
 
-A lookup can be of type `namespace` or `map`. A `map` lookup is passed as part of the query. 
-A `namespace` lookup is populated on all the nodes which handle queries as per [lookups](../querying/lookups.html)
+A property of `retainMissingValue` and `replaceMissingValueWith` can be specified at query time to hint how to handle
+missing values. Setting `replaceMissingValueWith` to `""` has the same effect as setting it to `null` or omitting the
+property. Setting `retainMissingValue` to true will use the dimension's original value if it is not found in the lookup.
+The default values are `replaceMissingValueWith = null` and `retainMissingValue = false` which causes missing values to
+be treated as missing.
 
-A property of `retainMissingValue` and `replaceMissingValueWith` can be specified at query time to hint how to handle missing values. Setting `replaceMissingValueWith` to `""` has the same effect as setting it to `null` or omitting the property. Setting `retainMissingValue` to true will use the dimension's original value if it is not found in the lookup. The default values are `replaceMissingValueWith = null` and `retainMissingValue = false` which causes missing values to be treated as missing.
- 
 It is illegal to set `retainMissingValue = true` and also specify a `replaceMissingValueWith`.
 
-A property of `injective` specifies if optimizations can be used which assume there is no combining of multiple names into one. For example: If ABC123 is the only key that maps to SomeCompany, that can be optimized since it is a unique lookup. But if both ABC123 and DEF456 BOTH map to SomeCompany, then that is NOT a unique lookup. Setting this value to true and setting `retainMissingValue` to FALSE (the default) may cause undesired behavior.
+A property of `injective` can override the lookup's own sense of whether or not it is
+[injective](lookups.html#query-execution). If left unspecified, Druid will use the registered cluster-wide lookup
+configuration.
 
-A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`). 
+A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`).
 The optimization layer will run on the broker and it will rewrite the extraction filter as clause of selector filters.
-For instance the following filter 
+For instance the following filter
 
 ```json
 {
@@ -385,21 +354,16 @@ For instance the following filter
         "dimension": "product",
         "value": "bar_1",
         "extractionFn": {
-            "type": "lookup",
+            "type": "registeredLookup",
             "optimize": true,
-            "lookup": {
-                "type": "map",
-                "map": {
-                    "product_1": "bar_1",
-                    "product_3": "bar_1"
-                }
-            }
+            "lookup": "some_lookup_name"
         }
     }
 }
 ```
 
-will be rewritten as
+will be rewritten as the following simpler query, assuming a lookup that maps "product_1" and "product_3" to the value
+"bar_1":
 
 ```json
 {
@@ -425,30 +389,49 @@ will be rewritten as
 }
 ```
 
-A null dimension value can be mapped to a specific value by specifying the empty string as the key.
+A null dimension value can be mapped to a specific value by specifying the empty string as the key in your lookup file.
 This allows distinguishing between a null dimension and a lookup resulting in a null.
 For example, specifying `{"":"bar","bat":"baz"}` with dimension values `[null, "foo", "bat"]` and replacing missing values with `"oof"` will yield results of `["bar", "oof", "baz"]`.
 Omitting the empty string key will cause the missing value to take over. For example, specifying `{"bat":"baz"}` with dimension values `[null, "foo", "bat"]` and replacing missing values with `"oof"` will yield results of `["oof", "oof", "baz"]`.
 
-### Registered Lookup Extraction Function
+### Inline lookup extraction function
 
-While it is recommended that the [lookup dimension spec](#lookup-dimensionspecs) be used whenever possible, any lookup that is registered for use as a lookup dimension spec can be used as a dimension extraction.
+Lookups are a concept in Druid where dimension values are (optionally) replaced with new values.
+For more documentation on using lookups, please see [Lookups](../querying/lookups.html).
+The "lookup" extraction function lets you specify an inline lookup map without registering one in the cluster-wide
+configuration.
 
-The specification for dimension extraction using dimension specification named lookups is formatted as per the following example:
+Examples:
 
 ```json
 {
-  "type":"registeredLookup",
-  "lookup":"some_lookup_name",
+  "type":"lookup",
+  "lookup":{
+    "type":"map",
+    "map":{"foo":"bar", "baz":"bat"}
+  },
   "retainMissingValue":true,
-  "injective":false
+  "injective":true
+}
+```
+
+```json
+{
+  "type":"lookup",
+  "lookup":{
+    "type":"map",
+    "map":{"foo":"bar", "baz":"bat"}
+  },
+  "retainMissingValue":false,
+  "injective":false,
+  "replaceMissingValueWith":"MISSING"
 }
 ```
 
-All the flags for [lookup extraction function](#lookup-extraction-function) apply here as well.
+The inline lookup should be of type `map`.
 
-In general, the dimension specification should be used. This dimension **extraction** implementation is made available for testing, validation, and transitioning from dimension extraction to the dimension specification style lookups.
-There is also a chance that a feature uses dimension extraction in such a way that it is not applied to dimension specification lookups. Such a scenario should be brought to the attention of the development mailing list.
+The properties `retainMissingValue`, `replaceMissingValueWith`, `injective`, and `optimize` behave similarly to the
+[registered lookup extraction function](#registered-lookup-extraction-function).
 
 ### Cascade Extraction Function
 
diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md
index 616b602eb3b..771c9adfb74 100644
--- a/docs/content/querying/lookups.md
+++ b/docs/content/querying/lookups.md
@@ -8,15 +8,17 @@ layout: doc_page
 Lookups are an <a href="../development/experimental.html">experimental</a> feature.
 </div>
 
-Lookups are a concept in Druid where dimension values are (optionally) replaced with new values. 
-See [dimension specs](../querying/dimensionspecs.html) for more information. For the purpose of these documents, 
-a "key" refers to a dimension value to match, and a "value" refers to its replacement. 
-So if you wanted to rename `appid-12345` to `Super Mega Awesome App` then the key would be `appid-12345` and the value 
-would be `Super Mega Awesome App`. 
-
-It is worth noting that lookups support use cases where keys map to unique values (injective) such as a country code and 
-a country name, and also supports use cases where multiple IDs map to the same value, e.g. multiple app-ids belonging to 
-a single account manager.
+Lookups are a concept in Druid where dimension values are (optionally) replaced with new values, allowing join-like
+functionality. Applying lookups in Druid is similar to joining a dimension table in a data warehouse. See
+[dimension specs](../querying/dimensionspecs.html) for more information. For the purpose of these documents, a "key"
+refers to a dimension value to match, and a "value" refers to its replacement. So if you wanted to map
+`appid-12345` to `Super Mega Awesome App` then the key would be `appid-12345` and the value would be
+`Super Mega Awesome App`.
+
+It is worth noting that lookups support not just use cases where keys map one-to-one to unique values, such as country
+code and country name, but also support use cases where multiple IDs map to the same value, e.g. multiple app-ids
+mapping to a single account manager. When lookups are one-to-one, Druid is able to apply additional optimizations at
+query time; see [Query execution](#query-execution) below for more details.
 
 Lookups do not have history. They always use the current data. This means that if the chief account manager for a 
 particular app-id changes, and you issue a query with a lookup to store the app-id to account manager relationship, 
@@ -33,6 +35,38 @@ Other lookup types are available as extensions, including:
 - Globally cached lookups from local files, remote URIs, or JDBC through [lookups-cached-global](../development/extensions-core/lookups-cached-global.html).
 - Globally cached lookups from a Kafka topic through [kafka-extraction-namespace](../development/extensions-core/kafka-extraction-namespace.html).
 
+Query Execution
+---------------
+When executing an aggregation query involving lookups, Druid can decide to apply lookups either while scanning and
+aggregating rows, or to apply them after aggregation is complete. It is more efficient to apply lookups after
+aggregation is complete, so Druid will do this if it can. Druid decides this by checking if the lookup is marked
+as "injective" or not. In general, you should set this property for any lookup that is naturally one-to-one, to allow
+Druid to run your queries as fast as possible.
+
+Injective lookups should include _all_ possible keys that may show up in your dataset, and should also map all keys to
+_unique values_. This matters because non-injective lookups may map different keys to the same value, which must be
+accounted for during aggregation, lest query results contain two result values that should have been aggregated into
+one.
+
+This lookup is injective (assuming it contains all possible keys from your data):
+
+```
+1 -> Foo
+2 -> Bar
+3 -> Billy
+```
+
+But this one is not, since both "2" and "3" map to the same key:
+
+```
+1 -> Foo
+2 -> Bar
+3 -> Bar
+```
+
+To tell Druid that your lookup is injective, you must specify `"injective" : true` in the lookup configuration. Druid
+will not detect this automatically.
+
 Dynamic Configuration
 ---------------------
 <div class="note caution">
diff --git a/docs/content/toc.md b/docs/content/toc.md
index d18de0e7c73..e51f95c05d4 100644
--- a/docs/content/toc.md
+++ b/docs/content/toc.md
@@ -90,6 +90,7 @@ layout: toc
   * [Broker](/docs/VERSION/configuration/broker.html)
   * [Realtime](/docs/VERSION/configuration/realtime.html)
   * [Configuring Logging](/docs/VERSION/configuration/logging.html)
+  * [Configuring Authentication and Authorization](/docs/VERSION/configuration/auth.html)
   
 ## Development
   * [Overview](/docs/VERSION/development/overview.html)
diff --git a/examples/conf-quickstart/druid/_common/common.runtime.properties b/examples/conf-quickstart/druid/_common/common.runtime.properties
index fd131b878cc..ee5b284b527 100644
--- a/examples/conf-quickstart/druid/_common/common.runtime.properties
+++ b/examples/conf-quickstart/druid/_common/common.runtime.properties
@@ -113,7 +113,7 @@ druid.selectors.coordinator.serviceName=druid/coordinator
 # Monitoring
 #
 
-druid.monitoring.monitors=["com.metamx.metrics.JvmMonitor"]
+druid.monitoring.monitors=["io.druid.java.util.metrics.JvmMonitor"]
 druid.emitter=logging
 druid.emitter.logging.logLevel=info
 
diff --git a/examples/conf/druid/_common/common.runtime.properties b/examples/conf/druid/_common/common.runtime.properties
index a018fa01780..1c967fcd60c 100644
--- a/examples/conf/druid/_common/common.runtime.properties
+++ b/examples/conf/druid/_common/common.runtime.properties
@@ -112,7 +112,7 @@ druid.selectors.coordinator.serviceName=druid/coordinator
 # Monitoring
 #
 
-druid.monitoring.monitors=["com.metamx.metrics.JvmMonitor"]
+druid.monitoring.monitors=["io.druid.java.util.metrics.JvmMonitor"]
 druid.emitter=logging
 druid.emitter.logging.logLevel=info
 
diff --git a/examples/pom.xml b/examples/pom.xml
index 231469f4b06..03f7754828c 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
     </parent>
 
     <dependencies>
diff --git a/extendedset/pom.xml b/extendedset/pom.xml
index 91f19144c8e..c2a37bc5f4a 100755
--- a/extendedset/pom.xml
+++ b/extendedset/pom.xml
@@ -18,8 +18,7 @@
   ~ under the License.
   -->
 
-<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; xmlns="http://maven.apache.org/POM/4.0.0";
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd";>
+<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>
 
   <artifactId>extendedset</artifactId>
@@ -32,7 +31,7 @@
   <parent>
     <groupId>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
   </parent>
 
   <dependencies>
diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml
index 14fe7cd44bb..23af2e5c705 100644
--- a/extensions-contrib/ambari-metrics-emitter/pom.xml
+++ b/extensions-contrib/ambari-metrics-emitter/pom.xml
@@ -18,14 +18,13 @@
  ~ 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";>
+<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>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
@@ -48,8 +47,9 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>com.metamx</groupId>
+      <groupId>io.druid</groupId>
       <artifactId>java-util</artifactId>
+      <version>${project.parent.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java
index 1838e8952c1..f8f2e76fa1f 100644
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java
+++ b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java
@@ -20,11 +20,11 @@
 package io.druid.emitter.ambari.metrics;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.metamx.common.ISE;
-import com.metamx.emitter.core.Emitter;
-import com.metamx.emitter.core.Event;
-import com.metamx.emitter.service.AlertEvent;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.Event;
+import io.druid.java.util.emitter.service.AlertEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.java.util.common.StringUtils;
 import io.druid.java.util.common.logger.Logger;
 import org.apache.hadoop.metrics2.sink.timeline.AbstractTimelineMetricsSink;
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitterModule.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitterModule.java
index ad029a08787..6dc5e49edc4 100644
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitterModule.java
+++ b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitterModule.java
@@ -28,7 +28,7 @@
 import com.google.inject.Provides;
 import com.google.inject.name.Named;
 import com.google.inject.name.Names;
-import com.metamx.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.Emitter;
 import io.druid.guice.JsonConfigProvider;
 import io.druid.guice.ManageLifecycle;
 import io.druid.initialization.DruidModule;
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/DruidToTimelineMetricConverter.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/DruidToTimelineMetricConverter.java
index bcc09f68905..9912ced6843 100644
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/DruidToTimelineMetricConverter.java
+++ b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/DruidToTimelineMetricConverter.java
@@ -21,7 +21,7 @@
 
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
 
 
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/SendAllTimelineEventConverter.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/SendAllTimelineEventConverter.java
index f5be89a2385..2189fac55f0 100644
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/SendAllTimelineEventConverter.java
+++ b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/SendAllTimelineEventConverter.java
@@ -26,7 +26,7 @@
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSortedSet;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
 
 /**
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverter.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverter.java
index 5d2a4629096..cf4bd86821e 100644
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverter.java
+++ b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverter.java
@@ -32,8 +32,8 @@
 import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.io.CharStreams;
 import com.google.common.io.Files;
-import com.metamx.common.ISE;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.java.util.common.logger.Logger;
 import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
 
diff --git a/extensions-contrib/ambari-metrics-emitter/src/test/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java b/extensions-contrib/ambari-metrics-emitter/src/test/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java
index eeb4048a9b4..be58f3c1934 100644
--- a/extensions-contrib/ambari-metrics-emitter/src/test/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java
+++ b/extensions-contrib/ambari-metrics-emitter/src/test/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java
@@ -20,7 +20,7 @@
 package io.druid.emitter.ambari.metrics;
 
 import com.google.common.collect.Maps;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.jackson.DefaultObjectMapper;
 import io.druid.java.util.common.DateTimes;
 import junitparams.JUnitParamsRunner;
diff --git a/extensions-contrib/azure-extensions/pom.xml b/extensions-contrib/azure-extensions/pom.xml
index 1c647b0e91d..e8996d7d3fa 100644
--- a/extensions-contrib/azure-extensions/pom.xml
+++ b/extensions-contrib/azure-extensions/pom.xml
@@ -18,8 +18,7 @@
  ~ 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";>
+<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>
 
     <groupId>io.druid.extensions.contrib</groupId>
@@ -30,7 +29,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPusher.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPusher.java
index 97bc1a0167a..592b956c260 100644
--- a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPusher.java
+++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPusher.java
@@ -25,7 +25,6 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.inject.Inject;
 import com.microsoft.azure.storage.StorageException;
-
 import io.druid.java.util.common.CompressionUtils;
 import io.druid.java.util.common.StringUtils;
 import io.druid.java.util.common.logger.Logger;
@@ -92,9 +91,9 @@ public File createSegmentDescriptorFile(final ObjectMapper jsonMapper, final Dat
     return descriptorFile;
   }
 
-  public Map<String, String> getAzurePaths(final DataSegment segment)
+  public Map<String, String> getAzurePaths(final DataSegment segment, final boolean useUniquePath)
   {
-    final String storageDir = this.getStorageDir(segment);
+    final String storageDir = this.getStorageDir(segment, useUniquePath);
 
     return ImmutableMap.of(
         "index", StringUtils.format("%s/%s", storageDir, AzureStorageDruidModule.INDEX_ZIP_FILE_NAME),
@@ -131,9 +130,9 @@ public DataSegment uploadDataSegment(
   }
 
   @Override
-  public DataSegment push(final File indexFilesDir, final DataSegment segment) throws IOException
+  public DataSegment push(final File indexFilesDir, final DataSegment segment, final boolean useUniquePath)
+      throws IOException
   {
-
     log.info("Uploading [%s] to Azure.", indexFilesDir);
 
     final int version = SegmentUtils.getVersionFromDir(indexFilesDir);
@@ -145,7 +144,7 @@ public DataSegment push(final File indexFilesDir, final DataSegment segment) thr
       final long size = CompressionUtils.zip(indexFilesDir, zipOutFile);
 
       final File descFile = descriptorFile = createSegmentDescriptorFile(jsonMapper, segment);
-      final Map<String, String> azurePaths = getAzurePaths(segment);
+      final Map<String, String> azurePaths = getAzurePaths(segment, useUniquePath);
 
       return AzureUtils.retryAzureOperation(
           new Callable<DataSegment>()
diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureStorage.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureStorage.java
index 8585bf04379..25a4764ad84 100644
--- a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureStorage.java
+++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureStorage.java
@@ -23,9 +23,7 @@
 import com.microsoft.azure.storage.blob.CloudBlob;
 import com.microsoft.azure.storage.blob.CloudBlobClient;
 import com.microsoft.azure.storage.blob.CloudBlobContainer;
-import com.microsoft.azure.storage.blob.CloudBlockBlob;
 import com.microsoft.azure.storage.blob.ListBlobItem;
-
 import io.druid.java.util.common.logger.Logger;
 
 import java.io.File;
@@ -78,17 +76,14 @@ public CloudBlobContainer getCloudBlobContainer(final String containerName)
     }
 
     return deletedFiles;
-
   }
 
   public void uploadBlob(final File file, final String containerName, final String blobPath)
       throws IOException, StorageException, URISyntaxException
-
   {
     CloudBlobContainer container = getCloudBlobContainer(containerName);
     try (FileInputStream stream = new FileInputStream(file)) {
-      CloudBlockBlob blob = container.getBlockBlobReference(blobPath);
-      blob.upload(stream, file.length());
+      container.getBlockBlobReference(blobPath).upload(stream, file.length());
     }
   }
 
diff --git a/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPusherTest.java b/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPusherTest.java
index af76f357142..3201c4afeca 100644
--- a/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPusherTest.java
+++ b/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPusherTest.java
@@ -82,6 +82,17 @@ public void before()
 
   @Test
   public void testPush() throws Exception
+  {
+    testPushInternal(false, "foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/index\\.zip");
+  }
+
+  @Test
+  public void testPushUseUniquePath() throws Exception
+  {
+    testPushInternal(true, "foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/[A-Za-z0-9-]{36}/index\\.zip");
+  }
+
+  private void testPushInternal(boolean useUniquePath, String matcher) throws Exception
   {
     AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, jsonMapper);
 
@@ -104,7 +115,12 @@ public void testPush() throws Exception
         size
     );
 
-    DataSegment segment = pusher.push(tempFolder.getRoot(), segmentToPush);
+    DataSegment segment = pusher.push(tempFolder.getRoot(), segmentToPush, useUniquePath);
+
+    Assert.assertTrue(
+        segment.getLoadSpec().get("blobPath").toString(),
+        segment.getLoadSpec().get("blobPath").toString().matches(matcher)
+    );
 
     Assert.assertEquals(segmentToPush.getSize(), segment.getSize());
   }
@@ -114,10 +130,13 @@ public void getAzurePathsTest()
   {
 
     AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, jsonMapper);
-    final String storageDir = pusher.getStorageDir(dataSegment);
-    Map<String, String> paths = pusher.getAzurePaths(dataSegment);
+    final String storageDir = pusher.getStorageDir(dataSegment, false);
+    Map<String, String> paths = pusher.getAzurePaths(dataSegment, false);
 
-    assertEquals(StringUtils.format("%s/%s", storageDir, AzureStorageDruidModule.INDEX_ZIP_FILE_NAME), paths.get("index"));
+    assertEquals(
+        StringUtils.format("%s/%s", storageDir, AzureStorageDruidModule.INDEX_ZIP_FILE_NAME),
+        paths.get("index")
+    );
     assertEquals(
         StringUtils.format("%s/%s", storageDir, AzureStorageDruidModule.DESCRIPTOR_FILE_NAME),
         paths.get("descriptor")
@@ -131,7 +150,7 @@ public void uploadDataSegmentTest() throws StorageException, IOException, URISyn
     final int version = 9;
     final File compressedSegmentData = new File("index.zip");
     final File descriptorFile = new File("descriptor.json");
-    final Map<String, String> azurePaths = pusher.getAzurePaths(dataSegment);
+    final Map<String, String> azurePaths = pusher.getAzurePaths(dataSegment, false);
 
     azureStorage.uploadBlob(compressedSegmentData, containerName, azurePaths.get("index"));
     expectLastCall();
diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml
index ea32b084b02..5100c57afa8 100644
--- a/extensions-contrib/cassandra-storage/pom.xml
+++ b/extensions-contrib/cassandra-storage/pom.xml
@@ -29,7 +29,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPusher.java b/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPusher.java
index 3595147ace8..16ff585327d 100644
--- a/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPusher.java
+++ b/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPusher.java
@@ -24,8 +24,9 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.inject.Inject;
 import com.netflix.astyanax.MutationBatch;
+import com.netflix.astyanax.connectionpool.exceptions.NotFoundException;
 import com.netflix.astyanax.recipes.storage.ChunkedStorage;
-
+import com.netflix.astyanax.recipes.storage.ChunkedStorageProvider;
 import io.druid.java.util.common.CompressionUtils;
 import io.druid.java.util.common.logger.Logger;
 import io.druid.segment.SegmentUtils;
@@ -53,7 +54,8 @@
   @Inject
   public CassandraDataSegmentPusher(
       CassandraDataSegmentConfig config,
-      ObjectMapper jsonMapper)
+      ObjectMapper jsonMapper
+  )
   {
     super(config);
     this.jsonMapper = jsonMapper;
@@ -73,13 +75,13 @@ public String getPathForHadoop(String dataSource)
   }
 
   @Override
-  public DataSegment push(final File indexFilesDir, DataSegment segment) throws IOException
+  public DataSegment push(final File indexFilesDir, DataSegment segment, final boolean useUniquePath) throws IOException
   {
     log.info("Writing [%s] to C*", indexFilesDir);
     String key = JOINER.join(
         config.getKeyspace().isEmpty() ? null : config.getKeyspace(),
-        this.getStorageDir(segment)
-        );
+        this.getStorageDir(segment, useUniquePath)
+    );
 
     // Create index
     final File compressedIndexFile = File.createTempFile("druid", "index.zip");
@@ -91,12 +93,12 @@ public DataSegment push(final File indexFilesDir, DataSegment segment) throws IO
     try {
       long start = System.currentTimeMillis();
       ChunkedStorage.newWriter(indexStorage, key, new FileInputStream(compressedIndexFile))
-          .withConcurrencyLevel(CONCURRENCY).call();
+                    .withConcurrencyLevel(CONCURRENCY).call();
       byte[] json = jsonMapper.writeValueAsBytes(segment);
       MutationBatch mutation = this.keyspace.prepareMutationBatch();
       mutation.withRow(descriptorStorage, key)
-        .putColumn("lastmodified", System.currentTimeMillis(), null)
-        .putColumn("descriptor", json, null);
+              .putColumn("lastmodified", System.currentTimeMillis(), null)
+              .putColumn("descriptor", json, null);
       mutation.execute();
       log.info("Wrote index to C* in [%s] ms", System.currentTimeMillis() - start);
     }
@@ -105,10 +107,8 @@ public DataSegment push(final File indexFilesDir, DataSegment segment) throws IO
     }
 
     segment = segment.withSize(indexSize)
-        .withLoadSpec(
-            ImmutableMap.<String, Object> of("type", "c*", "key", key)
-        )
-        .withBinaryVersion(version);
+                     .withLoadSpec(ImmutableMap.<String, Object>of("type", "c*", "key", key))
+                     .withBinaryVersion(version);
 
     log.info("Deleting zipped index File[%s]", compressedIndexFile);
     compressedIndexFile.delete();
@@ -120,4 +120,14 @@ public DataSegment push(final File indexFilesDir, DataSegment segment) throws IO
   {
     throw new UnsupportedOperationException("not supported");
   }
+
+  private boolean doesObjectExist(ChunkedStorageProvider provider, String objectName) throws Exception
+  {
+    try {
+      return ChunkedStorage.newInfoReader(provider, objectName).call().isValidForRead();
+    }
+    catch (NotFoundException e) {
+      return false;
+    }
+  }
 }
diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml
index da06a9e9e51..726e0bc916c 100644
--- a/extensions-contrib/cloudfiles-extensions/pom.xml
+++ b/extensions-contrib/cloudfiles-extensions/pom.xml
@@ -18,8 +18,7 @@
  ~ under the License.
   -->
 
-<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd";
-         xmlns="http://maven.apache.org/POM/4.0.0"; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";>
+<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>
 
     <groupId>io.druid.extensions.contrib</groupId>
@@ -30,7 +29,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java
index a08105b0575..be3e2547235 100644
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java
+++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java
@@ -31,7 +31,6 @@
 import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
 
 import java.io.File;
-import java.io.IOException;
 import java.net.URI;
 import java.nio.file.Files;
 import java.util.Map;
@@ -74,9 +73,12 @@ public String getPathForHadoop(final String dataSource)
   }
 
   @Override
-  public DataSegment push(final File indexFilesDir, final DataSegment inSegment) throws IOException
+  public DataSegment push(final File indexFilesDir, final DataSegment inSegment, final boolean useUniquePath)
   {
-    final String segmentPath = CloudFilesUtils.buildCloudFilesPath(this.config.getBasePath(), getStorageDir(inSegment));
+    final String segmentPath = CloudFilesUtils.buildCloudFilesPath(
+        this.config.getBasePath(),
+        getStorageDir(inSegment, useUniquePath)
+    );
 
     File descriptorFile = null;
     File zipOutFile = null;
@@ -98,6 +100,7 @@ public DataSegment call() throws Exception
                   segmentPath, outFile, objectApi.getRegion(),
                   objectApi.getContainer()
               );
+
               log.info("Pushing %s.", segmentData.getPath());
               objectApi.put(segmentData);
 
@@ -111,6 +114,7 @@ public DataSegment call() throws Exception
               log.info("Pushing %s.", descriptorData.getPath());
               objectApi.put(descriptorData);
 
+
               final DataSegment outSegment = inSegment
                   .withSize(indexSize)
                   .withLoadSpec(makeLoadSpec(new URI(segmentData.getPath())))
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesObjectApiProxy.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesObjectApiProxy.java
index 7f41497b37c..d495fcc1c77 100644
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesObjectApiProxy.java
+++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesObjectApiProxy.java
@@ -58,4 +58,9 @@ public CloudFilesObject get(String path)
     Payload payload = swiftObject.getPayload();
     return new CloudFilesObject(payload, this.region, this.container, path);
   }
+
+  public boolean exists(String path)
+  {
+    return objectApi.getWithoutBody(path) != null;
+  }
 }
diff --git a/extensions-contrib/cloudfiles-extensions/src/test/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusherTest.java b/extensions-contrib/cloudfiles-extensions/src/test/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusherTest.java
index b257efb1282..2be6f584811 100644
--- a/extensions-contrib/cloudfiles-extensions/src/test/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusherTest.java
+++ b/extensions-contrib/cloudfiles-extensions/src/test/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusherTest.java
@@ -84,7 +84,7 @@ public void testPush() throws Exception
         size
     );
 
-    DataSegment segment = pusher.push(tempFolder.getRoot(), segmentToPush);
+    DataSegment segment = pusher.push(tempFolder.getRoot(), segmentToPush, false);
 
     Assert.assertEquals(segmentToPush.getSize(), segment.getSize());
 
diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml
index 86c3fcc09c9..54aefa78436 100644
--- a/extensions-contrib/distinctcount/pom.xml
+++ b/extensions-contrib/distinctcount/pom.xml
@@ -18,8 +18,7 @@
  ~ 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";>
+<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>
 
     <groupId>io.druid.extensions.contrib</groupId>
@@ -30,7 +29,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-contrib/druid-rocketmq/pom.xml b/extensions-contrib/druid-rocketmq/pom.xml
index 924a07cdd42..02f9519cc83 100644
--- a/extensions-contrib/druid-rocketmq/pom.xml
+++ b/extensions-contrib/druid-rocketmq/pom.xml
@@ -17,14 +17,12 @@
  ~ 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";>
+<project xmlns="http://maven.apache.org/POM/4.0.0"; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd";>
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <artifactId>druid</artifactId>
     <groupId>io.druid</groupId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/extensions-contrib/google-extensions/pom.xml b/extensions-contrib/google-extensions/pom.xml
index 20a3dd7ba3d..dc50b943f77 100644
--- a/extensions-contrib/google-extensions/pom.xml
+++ b/extensions-contrib/google-extensions/pom.xml
@@ -29,7 +29,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPusher.java b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPusher.java
index 527e7e14025..7a3581462d3 100644
--- a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPusher.java
+++ b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPusher.java
@@ -93,7 +93,8 @@ public File createDescriptorFile(final ObjectMapper jsonMapper, final DataSegmen
     return descriptorFile;
   }
 
-  public void insert(final File file, final String contentType, final String path) throws IOException
+  public void insert(final File file, final String contentType, final String path)
+      throws IOException
   {
     LOG.info("Inserting [%s] to [%s]", file, path);
 
@@ -106,7 +107,8 @@ public void insert(final File file, final String contentType, final String path)
   }
 
   @Override
-  public DataSegment push(final File indexFilesDir, final DataSegment segment) throws IOException
+  public DataSegment push(final File indexFilesDir, final DataSegment segment, final boolean useUniquePath)
+      throws IOException
   {
     LOG.info("Uploading [%s] to Google.", indexFilesDir);
 
@@ -117,7 +119,7 @@ public DataSegment push(final File indexFilesDir, final DataSegment segment) thr
     try {
       indexFile = File.createTempFile("index", ".zip");
       final long indexSize = CompressionUtils.zip(indexFilesDir, indexFile);
-      final String storageDir = this.getStorageDir(segment);
+      final String storageDir = this.getStorageDir(segment, useUniquePath);
       final String indexPath = buildPath(storageDir + "/" + "index.zip");
       final String descriptorPath = buildPath(storageDir + "/" + "descriptor.json");
 
diff --git a/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java b/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java
index 6c845d433c7..b90f0d5f560 100644
--- a/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java
+++ b/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java
@@ -89,7 +89,7 @@ public void testPush() throws Exception
         "foo",
         Intervals.of("2015/2016"),
         "0",
-        Maps.<String, Object>newHashMap(),
+        Maps.newHashMap(),
         Lists.<String>newArrayList(),
         Lists.<String>newArrayList(),
         new NoneShardSpec(),
@@ -105,18 +105,26 @@ public void testPush() throws Exception
          jsonMapper
     ).addMockedMethod("insert", File.class, String.class, String.class).createMock();
 
-    final String storageDir = pusher.getStorageDir(segmentToPush);
+    final String storageDir = pusher.getStorageDir(segmentToPush, false);
     final String indexPath = prefix + "/" + storageDir + "/" + "index.zip";
     final String descriptorPath = prefix + "/" + storageDir + "/" + "descriptor.json";
 
-    pusher.insert(EasyMock.anyObject(File.class), EasyMock.eq("application/zip"), EasyMock.eq(indexPath));
+    pusher.insert(
+        EasyMock.anyObject(File.class),
+        EasyMock.eq("application/zip"),
+        EasyMock.eq(indexPath)
+    );
     expectLastCall();
-    pusher.insert(EasyMock.anyObject(File.class), EasyMock.eq("application/json"), EasyMock.eq(descriptorPath));
+    pusher.insert(
+        EasyMock.anyObject(File.class),
+        EasyMock.eq("application/json"),
+        EasyMock.eq(descriptorPath)
+    );
     expectLastCall();
 
     replayAll();
 
-    DataSegment segment = pusher.push(tempFolder.getRoot(), segmentToPush);
+    DataSegment segment = pusher.push(tempFolder.getRoot(), segmentToPush, false);
 
     Assert.assertEquals(segmentToPush.getSize(), segment.getSize());
     Assert.assertEquals(segmentToPush, segment);
diff --git a/extensions-contrib/graphite-emitter/pom.xml b/extensions-contrib/graphite-emitter/pom.xml
index 8569823ea94..8f14ee1a80f 100644
--- a/extensions-contrib/graphite-emitter/pom.xml
+++ b/extensions-contrib/graphite-emitter/pom.xml
@@ -18,14 +18,13 @@
  ~ 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";>
+<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>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
@@ -54,8 +53,9 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>com.metamx</groupId>
+      <groupId>io.druid</groupId>
       <artifactId>java-util</artifactId>
+      <version>${project.parent.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
diff --git a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/DruidToGraphiteEventConverter.java b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/DruidToGraphiteEventConverter.java
index 673fa2af2af..243b6119799 100644
--- a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/DruidToGraphiteEventConverter.java
+++ b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/DruidToGraphiteEventConverter.java
@@ -21,7 +21,7 @@
 
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 
 
 @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = WhiteListBasedConverter.class)
diff --git a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java
index cb0fb30d05b..bba2796862c 100644
--- a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java
+++ b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java
@@ -23,10 +23,10 @@
 import com.codahale.metrics.graphite.GraphiteSender;
 import com.codahale.metrics.graphite.PickledGraphite;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.metamx.emitter.core.Emitter;
-import com.metamx.emitter.core.Event;
-import com.metamx.emitter.service.AlertEvent;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.Event;
+import io.druid.java.util.emitter.service.AlertEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.logger.Logger;
diff --git a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterModule.java b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterModule.java
index b7fd38b3681..12b27213c79 100644
--- a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterModule.java
+++ b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterModule.java
@@ -29,7 +29,7 @@
 import com.google.inject.Provides;
 import com.google.inject.name.Named;
 import com.google.inject.name.Names;
-import com.metamx.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.Emitter;
 import io.druid.guice.JsonConfigProvider;
 import io.druid.guice.ManageLifecycle;
 import io.druid.initialization.DruidModule;
diff --git a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/SendAllGraphiteEventConverter.java b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/SendAllGraphiteEventConverter.java
index dc26014e1b3..e3b70568429 100644
--- a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/SendAllGraphiteEventConverter.java
+++ b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/SendAllGraphiteEventConverter.java
@@ -26,7 +26,7 @@
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSortedSet;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 
 import java.util.concurrent.TimeUnit;
 
diff --git a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/WhiteListBasedConverter.java b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/WhiteListBasedConverter.java
index ac53033ca6a..25018c5f504 100644
--- a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/WhiteListBasedConverter.java
+++ b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/WhiteListBasedConverter.java
@@ -33,7 +33,7 @@
 import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.io.Files;
 import com.google.common.io.Resources;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.logger.Logger;
 
diff --git a/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/WhiteListBasedConverterTest.java b/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/WhiteListBasedConverterTest.java
index 11908fb1bb2..10022a4023b 100644
--- a/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/WhiteListBasedConverterTest.java
+++ b/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/WhiteListBasedConverterTest.java
@@ -20,7 +20,7 @@
 package io.druid.emitter.graphite;
 
 import com.google.common.collect.Maps;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.jackson.DefaultObjectMapper;
 import io.druid.java.util.common.DateTimes;
 import junitparams.JUnitParamsRunner;
diff --git a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml
index 1c9f5c17ffe..b69d3067069 100644
--- a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml
+++ b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml
@@ -16,8 +16,7 @@
 ~ 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";>
+<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>
   <groupId>io.druid.extensions.contrib</groupId>
   <artifactId>druid-kafka-eight-simple-consumer</artifactId>
@@ -27,7 +26,7 @@
   <parent>
     <groupId>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
@@ -39,8 +38,9 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>com.metamx</groupId>
+      <groupId>io.druid</groupId>
       <artifactId>java-util</artifactId>
+      <version>${project.parent.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
diff --git a/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java b/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java
index 0c53007311d..c65fa808d70 100644
--- a/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java
+++ b/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java
@@ -25,8 +25,8 @@
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Maps;
 import com.google.common.io.Closeables;
-import com.metamx.common.parsers.ParseException;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.common.parsers.ParseException;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.data.input.ByteBufferInputRowParser;
 import io.druid.data.input.Committer;
 import io.druid.data.input.FirehoseFactoryV2;
diff --git a/extensions-contrib/kafka-emitter/pom.xml b/extensions-contrib/kafka-emitter/pom.xml
index 9130834d243..652407e7bac 100644
--- a/extensions-contrib/kafka-emitter/pom.xml
+++ b/extensions-contrib/kafka-emitter/pom.xml
@@ -18,14 +18,13 @@
  ~ 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";>
+<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>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
@@ -53,8 +52,9 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>com.metamx</groupId>
+      <groupId>io.druid</groupId>
       <artifactId>java-util</artifactId>
+      <version>${project.parent.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
diff --git a/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitter.java b/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitter.java
index 7a791be5c84..7628521dd8d 100644
--- a/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitter.java
+++ b/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitter.java
@@ -22,10 +22,10 @@
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
-import com.metamx.emitter.core.Emitter;
-import com.metamx.emitter.core.Event;
-import com.metamx.emitter.service.AlertEvent;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.Event;
+import io.druid.java.util.emitter.service.AlertEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.emitter.kafka.MemoryBoundLinkedBlockingQueue.ObjectContainer;
 import io.druid.java.util.common.StringUtils;
 import io.druid.java.util.common.lifecycle.LifecycleStart;
diff --git a/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitterModule.java b/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitterModule.java
index 0608b1f0d89..7db5724787f 100644
--- a/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitterModule.java
+++ b/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitterModule.java
@@ -24,7 +24,7 @@
 import com.google.inject.Binder;
 import com.google.inject.Provides;
 import com.google.inject.name.Named;
-import com.metamx.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.Emitter;
 import io.druid.guice.JsonConfigProvider;
 import io.druid.guice.ManageLifecycle;
 import io.druid.initialization.DruidModule;
diff --git a/extensions-contrib/orc-extensions/pom.xml b/extensions-contrib/orc-extensions/pom.xml
index 4eb48e949eb..f27ecded100 100644
--- a/extensions-contrib/orc-extensions/pom.xml
+++ b/extensions-contrib/orc-extensions/pom.xml
@@ -15,9 +15,7 @@
   ~ 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";>
+<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";>
     <groupId>io.druid.extensions.contrib</groupId>
     <artifactId>druid-orc-extensions</artifactId>
     <name>druid-orc-extensions</name>
@@ -26,7 +24,7 @@
     <parent>
         <artifactId>druid</artifactId>
         <groupId>io.druid</groupId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <modelVersion>4.0.0</modelVersion>
diff --git a/extensions-contrib/parquet-extensions/pom.xml b/extensions-contrib/parquet-extensions/pom.xml
index b2a460fc289..d0915f75ab7 100644
--- a/extensions-contrib/parquet-extensions/pom.xml
+++ b/extensions-contrib/parquet-extensions/pom.xml
@@ -1,7 +1,5 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<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";>
+<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";>
     <groupId>io.druid.extensions.contrib</groupId>
     <artifactId>druid-parquet-extensions</artifactId>
     <name>druid-parquet-extensions</name>
@@ -10,7 +8,7 @@
     <parent>
         <artifactId>druid</artifactId>
         <groupId>io.druid</groupId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <modelVersion>4.0.0</modelVersion>
diff --git a/extensions-contrib/rabbitmq/pom.xml b/extensions-contrib/rabbitmq/pom.xml
index 8f1ebc47f19..abb09485bcc 100644
--- a/extensions-contrib/rabbitmq/pom.xml
+++ b/extensions-contrib/rabbitmq/pom.xml
@@ -29,7 +29,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-contrib/redis-cache/pom.xml b/extensions-contrib/redis-cache/pom.xml
index 9d0c578bcd0..f49de22e356 100644
--- a/extensions-contrib/redis-cache/pom.xml
+++ b/extensions-contrib/redis-cache/pom.xml
@@ -19,8 +19,7 @@
   ~ 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";>
+<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>
 
     <groupId>io.druid.extensions.contrib</groupId>
@@ -30,7 +29,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCache.java b/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCache.java
index 11b63a2e0f9..9b4744fd961 100644
--- a/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCache.java
+++ b/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCache.java
@@ -21,8 +21,8 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.java.util.common.logger.Logger;
 import redis.clients.jedis.Jedis;
 import redis.clients.jedis.JedisPool;
diff --git a/extensions-contrib/sqlserver-metadata-storage/pom.xml b/extensions-contrib/sqlserver-metadata-storage/pom.xml
index 0f3970cdf20..26eb5d99ef2 100644
--- a/extensions-contrib/sqlserver-metadata-storage/pom.xml
+++ b/extensions-contrib/sqlserver-metadata-storage/pom.xml
@@ -26,7 +26,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-contrib/statsd-emitter/pom.xml b/extensions-contrib/statsd-emitter/pom.xml
index 061b2a961a8..c3a256d4d94 100644
--- a/extensions-contrib/statsd-emitter/pom.xml
+++ b/extensions-contrib/statsd-emitter/pom.xml
@@ -17,13 +17,11 @@
  ~ 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";>
+<project xmlns="http://maven.apache.org/POM/4.0.0"; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd";>
   <parent>
     <artifactId>druid</artifactId>
     <groupId>io.druid</groupId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
@@ -47,8 +45,9 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>com.metamx</groupId>
+      <groupId>io.druid</groupId>
       <artifactId>java-util</artifactId>
+      <version>${project.parent.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
diff --git a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java
index c949f13f477..d395146f87e 100644
--- a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java
+++ b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java
@@ -22,9 +22,9 @@
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
-import com.metamx.emitter.core.Emitter;
-import com.metamx.emitter.core.Event;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.Event;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import com.timgroup.statsd.NonBlockingStatsDClient;
 import com.timgroup.statsd.StatsDClient;
 import com.timgroup.statsd.StatsDClientErrorHandler;
diff --git a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java
index 3d00da34eff..2d8bd2087c0 100644
--- a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java
+++ b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java
@@ -24,7 +24,7 @@
 import com.google.inject.Binder;
 import com.google.inject.Provides;
 import com.google.inject.name.Named;
-import com.metamx.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.Emitter;
 import io.druid.guice.JsonConfigProvider;
 import io.druid.guice.ManageLifecycle;
 import io.druid.initialization.DruidModule;
diff --git a/extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java b/extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java
index b4c8fa33243..62132a266cf 100644
--- a/extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java
+++ b/extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java
@@ -19,7 +19,7 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.emitter.statsd.DimensionConverter;
 import io.druid.emitter.statsd.StatsDMetric;
 import io.druid.java.util.common.DateTimes;
diff --git a/extensions-contrib/statsd-emitter/src/test/java/StatsDEmitterTest.java b/extensions-contrib/statsd-emitter/src/test/java/StatsDEmitterTest.java
index 531ca7a5dbf..16175d917a4 100644
--- a/extensions-contrib/statsd-emitter/src/test/java/StatsDEmitterTest.java
+++ b/extensions-contrib/statsd-emitter/src/test/java/StatsDEmitterTest.java
@@ -18,7 +18,7 @@
  */
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import com.timgroup.statsd.StatsDClient;
 import io.druid.emitter.statsd.StatsDEmitter;
 import io.druid.emitter.statsd.StatsDEmitterConfig;
diff --git a/extensions-contrib/thrift-extensions/pom.xml b/extensions-contrib/thrift-extensions/pom.xml
index 18c51677a61..3eca95a1251 100644
--- a/extensions-contrib/thrift-extensions/pom.xml
+++ b/extensions-contrib/thrift-extensions/pom.xml
@@ -1,7 +1,5 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<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";>
+<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";>
 
   <groupId>io.druid.extensions.contrib</groupId>
   <artifactId>druid-thrift-extensions</artifactId>
@@ -11,7 +9,7 @@
   <parent>
     <artifactId>druid</artifactId>
     <groupId>io.druid</groupId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftInputRowParser.java b/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftInputRowParser.java
index 87169252b8f..aa523f34993 100644
--- a/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftInputRowParser.java
+++ b/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftInputRowParser.java
@@ -23,7 +23,7 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
-import com.metamx.common.IAE;
+import io.druid.java.util.common.IAE;
 import com.twitter.elephantbird.mapreduce.io.ThriftWritable;
 import io.druid.data.input.InputRow;
 import io.druid.data.input.MapBasedInputRow;
diff --git a/extensions-contrib/time-min-max/pom.xml b/extensions-contrib/time-min-max/pom.xml
index 59bc7ee6b1a..9c33c8bc8d0 100644
--- a/extensions-contrib/time-min-max/pom.xml
+++ b/extensions-contrib/time-min-max/pom.xml
@@ -17,13 +17,11 @@
  ~ 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";>
+<project xmlns="http://maven.apache.org/POM/4.0.0"; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd";>
   <parent>
     <artifactId>druid</artifactId>
     <groupId>io.druid</groupId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/extensions-contrib/virtual-columns/pom.xml b/extensions-contrib/virtual-columns/pom.xml
index 3102a75d1d9..9a3a4337cb1 100644
--- a/extensions-contrib/virtual-columns/pom.xml
+++ b/extensions-contrib/virtual-columns/pom.xml
@@ -29,7 +29,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/DruidVirtualColumnsModule.java b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/DruidVirtualColumnsModule.java
index 184225d0245..9db5be737a2 100644
--- a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/DruidVirtualColumnsModule.java
+++ b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/DruidVirtualColumnsModule.java
@@ -20,6 +20,7 @@
 package io.druid.segment;
 
 import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
 import com.fasterxml.jackson.databind.module.SimpleModule;
 import com.google.common.collect.ImmutableList;
 import com.google.inject.Binder;
@@ -34,7 +35,12 @@
   @Override
   public List<? extends Module> getJacksonModules()
   {
-    return ImmutableList.of(new SimpleModule().registerSubtypes(MapVirtualColumn.class));
+    return ImmutableList.of(
+        new SimpleModule(getClass().getSimpleName())
+            .registerSubtypes(
+                new NamedType(MapVirtualColumn.class, "map")
+            )
+    );
   }
 
   @Override
diff --git a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java
index c17b219a038..79950efd087 100644
--- a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java
+++ b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java
@@ -24,7 +24,7 @@
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Maps;
-import com.metamx.common.StringUtils;
+import io.druid.java.util.common.StringUtils;
 import io.druid.query.dimension.DefaultDimensionSpec;
 import io.druid.query.dimension.DimensionSpec;
 import io.druid.query.filter.DimFilterUtils;
diff --git a/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java b/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java
index 283aa8684a9..cc9417f7a0b 100644
--- a/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java
@@ -19,6 +19,7 @@
 
 package io.druid.segment;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Supplier;
 import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableMap;
@@ -138,6 +139,18 @@ public MapVirtualColumnTest(QueryRunner runner)
                  .pagingSpec(new PagingSpec(null, 3));
   }
 
+  @Test
+  public void testSerde() throws IOException
+  {
+    final ObjectMapper mapper = new DefaultObjectMapper();
+    new DruidVirtualColumnsModule().getJacksonModules().forEach(mapper::registerModule);
+
+    final MapVirtualColumn column = new MapVirtualColumn("keys", "values", "params");
+    final String json = mapper.writeValueAsString(column);
+    final VirtualColumn fromJson = mapper.readValue(json, VirtualColumn.class);
+    Assert.assertEquals(column, fromJson);
+  }
+
   @Test
   public void testBasic() throws Exception
   {
diff --git a/extensions-core/avro-extensions/pom.xml b/extensions-core/avro-extensions/pom.xml
index 9102202fac3..ce2aeb08063 100644
--- a/extensions-core/avro-extensions/pom.xml
+++ b/extensions-core/avro-extensions/pom.xml
@@ -18,8 +18,7 @@
  ~ 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";>
+<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>
 
   <groupId>io.druid.extensions</groupId>
@@ -30,7 +29,7 @@
   <parent>
     <groupId>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml
index f7cf74e7458..2cd4e1fb560 100644
--- a/extensions-core/datasketches/pom.xml
+++ b/extensions-core/datasketches/pom.xml
@@ -18,8 +18,7 @@
   ~ 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";>
+<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>
 
   <groupId>io.druid.extensions</groupId>
@@ -30,7 +29,7 @@
   <parent>
     <groupId>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java
index 7ebb40b5d7a..bfbaaac86f7 100644
--- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java
+++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java
@@ -21,7 +21,7 @@
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.metamx.common.IAE;
+import io.druid.java.util.common.IAE;
 import com.yahoo.sketches.Util;
 import com.yahoo.sketches.quantiles.DoublesSketch;
 import com.yahoo.sketches.quantiles.DoublesUnion;
diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java
index 0e3081965b0..f26bf348a33 100644
--- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java
+++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java
@@ -35,6 +35,7 @@
 {
 
   public static final String DOUBLES_SKETCH = "quantilesDoublesSketch";
+  public static final String DOUBLES_SKETCH_MERGE = "quantilesDoublesSketchMerge";
 
   public static final String DOUBLES_SKETCH_HISTOGRAM_POST_AGG = "quantilesDoublesSketchToHistogram";
   public static final String DOUBLES_SKETCH_QUANTILE_POST_AGG = "quantilesDoublesSketchToQuantile";
@@ -55,6 +56,7 @@ public void configure(final Binder binder)
     return Arrays.<Module> asList(
         new SimpleModule("DoublesQuantilesSketchModule").registerSubtypes(
             new NamedType(DoublesSketchAggregatorFactory.class, DOUBLES_SKETCH),
+            new NamedType(DoublesSketchMergeAggregatorFactory.class, DOUBLES_SKETCH_MERGE),
             new NamedType(DoublesSketchToHistogramPostAggregator.class, DOUBLES_SKETCH_HISTOGRAM_POST_AGG),
             new NamedType(DoublesSketchToQuantilePostAggregator.class, DOUBLES_SKETCH_QUANTILE_POST_AGG),
             new NamedType(DoublesSketchToQuantilesPostAggregator.class, DOUBLES_SKETCH_QUANTILES_POST_AGG),
diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java
index 899c8392e4f..ea07be5567b 100644
--- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java
@@ -28,6 +28,7 @@
 import io.druid.java.util.common.guava.Sequence;
 import io.druid.java.util.common.guava.Sequences;
 import io.druid.query.aggregation.AggregationTestHelper;
+import io.druid.query.aggregation.AggregatorFactory;
 import io.druid.query.groupby.GroupByQueryConfig;
 import io.druid.query.groupby.GroupByQueryRunnerTest;
 import org.junit.Assert;
@@ -60,7 +61,8 @@ public DoublesSketchAggregatorTest(final GroupByQueryConfig config)
         module.getJacksonModules(), config, tempFolder);
     timeSeriesHelper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper(
         module.getJacksonModules(),
-        tempFolder);
+        tempFolder
+    );
   }
 
   @Parameterized.Parameters(name = "{0}")
@@ -68,7 +70,7 @@ public DoublesSketchAggregatorTest(final GroupByQueryConfig config)
   {
     final List<Object[]> constructors = Lists.newArrayList();
     for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
-      constructors.add(new Object[] {config});
+      constructors.add(new Object[]{config});
     }
     return constructors;
   }
@@ -78,11 +80,29 @@ public DoublesSketchAggregatorTest(final GroupByQueryConfig config)
   public void serializeDeserializeFactoryWithFieldName() throws Exception
   {
     ObjectMapper objectMapper = new DefaultObjectMapper();
+    new DoublesSketchModule().getJacksonModules().forEach(objectMapper::registerModule);
     DoublesSketchAggregatorFactory factory = new DoublesSketchAggregatorFactory("name", "filedName", 128);
 
-    DoublesSketchAggregatorFactory other = objectMapper.readValue(
+    AggregatorFactory other = objectMapper.readValue(
         objectMapper.writeValueAsString(factory),
-        DoublesSketchAggregatorFactory.class);
+        AggregatorFactory.class
+    );
+
+    Assert.assertEquals(factory, other);
+  }
+
+  // this is to test Json properties and equals for the combining factory
+  @Test
+  public void serializeDeserializeCombiningFactoryWithFieldName() throws Exception
+  {
+    ObjectMapper objectMapper = new DefaultObjectMapper();
+    new DoublesSketchModule().getJacksonModules().forEach(objectMapper::registerModule);
+    DoublesSketchAggregatorFactory factory = new DoublesSketchMergeAggregatorFactory("name", 128);
+
+    AggregatorFactory other = objectMapper.readValue(
+        objectMapper.writeValueAsString(factory),
+        AggregatorFactory.class
+    );
 
     Assert.assertEquals(factory, other);
   }
@@ -92,7 +112,8 @@ public void ingestingSketches() throws Exception
   {
     Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("quantiles/doubles_sketch_data.tsv").getFile()),
-        String.join("\n",
+        String.join(
+            "\n",
             "{",
             "  \"type\": \"string\",",
             "  \"parseSpec\": {",
@@ -105,16 +126,20 @@ public void ingestingSketches() throws Exception
             "    },",
             "    \"columns\": [\"timestamp\", \"product\", \"sketch\"]",
             "  }",
-            "}"),
-        String.join("\n",
+            "}"
+        ),
+        String.join(
+            "\n",
             "[",
             "  {\"type\": \"quantilesDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"k\": 128},",
             "  {\"type\": \"quantilesDoublesSketch\", \"name\": \"non_existent_sketch\", \"fieldName\": \"non_existent_sketch\", \"k\": 128}",
-            "]"),
+            "]"
+        ),
         0, // minTimestamp
         Granularities.NONE,
         10, // maxRowCount
-        String.join("\n",
+        String.join(
+            "\n",
             "{",
             "  \"queryType\": \"groupBy\",",
             "  \"dataSource\": \"test_datasource\",",
@@ -129,8 +154,10 @@ public void ingestingSketches() throws Exception
             "    {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}",
             "  ],",
             "  \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
-            "}"));
-    List<Row> results = Sequences.toList(seq, new ArrayList<Row>());
+            "}"
+        )
+    );
+    List<Row> results = Sequences.toList(seq, new ArrayList<>());
     Assert.assertEquals(1, results.size());
     Row row = results.get(0);
 
@@ -156,9 +183,9 @@ public void ingestingSketches() throws Exception
     Object histogramObject = row.getRaw("histogram");
     Assert.assertTrue(histogramObject instanceof double[]);
     double[] histogram = (double[]) histogramObject;
-    for (final double bin: histogram) {
-      Assert.assertEquals(100, bin, 100 * 0.2); // 400 items uniformly
-                                                // distributed into 4 bins
+    for (final double bin : histogram) {
+      // 400 items uniformly distributed into 4 bins
+      Assert.assertEquals(100, bin, 100 * 0.2);
     }
   }
 
@@ -167,7 +194,8 @@ public void buildingSketchesAtIngestionTime() throws Exception
   {
     Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),
-        String.join("\n",
+        String.join(
+            "\n",
             "{",
             "  \"type\": \"string\",",
             "  \"parseSpec\": {",
@@ -180,12 +208,14 @@ public void buildingSketchesAtIngestionTime() throws Exception
             "    },",
             "    \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]",
             "  }",
-            "}"),
+            "}"
+        ),
         "[{\"type\": \"quantilesDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 128}]",
         0, // minTimestamp
         Granularities.NONE,
         10, // maxRowCount
-        String.join("\n",
+        String.join(
+            "\n",
             "{",
             "  \"queryType\": \"groupBy\",",
             "  \"dataSource\": \"test_datasource\",",
@@ -200,8 +230,10 @@ public void buildingSketchesAtIngestionTime() throws Exception
             "    {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}",
             "  ],",
             "  \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
-            "}"));
-    List<Row> results = Sequences.toList(seq, new ArrayList<Row>());
+            "}"
+        )
+    );
+    List<Row> results = Sequences.toList(seq, new ArrayList<>());
     Assert.assertEquals(1, results.size());
     Row row = results.get(0);
 
@@ -225,7 +257,7 @@ public void buildingSketchesAtIngestionTime() throws Exception
     Assert.assertEquals(4, histogram.length);
     for (final double bin: histogram) {
       Assert.assertEquals(100, bin, 100 * 0.2); // 400 items uniformly
-                                                // distributed into 4 bins
+      // distributed into 4 bins
     }
   }
 
@@ -234,7 +266,8 @@ public void buildingSketchesAtQueryTime() throws Exception
   {
     Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
         new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),
-        String.join("\n",
+        String.join(
+            "\n",
             "{",
             "  \"type\": \"string\",",
             "  \"parseSpec\": {",
@@ -247,12 +280,14 @@ public void buildingSketchesAtQueryTime() throws Exception
             "    },",
             "    \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]",
             "  }",
-            "}"),
+            "}"
+        ),
         "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]",
         0, // minTimestamp
         Granularities.NONE,
         10, // maxRowCount
-        String.join("\n",
+        String.join(
+            "\n",
             "{",
             "  \"queryType\": \"groupBy\",",
             "  \"dataSource\": \"test_datasource\",",
@@ -267,8 +302,10 @@ public void buildingSketchesAtQueryTime() throws Exception
             "    {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}",
             "  ],",
             "  \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
-            "}"));
-    List<Row> results = Sequences.toList(seq, new ArrayList<Row>());
+            "}"
+        )
+    );
+    List<Row> results = Sequences.toList(seq, new ArrayList<>());
     Assert.assertEquals(1, results.size());
     Row row = results.get(0);
 
@@ -296,7 +333,7 @@ public void buildingSketchesAtQueryTime() throws Exception
     double[] histogram = (double[]) histogramObject;
     for (final double bin: histogram) {
       Assert.assertEquals(100, bin, 100 * 0.2); // 400 items uniformly
-                                                // distributed into 4 bins
+      // distributed into 4 bins
     }
   }
 
@@ -319,7 +356,8 @@ public void QueryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Except
             "    },",
             "    \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]",
             "  }",
-            "}"),
+            "}"
+        ),
         "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]",
         0, // minTimestamp
         Granularities.NONE,
@@ -340,8 +378,10 @@ public void QueryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Except
             "    {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}",
             "  ],",
             "  \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
-            "}"));
-    List<Row> results = Sequences.toList(seq, new ArrayList<Row>());
+            "}"
+        )
+    );
+    List<Row> results = Sequences.toList(seq, new ArrayList<>());
     Assert.assertEquals(1, results.size());
     Row row = results.get(0);
 
@@ -369,7 +409,7 @@ public void QueryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Except
     double[] histogram = (double[]) histogramObject;
     for (final double bin: histogram) {
       Assert.assertEquals(100, bin, 100 * 0.2); // 400 items uniformly
-                                                // distributed into 4 bins
+      // distributed into 4 bins
     }
   }
 
@@ -392,7 +432,8 @@ public void TimeSeriesQueryInputAsFloat() throws Exception
             "    },",
             "    \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]",
             "  }",
-            "}"),
+            "}"
+        ),
         "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]",
         0, // minTimestamp
         Granularities.NONE,
@@ -412,8 +453,10 @@ public void TimeSeriesQueryInputAsFloat() throws Exception
             "    {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram1\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}",
             "  ],",
             "  \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
-            "}"));
-    List<Row> results = Sequences.toList(seq, new ArrayList<Row>());
+            "}"
+        )
+    );
+    List<Row> results = Sequences.toList(seq, new ArrayList<>());
     Assert.assertEquals(1, results.size());
   }
 }
diff --git a/extensions-core/druid-basic-security/pom.xml b/extensions-core/druid-basic-security/pom.xml
index 1de01638186..0883655f9c1 100644
--- a/extensions-core/druid-basic-security/pom.xml
+++ b/extensions-core/druid-basic-security/pom.xml
@@ -17,9 +17,7 @@
   ~ 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";>
+<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>
 
   <groupId>io.druid.extensions</groupId>
@@ -30,7 +28,7 @@
   <parent>
     <groupId>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/CommonCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/CommonCacheNotifier.java
index 4b7a557197c..9554928ba60 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/CommonCacheNotifier.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/CommonCacheNotifier.java
@@ -21,12 +21,12 @@
 
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.ClientResponse;
-import com.metamx.http.client.response.HttpResponseHandler;
-import com.metamx.http.client.response.StatusResponseHolder;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.ClientResponse;
+import io.druid.java.util.http.client.response.HttpResponseHandler;
+import io.druid.java.util.http.client.response.StatusResponseHolder;
 import io.druid.discovery.DiscoveryDruidNode;
 import io.druid.discovery.DruidNodeDiscovery;
 import io.druid.discovery.DruidNodeDiscoveryProvider;
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPAuthenticator.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPAuthenticator.java
index b127687999b..eeb406ee681 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPAuthenticator.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPAuthenticator.java
@@ -105,7 +105,7 @@ public AuthenticationResult authenticateJDBCContext(Map<String, Object> context)
     }
 
     if (checkCredentials(user, password.toCharArray())) {
-      return new AuthenticationResult(user, name, null);
+      return new AuthenticationResult(user, authorizerName, name, null);
     } else {
       return null;
     }
@@ -173,7 +173,7 @@ public void doFilter(
       char[] password = splits[1].toCharArray();
 
       if (checkCredentials(user, password)) {
-        AuthenticationResult authenticationResult = new AuthenticationResult(user, authorizerName, null);
+        AuthenticationResult authenticationResult = new AuthenticationResult(user, authorizerName, name, null);
         servletRequest.setAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult);
       }
 
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPEscalator.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPEscalator.java
index 2eb73a23bf2..d2993f6509d 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPEscalator.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPEscalator.java
@@ -22,21 +22,11 @@
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Throwables;
-import com.metamx.http.client.CredentialedHttpClient;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.auth.BasicCredentials;
-import io.druid.java.util.common.StringUtils;
-import io.druid.security.basic.BasicAuthUtils;
+import io.druid.java.util.http.client.CredentialedHttpClient;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.auth.BasicCredentials;
 import io.druid.server.security.AuthenticationResult;
 import io.druid.server.security.Escalator;
-import org.eclipse.jetty.client.api.Authentication;
-import org.eclipse.jetty.client.api.ContentResponse;
-import org.eclipse.jetty.client.api.Request;
-import org.eclipse.jetty.util.Attributes;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-
-import java.net.URI;
 
 @JsonTypeName("basic")
 public class BasicHTTPEscalator implements Escalator
@@ -66,51 +56,11 @@ public HttpClient createEscalatedClient(HttpClient baseClient)
     );
   }
 
-  @Override
-  public org.eclipse.jetty.client.HttpClient createEscalatedJettyClient(org.eclipse.jetty.client.HttpClient baseClient)
-  {
-    baseClient.getAuthenticationStore().addAuthentication(new Authentication()
-    {
-      @Override
-      public boolean matches(String type, URI uri, String realm)
-      {
-        return true;
-      }
-
-      @Override
-      public Result authenticate(
-          final Request request, ContentResponse response, Authentication.HeaderInfo headerInfo, Attributes context
-      )
-      {
-        return new Result()
-        {
-          @Override
-          public URI getURI()
-          {
-            return request.getURI();
-          }
-
-          @Override
-          public void apply(Request request)
-          {
-            try {
-              final String unencodedCreds = StringUtils.format("%s:%s", internalClientUsername, internalClientPassword);
-              final String base64Creds = BasicAuthUtils.getEncodedCredentials(unencodedCreds);
-              request.getHeaders().add(HttpHeaders.Names.AUTHORIZATION, "Basic " + base64Creds);
-            }
-            catch (Throwable e) {
-              Throwables.propagate(e);
-            }
-          }
-        };
-      }
-    });
-    return baseClient;
-  }
-
   @Override
   public AuthenticationResult createEscalatedAuthenticationResult()
   {
-    return new AuthenticationResult(internalClientUsername, authorizerName, null);
+    // if you found your self asking why the authenticatedBy field is set to null please read this:
+    // https://github.com/druid-io/druid/pull/5706#discussion_r185940889
+    return new AuthenticationResult(internalClientUsername, authorizerName, null, null);
   }
 }
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BytesFullResponseHandler.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BytesFullResponseHandler.java
index d8b65a5e8ff..c548cfc0ac1 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BytesFullResponseHandler.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BytesFullResponseHandler.java
@@ -19,9 +19,9 @@
 
 package io.druid.security.basic.authentication;
 
-import com.metamx.http.client.response.ClientResponse;
-import com.metamx.http.client.response.FullResponseHolder;
-import com.metamx.http.client.response.HttpResponseHandler;
+import io.druid.java.util.http.client.response.ClientResponse;
+import io.druid.java.util.http.client.response.FullResponseHolder;
+import io.druid.java.util.http.client.response.HttpResponseHandler;
 import org.jboss.netty.handler.codec.http.HttpChunk;
 import org.jboss.netty.handler.codec.http.HttpResponse;
 
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BytesFullResponseHolder.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BytesFullResponseHolder.java
index a273f503098..b701468b461 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BytesFullResponseHolder.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BytesFullResponseHolder.java
@@ -19,7 +19,7 @@
 
 package io.druid.security.basic.authentication;
 
-import com.metamx.http.client.response.FullResponseHolder;
+import io.druid.java.util.http.client.response.FullResponseHolder;
 import org.jboss.netty.handler.codec.http.HttpResponse;
 import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java
index 2eff574cfd9..5ab501b5166 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java
@@ -21,8 +21,8 @@
 
 import com.google.common.base.Preconditions;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.http.client.HttpClient;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.http.client.HttpClient;
 import io.druid.concurrent.LifecycleLock;
 import io.druid.discovery.DruidNodeDiscoveryProvider;
 import io.druid.guice.ManageLifecycle;
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java
index 178232e13df..bc9a713b9c1 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java
@@ -21,16 +21,16 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
-import com.google.common.io.Files;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.http.client.Request;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.http.client.Request;
 import io.druid.client.coordinator.Coordinator;
 import io.druid.concurrent.LifecycleLock;
 import io.druid.discovery.DruidLeaderClient;
 import io.druid.guice.ManageLifecycle;
 import io.druid.guice.annotations.Smile;
+import io.druid.java.util.common.FileUtils;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.RetryUtils;
 import io.druid.java.util.common.StringUtils;
@@ -235,7 +235,7 @@ private void writeUserMapToDisk(String prefix, byte[] userMapBytes) throws IOExc
     File cacheDir = new File(commonCacheConfig.getCacheDirectory());
     cacheDir.mkdirs();
     File userMapFile = new File(commonCacheConfig.getCacheDirectory(), getUserMapFilename(prefix));
-    Files.write(userMapBytes, userMapFile);
+    FileUtils.writeAtomically(userMapFile, out -> out.write(userMapBytes));
   }
 
   private Map<String, BasicAuthenticatorUser> tryFetchUserMapFromCoordinator(String prefix) throws Exception
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java
index eacc0242237..872cdc10320 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java
@@ -22,7 +22,7 @@
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.common.config.ConfigManager;
 import io.druid.concurrent.LifecycleLock;
 import io.druid.guice.ManageLifecycle;
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java
index 924107cf92a..9f7a1def62a 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java
@@ -21,8 +21,8 @@
 
 import com.google.common.base.Preconditions;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.http.client.HttpClient;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.http.client.HttpClient;
 import io.druid.concurrent.LifecycleLock;
 import io.druid.discovery.DruidNodeDiscoveryProvider;
 import io.druid.guice.ManageLifecycle;
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java
index a57d931c97f..5f993d3121f 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java
@@ -21,16 +21,16 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
-import com.google.common.io.Files;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.http.client.Request;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.http.client.Request;
 import io.druid.client.coordinator.Coordinator;
 import io.druid.concurrent.LifecycleLock;
 import io.druid.discovery.DruidLeaderClient;
 import io.druid.guice.ManageLifecycle;
 import io.druid.guice.annotations.Smile;
+import io.druid.java.util.common.FileUtils;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.RetryUtils;
 import io.druid.java.util.common.StringUtils;
@@ -212,7 +212,7 @@ private void writeMapToDisk(String prefix, byte[] userMapBytes) throws IOExcepti
     File cacheDir = new File(commonCacheConfig.getCacheDirectory());
     cacheDir.mkdirs();
     File userMapFile = new File(commonCacheConfig.getCacheDirectory(), getUserRoleMapFilename(prefix));
-    Files.write(userMapBytes, userMapFile);
+    FileUtils.writeAtomically(userMapFile, out -> out.write(userMapBytes));
   }
 
   @Nullable
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java
index ca80d19712d..adc0a60591c 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java
@@ -23,7 +23,7 @@
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.common.config.ConfigManager;
 import io.druid.concurrent.LifecycleLock;
 import io.druid.guice.ManageLifecycle;
diff --git a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java
index 6132753c684..aafac305525 100644
--- a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java
+++ b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java
@@ -116,7 +116,7 @@ public void testAuth()
 
     updater.setPermissions(AUTHORIZER_NAME, "druidRole", permissions);
 
-    AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid", null);
+    AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid", null, null);
 
     Access access = authorizer.authorize(
         authenticationResult,
diff --git a/extensions-core/druid-kerberos/pom.xml b/extensions-core/druid-kerberos/pom.xml
index 08696bed812..a8960f4411a 100644
--- a/extensions-core/druid-kerberos/pom.xml
+++ b/extensions-core/druid-kerberos/pom.xml
@@ -18,8 +18,7 @@
   ~ 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";>
+<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>
 
   <groupId>io.druid.extensions</groupId>
@@ -30,7 +29,7 @@
   <parent>
     <groupId>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java
index 785a5a415cb..df799c09df5 100644
--- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java
+++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java
@@ -23,6 +23,8 @@
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import io.druid.guice.annotations.Self;
 import io.druid.java.util.common.StringUtils;
@@ -41,6 +43,8 @@
 import org.apache.hadoop.security.authentication.util.Signer;
 import org.apache.hadoop.security.authentication.util.SignerException;
 import org.apache.hadoop.security.authentication.util.SignerSecretProvider;
+import org.eclipse.jetty.client.api.Request;
+import org.eclipse.jetty.http.HttpHeader;
 import sun.security.krb5.EncryptedData;
 import sun.security.krb5.EncryptionKey;
 import sun.security.krb5.internal.APReq;
@@ -72,18 +76,25 @@
 import javax.servlet.http.HttpServletResponse;
 import java.io.File;
 import java.io.IOException;
+import java.net.HttpCookie;
 import java.security.Principal;
+import java.text.SimpleDateFormat;
 import java.util.Collections;
+import java.util.Date;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
+import java.util.TimeZone;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
 
 @JsonTypeName("kerberos")
 public class KerberosAuthenticator implements Authenticator
@@ -91,40 +102,44 @@
   private static final Logger log = new Logger(KerberosAuthenticator.class);
   private static final Pattern HADOOP_AUTH_COOKIE_REGEX = Pattern.compile(".*p=(\\S+)&t=.*");
   public static final List<String> DEFAULT_EXCLUDED_PATHS = Collections.emptyList();
+  public static final String SIGNED_TOKEN_ATTRIBUTE = "signedToken";
 
   private final DruidNode node;
   private final String serverPrincipal;
   private final String serverKeytab;
-  private final String internalClientPrincipal;
-  private final String internalClientKeytab;
   private final String authToLocal;
   private final List<String> excludedPaths;
   private final String cookieSignatureSecret;
   private final String authorizerName;
+  private final String name;
   private LoginContext loginContext;
 
   @JsonCreator
   public KerberosAuthenticator(
       @JsonProperty("serverPrincipal") String serverPrincipal,
       @JsonProperty("serverKeytab") String serverKeytab,
-      @JsonProperty("internalClientPrincipal") String internalClientPrincipal,
-      @JsonProperty("internalClientKeytab") String internalClientKeytab,
       @JsonProperty("authToLocal") String authToLocal,
       @JsonProperty("excludedPaths") List<String> excludedPaths,
       @JsonProperty("cookieSignatureSecret") String cookieSignatureSecret,
       @JsonProperty("authorizerName") String authorizerName,
+      @JsonProperty("name") String name,
       @JacksonInject @Self DruidNode node
   )
   {
     this.node = node;
-    this.serverPrincipal = serverPrincipal;
     this.serverKeytab = serverKeytab;
-    this.internalClientPrincipal = internalClientPrincipal;
-    this.internalClientKeytab = internalClientKeytab;
     this.authToLocal = authToLocal == null ? "DEFAULT" : authToLocal;
     this.excludedPaths = excludedPaths == null ? DEFAULT_EXCLUDED_PATHS : excludedPaths;
     this.cookieSignatureSecret = cookieSignatureSecret;
     this.authorizerName = authorizerName;
+    this.name = Preconditions.checkNotNull(name);
+
+    try {
+      this.serverPrincipal = SecurityUtil.getServerPrincipal(serverPrincipal, node.getHost());
+    }
+    catch (Exception e) {
+      throw new RuntimeException(e);
+    }
   }
 
   @Override
@@ -259,7 +274,7 @@ public void doFilter(
           if (clientPrincipal != null) {
             request.setAttribute(
                 AuthConfig.DRUID_AUTHENTICATION_RESULT,
-                new AuthenticationResult(clientPrincipal, authorizerName, null)
+                new AuthenticationResult(clientPrincipal, authorizerName, name, null)
             );
           }
         }
@@ -333,7 +348,20 @@ public Principal getUserPrincipal()
                 createAuthCookie(httpResponse, signedToken, getCookieDomain(),
                                  getCookiePath(), token.getExpires(), isHttps
                 );
+                request.setAttribute(SIGNED_TOKEN_ATTRIBUTE, tokenToCookieString(
+                    signedToken,
+                    getCookieDomain(),
+                    getCookiePath(),
+                    token.getExpires(),
+                    !token.isExpired() && token.getExpires() > 0,
+                    isHttps
+                ));
               }
+              // Since this request is validated also set DRUID_AUTHENTICATION_RESULT
+              request.setAttribute(
+                  AuthConfig.DRUID_AUTHENTICATION_RESULT,
+                  new AuthenticationResult(token.getName(), authorizerName, name, null)
+              );
               doFilter(filterChain, httpRequest, httpResponse);
             }
           } else {
@@ -345,7 +373,7 @@ public Principal getUserPrincipal()
           errCode = HttpServletResponse.SC_FORBIDDEN;
           authenticationEx = ex;
           if (log.isDebugEnabled()) {
-            log.debug("Authentication exception: " + ex.getMessage(), ex);
+            log.debug(ex, "Authentication exception: " + ex.getMessage());
           } else {
             log.warn("Authentication exception: " + ex.getMessage());
           }
@@ -388,20 +416,12 @@ public Principal getUserPrincipal()
   public Map<String, String> getInitParameters()
   {
     Map<String, String> params = new HashMap<String, String>();
-    try {
-      params.put(
-          "kerberos.principal",
-          SecurityUtil.getServerPrincipal(serverPrincipal, node.getHost())
-      );
-      params.put("kerberos.keytab", serverKeytab);
-      params.put(AuthenticationFilter.AUTH_TYPE, DruidKerberosAuthenticationHandler.class.getName());
-      params.put("kerberos.name.rules", authToLocal);
-      if (cookieSignatureSecret != null) {
-        params.put("signature.secret", cookieSignatureSecret);
-      }
-    }
-    catch (IOException e) {
-      Throwables.propagate(e);
+    params.put("kerberos.principal", serverPrincipal);
+    params.put("kerberos.keytab", serverKeytab);
+    params.put(AuthenticationFilter.AUTH_TYPE, DruidKerberosAuthenticationHandler.class.getName());
+    params.put("kerberos.name.rules", authToLocal);
+    if (cookieSignatureSecret != null) {
+      params.put("signature.secret", cookieSignatureSecret);
     }
     return params;
   }
@@ -440,6 +460,22 @@ private boolean isExcluded(String path)
     return false;
   }
 
+  @Override
+  public void decorateProxyRequest(
+      HttpServletRequest clientRequest, HttpServletResponse proxyResponse, Request proxyRequest
+  )
+  {
+    Object cookieToken = clientRequest.getAttribute(SIGNED_TOKEN_ATTRIBUTE);
+    if (cookieToken != null && cookieToken instanceof String) {
+      log.debug("Found cookie token will attache it to proxyRequest as cookie");
+      String authResult = (String) cookieToken;
+      String existingCookies = proxyRequest.getCookies()
+                                           .stream()
+                                           .map(HttpCookie::toString)
+                                           .collect(Collectors.joining(";"));
+      proxyRequest.header(HttpHeader.COOKIE, Joiner.on(";").join(authResult, existingCookies));
+    }
+  }
 
   /**
    * Kerberos context configuration for the JDK GSS library. Copied from hadoop-auth's KerberosAuthenticationHandler.
@@ -533,8 +569,8 @@ private String getPrincipalFromRequestNew(HttpServletRequest req)
             for (Object cred : serverCreds) {
               if (cred instanceof KeyTab) {
                 KeyTab serverKeyTab = (KeyTab) cred;
-                KerberosPrincipal serverPrincipal = new KerberosPrincipal(this.serverPrincipal);
-                KerberosKey[] serverKeys = serverKeyTab.getKeys(serverPrincipal);
+                KerberosPrincipal kerberosPrincipal = new KerberosPrincipal(serverPrincipal);
+                KerberosKey[] serverKeys = serverKeyTab.getKeys(kerberosPrincipal);
                 for (KerberosKey key : serverKeys) {
                   if (key.getKeyType() == eType) {
                     finalKey = new EncryptionKey(key.getKeyType(), key.getEncoded());
@@ -573,12 +609,10 @@ private boolean isValueAPReq(DerValue value)
 
   private void initializeKerberosLogin() throws ServletException
   {
-    String principal;
     String keytab;
 
     try {
-      principal = SecurityUtil.getServerPrincipal(serverPrincipal, node.getHost());
-      if (principal == null || principal.trim().length() == 0) {
+      if (serverPrincipal == null || serverPrincipal.trim().length() == 0) {
         throw new ServletException("Principal not defined in configuration");
       }
       keytab = serverKeytab;
@@ -590,19 +624,85 @@ private void initializeKerberosLogin() throws ServletException
       }
 
       Set<Principal> principals = new HashSet<Principal>();
-      principals.add(new KerberosPrincipal(principal));
+      principals.add(new KerberosPrincipal(serverPrincipal));
       Subject subject = new Subject(false, principals, new HashSet<Object>(), new HashSet<Object>());
 
-      DruidKerberosConfiguration kerberosConfiguration = new DruidKerberosConfiguration(keytab, principal);
+      DruidKerberosConfiguration kerberosConfiguration = new DruidKerberosConfiguration(keytab, serverPrincipal);
 
-      log.info("Login using keytab " + keytab + ", for principal " + principal);
+      log.info("Login using keytab " + keytab + ", for principal " + serverPrincipal);
       loginContext = new LoginContext("", subject, null, kerberosConfiguration);
       loginContext.login();
 
-      log.info("Initialized, principal %s from keytab %s", principal, keytab);
+      log.info("Initialized, principal %s from keytab %s", serverPrincipal, keytab);
     }
     catch (Exception ex) {
       throw new ServletException(ex);
     }
   }
+
+  /**
+   * Creates the Hadoop authentication HTTP cookie.
+   *
+   * @param resp the response object.
+   * @param token authentication token for the cookie.
+   * @param domain the cookie domain.
+   * @param path the cookie path.
+   * @param expires UNIX timestamp that indicates the expire date of the
+   *                cookie. It has no effect if its value &lt; 0.
+   * @param isSecure is the cookie secure?
+   * @param isCookiePersistent whether the cookie is persistent or not.
+   *the following code copy/past from Hadoop 3.0.0 copied to avoid compilation issue due to new signature,
+   *                           org.apache.hadoop.security.authentication.server.AuthenticationFilter#createAuthCookie
+   *                           (
+   *                           javax.servlet.http.HttpServletResponse,
+   *                           java.lang.String,
+   *                           java.lang.String,
+   *                           java.lang.String,
+   *                           long, boolean, boolean)
+   */
+  private static void tokenToAuthCookie(
+      HttpServletResponse resp, String token,
+      String domain, String path, long expires,
+      boolean isCookiePersistent,
+      boolean isSecure
+  )
+  {
+    resp.addHeader("Set-Cookie", tokenToCookieString(token, domain, path, expires, isCookiePersistent, isSecure));
+  }
+
+  private static String tokenToCookieString(
+      String token,
+      String domain, String path, long expires,
+      boolean isCookiePersistent,
+      boolean isSecure
+  )
+  {
+    StringBuilder sb = new StringBuilder(AuthenticatedURL.AUTH_COOKIE)
+        .append("=");
+    if (token != null && token.length() > 0) {
+      sb.append("\"").append(token).append("\"");
+    }
+
+    if (path != null) {
+      sb.append("; Path=").append(path);
+    }
+
+    if (domain != null) {
+      sb.append("; Domain=").append(domain);
+    }
+
+    if (expires >= 0 && isCookiePersistent) {
+      Date date = new Date(expires);
+      SimpleDateFormat df = new SimpleDateFormat("EEE, dd-MMM-yyyy HH:mm:ss zzz", Locale.ENGLISH);
+      df.setTimeZone(TimeZone.getTimeZone("GMT"));
+      sb.append("; Expires=").append(df.format(date));
+    }
+
+    if (isSecure) {
+      sb.append("; Secure");
+    }
+
+    sb.append("; HttpOnly");
+    return sb.toString();
+  }
 }
diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosEscalator.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosEscalator.java
index e995233a23c..3b25f5e8af2 100644
--- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosEscalator.java
+++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosEscalator.java
@@ -22,20 +22,10 @@
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Throwables;
-import com.metamx.http.client.HttpClient;
 import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.http.client.HttpClient;
 import io.druid.server.security.AuthenticationResult;
 import io.druid.server.security.Escalator;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.eclipse.jetty.client.api.Authentication;
-import org.eclipse.jetty.client.api.ContentResponse;
-import org.eclipse.jetty.client.api.Request;
-import org.eclipse.jetty.util.Attributes;
-import org.jboss.netty.handler.codec.http.HttpHeaders;
-
-import java.net.URI;
-import java.security.PrivilegedExceptionAction;
 
 @JsonTypeName("kerberos")
 public class KerberosEscalator implements Escalator
@@ -64,71 +54,12 @@ public HttpClient createEscalatedClient(HttpClient baseClient)
     return new KerberosHttpClient(baseClient, internalClientPrincipal, internalClientKeytab);
   }
 
-  @Override
-  public org.eclipse.jetty.client.HttpClient createEscalatedJettyClient(org.eclipse.jetty.client.HttpClient baseClient)
-  {
-    baseClient.getAuthenticationStore().addAuthentication(new Authentication()
-    {
-      @Override
-      public boolean matches(String type, URI uri, String realm)
-      {
-        return true;
-      }
-
-      @Override
-      public Result authenticate(
-          final Request request, ContentResponse response, Authentication.HeaderInfo headerInfo, Attributes context
-      )
-      {
-        return new Result()
-        {
-          @Override
-          public URI getURI()
-          {
-            return request.getURI();
-          }
-
-          @Override
-          public void apply(Request request)
-          {
-            try {
-              // No need to set cookies as they are handled by Jetty Http Client itself.
-              URI uri = request.getURI();
-              if (DruidKerberosUtil.needToSendCredentials(baseClient.getCookieStore(), uri)) {
-                log.debug(
-                    "No Auth Cookie found for URI[%s]. Existing Cookies[%s] Authenticating... ",
-                    uri,
-                    baseClient.getCookieStore().getCookies()
-                );
-                final String host = request.getHost();
-                DruidKerberosUtil.authenticateIfRequired(internalClientPrincipal, internalClientKeytab);
-                UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
-                String challenge = currentUser.doAs(new PrivilegedExceptionAction<String>()
-                {
-                  @Override
-                  public String run() throws Exception
-                  {
-                    return DruidKerberosUtil.kerberosChallenge(host);
-                  }
-                });
-                request.getHeaders().add(HttpHeaders.Names.AUTHORIZATION, "Negotiate " + challenge);
-              } else {
-                log.debug("Found Auth Cookie found for URI[%s].", uri);
-              }
-            }
-            catch (Throwable e) {
-              Throwables.propagate(e);
-            }
-          }
-        };
-      }
-    });
-    return baseClient;
-  }
-
   @Override
   public AuthenticationResult createEscalatedAuthenticationResult()
   {
-    return new AuthenticationResult(internalClientPrincipal, authorizerName, null);
+    // if you found your self asking why the authenticatedBy field is set to null please read this:
+    // https://github.com/druid-io/druid/pull/5706#discussion_r185940889
+    return new AuthenticationResult(internalClientPrincipal, authorizerName, null, null);
   }
+
 }
diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java
index c79deaf60e3..b1a66a8b2a7 100644
--- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java
+++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java
@@ -24,10 +24,10 @@
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
-import com.metamx.http.client.AbstractHttpClient;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.HttpResponseHandler;
+import io.druid.java.util.http.client.AbstractHttpClient;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.HttpResponseHandler;
 import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.logger.Logger;
 import org.apache.hadoop.security.UserGroupInformation;
diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/ResponseCookieHandler.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/ResponseCookieHandler.java
index 5a4d4df0b43..755cdb96547 100644
--- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/ResponseCookieHandler.java
+++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/ResponseCookieHandler.java
@@ -21,8 +21,8 @@
 
 import com.google.common.base.Function;
 import com.google.common.collect.Maps;
-import com.metamx.http.client.response.ClientResponse;
-import com.metamx.http.client.response.HttpResponseHandler;
+import io.druid.java.util.http.client.response.ClientResponse;
+import io.druid.java.util.http.client.response.HttpResponseHandler;
 import io.druid.java.util.common.logger.Logger;
 import org.jboss.netty.handler.codec.http.HttpChunk;
 import org.jboss.netty.handler.codec.http.HttpHeaders;
diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java
index 8d8148842c8..01a7987e2ba 100644
--- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java
+++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java
@@ -19,8 +19,8 @@
 
 package io.druid.security.kerberos;
 
-import com.metamx.http.client.response.ClientResponse;
-import com.metamx.http.client.response.HttpResponseHandler;
+import io.druid.java.util.http.client.response.ClientResponse;
+import io.druid.java.util.http.client.response.HttpResponseHandler;
 import io.druid.java.util.common.logger.Logger;
 import org.jboss.netty.handler.codec.http.HttpChunk;
 import org.jboss.netty.handler.codec.http.HttpResponse;
diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterConfig.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterConfig.java
deleted file mode 100644
index 4dd4f9b01f9..00000000000
--- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterConfig.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * Licensed to Metamarkets Group Inc. (Metamarkets) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. Metamarkets 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 io.druid.security.kerberos;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.util.Collections;
-import java.util.List;
-
-public class SpnegoFilterConfig
-{
-
-  public static final List<String> DEFAULT_EXCLUDED_PATHS = Collections.emptyList();
-
-  @JsonProperty
-  private final String principal;
-
-  @JsonProperty
-  private final String keytab;
-
-  @JsonProperty
-  private final String authToLocal;
-
-  @JsonProperty
-  private final List<String> excludedPaths;
-
-  @JsonProperty
-  private final String cookieSignatureSecret;
-
-  @JsonCreator
-  public SpnegoFilterConfig(
-      @JsonProperty("principal") String principal,
-      @JsonProperty("keytab") String keytab,
-      @JsonProperty("authToLocal") String authToLocal,
-      @JsonProperty("excludedPaths") List<String> excludedPaths,
-      @JsonProperty("cookieSignatureSecret") String cookieSignatureSecret
-  )
-  {
-    this.principal = principal;
-    this.keytab = keytab;
-    this.authToLocal = authToLocal == null ? "DEFAULT" : authToLocal;
-    this.excludedPaths = excludedPaths == null ? DEFAULT_EXCLUDED_PATHS : excludedPaths;
-    this.cookieSignatureSecret = cookieSignatureSecret;
-  }
-
-  @JsonProperty
-  public String getPrincipal()
-  {
-    return principal;
-  }
-
-  @JsonProperty
-  public String getKeytab()
-  {
-    return keytab;
-  }
-
-  @JsonProperty
-  public String getAuthToLocal()
-  {
-    return authToLocal;
-  }
-
-  @JsonProperty
-  public List<String> getExcludedPaths()
-  {
-    return excludedPaths;
-  }
-
-  @JsonProperty
-  public String getCookieSignatureSecret()
-  {
-    return cookieSignatureSecret;
-  }
-
-  @Override
-  public boolean equals(Object o)
-  {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-
-    SpnegoFilterConfig that = (SpnegoFilterConfig) o;
-
-    if (principal != null ? !principal.equals(that.principal) : that.principal != null) {
-      return false;
-    }
-    if (keytab != null ? !keytab.equals(that.keytab) : that.keytab != null) {
-      return false;
-    }
-    if (authToLocal != null ? !authToLocal.equals(that.authToLocal) : that.authToLocal != null) {
-      return false;
-    }
-    if (excludedPaths != null ? !excludedPaths.equals(that.excludedPaths) : that.excludedPaths != null) {
-      return false;
-    }
-    return cookieSignatureSecret != null
-           ? cookieSignatureSecret.equals(that.cookieSignatureSecret)
-           : that.cookieSignatureSecret == null;
-
-  }
-
-  @Override
-  public int hashCode()
-  {
-    int result = principal != null ? principal.hashCode() : 0;
-    result = 31 * result + (keytab != null ? keytab.hashCode() : 0);
-    result = 31 * result + (authToLocal != null ? authToLocal.hashCode() : 0);
-    result = 31 * result + (excludedPaths != null ? excludedPaths.hashCode() : 0);
-    result = 31 * result + (cookieSignatureSecret != null ? cookieSignatureSecret.hashCode() : 0);
-    return result;
-  }
-}
diff --git a/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/SpnegoFilterConfigTest.java b/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/SpnegoFilterConfigTest.java
deleted file mode 100644
index e1ee9860632..00000000000
--- a/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/SpnegoFilterConfigTest.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to Metamarkets Group Inc. (Metamarkets) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. Metamarkets 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 io.druid.security.kerberos;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.inject.Binder;
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.Module;
-import com.google.inject.Provides;
-import io.druid.guice.ConfigModule;
-import io.druid.guice.DruidGuiceExtensions;
-import io.druid.guice.JsonConfigProvider;
-import io.druid.guice.LazySingleton;
-import io.druid.guice.PropertiesModule;
-import io.druid.jackson.DefaultObjectMapper;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.Properties;
-
-public class SpnegoFilterConfigTest
-{
-  @Test
-  public void testserde()
-  {
-    Injector injector = Guice.createInjector(
-        new Module()
-        {
-          @Override
-          public void configure(Binder binder)
-          {
-            binder.install(new PropertiesModule(Arrays.asList("test.runtime.properties")));
-            binder.install(new ConfigModule());
-            binder.install(new DruidGuiceExtensions());
-            JsonConfigProvider.bind(binder, "druid.hadoop.security.spnego", SpnegoFilterConfig.class);
-          }
-
-          @Provides
-          @LazySingleton
-          public ObjectMapper jsonMapper()
-          {
-            return new DefaultObjectMapper();
-          }
-        }
-    );
-
-    Properties props = injector.getInstance(Properties.class);
-    SpnegoFilterConfig config = injector.getInstance(SpnegoFilterConfig.class);
-
-    Assert.assertEquals(props.getProperty("druid.hadoop.security.spnego.principal"), config.getPrincipal());
-    Assert.assertEquals(props.getProperty("druid.hadoop.security.spnego.keytab"), config.getKeytab());
-    Assert.assertEquals(props.getProperty("druid.hadoop.security.spnego.authToLocal"), config.getAuthToLocal());
-
-
-  }
-}
diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml
index 5b6bb1cd432..9510c5e6d38 100644
--- a/extensions-core/hdfs-storage/pom.xml
+++ b/extensions-core/hdfs-storage/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
@@ -44,6 +44,12 @@
             <version>${project.parent.version}</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+          <groupId>io.druid</groupId>
+          <artifactId>java-util</artifactId>
+          <version>${project.parent.version}</version>
+          <scope>provided</scope>
+        </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client</artifactId>
@@ -135,11 +141,6 @@
               </exclusion>
             </exclusions>
         </dependency>
-        <dependency>
-            <groupId>com.metamx</groupId>
-            <artifactId>java-util</artifactId>
-            <scope>provided</scope>
-        </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-aws</artifactId>
diff --git a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentFinder.java b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentFinder.java
index 6fba009cf86..8e75b36057c 100644
--- a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentFinder.java
+++ b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentFinder.java
@@ -20,13 +20,14 @@
 package io.druid.storage.hdfs;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.Sets;
+import com.google.common.base.Preconditions;
 import com.google.inject.Inject;
+import io.druid.java.util.common.Pair;
+import io.druid.java.util.common.StringUtils;
 import io.druid.java.util.common.logger.Logger;
 import io.druid.segment.loading.DataSegmentFinder;
 import io.druid.segment.loading.SegmentLoadingException;
 import io.druid.timeline.DataSegment;
-import io.druid.java.util.common.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -34,14 +35,15 @@
 import org.apache.hadoop.fs.RemoteIterator;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 /**
  */
 public class HdfsDataSegmentFinder implements DataSegmentFinder
 {
-
   private static final Logger log = new Logger(HdfsDataSegmentFinder.class);
 
   private final Configuration config;
@@ -58,7 +60,7 @@ public HdfsDataSegmentFinder(Configuration config, ObjectMapper mapper)
   public Set<DataSegment> findSegments(String workingDirPathStr, boolean updateDescriptor)
       throws SegmentLoadingException
   {
-    final Set<DataSegment> segments = Sets.newHashSet();
+    final Map<String, Pair<DataSegment, Long>> timestampedSegments = new HashMap<>();
     final Path workingDirPath = new Path(workingDirPathStr);
     FileSystem fs;
     try {
@@ -80,15 +82,31 @@ public HdfsDataSegmentFinder(Configuration config, ObjectMapper mapper)
         final LocatedFileStatus locatedFileStatus = it.next();
         final Path path = locatedFileStatus.getPath();
         if (path.getName().endsWith("descriptor.json")) {
-          final Path indexZip;
+
+          // There are 3 supported path formats:
+          //    - hdfs://nn1/hdfs_base_directory/data_source_name/interval/version/shardNum/descriptor.json
+          //    - hdfs://nn1/hdfs_base_directory/data_source_name/interval/version/shardNum_descriptor.json
+          //    - hdfs://nn1/hdfs_base_directory/data_source_name/interval/version/shardNum_UUID_descriptor.json
           final String descriptorParts[] = path.getName().split("_");
-          if (descriptorParts.length == 2
-              && descriptorParts[1].equals("descriptor.json")
-              && org.apache.commons.lang.StringUtils.isNumeric(descriptorParts[0])) {
-            indexZip = new Path(path.getParent(), StringUtils.format("%s_index.zip", descriptorParts[0]));
-          } else {
-            indexZip = new Path(path.getParent(), "index.zip");
+
+          Path indexZip = new Path(path.getParent(), "index.zip");
+          if (descriptorParts.length > 1) {
+            Preconditions.checkState(descriptorParts.length <= 3 &&
+                                     org.apache.commons.lang.StringUtils.isNumeric(descriptorParts[0]) &&
+                                     "descriptor.json".equals(descriptorParts[descriptorParts.length - 1]),
+                                     "Unexpected descriptor filename format [%s]", path
+            );
+
+            indexZip = new Path(
+                path.getParent(),
+                StringUtils.format(
+                    "%s_%sindex.zip",
+                    descriptorParts[0],
+                    descriptorParts.length == 2 ? "" : descriptorParts[1] + "_"
+                )
+            );
           }
+
           if (fs.exists(indexZip)) {
             final DataSegment dataSegment = mapper.readValue(fs.open(path), DataSegment.class);
             log.info("Found segment [%s] located at [%s]", dataSegment.getIdentifier(), indexZip);
@@ -105,7 +123,12 @@ public HdfsDataSegmentFinder(Configuration config, ObjectMapper mapper)
                 mapper.writeValue(fs.create(path, true), dataSegment);
               }
             }
-            segments.add(dataSegment);
+
+            DataSegmentFinder.putInMapRetainingNewest(
+                timestampedSegments,
+                dataSegment,
+                locatedFileStatus.getModificationTime()
+            );
           } else {
             throw new SegmentLoadingException(
                 "index.zip didn't exist at [%s] while descripter.json exists!?",
@@ -119,7 +142,6 @@ public HdfsDataSegmentFinder(Configuration config, ObjectMapper mapper)
       throw new SegmentLoadingException(e, "Problems interacting with filesystem[%s].", workingDirPath);
     }
 
-    return segments;
+    return timestampedSegments.values().stream().map(x -> x.lhs).collect(Collectors.toSet());
   }
-
 }
diff --git a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java
index d910ae51b0a..6cbd1eccd7c 100644
--- a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java
+++ b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java
@@ -19,8 +19,9 @@
 
 package io.druid.storage.hdfs;
 
+import com.google.common.base.Preconditions;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.segment.loading.DataSegmentKiller;
 import io.druid.segment.loading.SegmentLoadingException;
 import io.druid.timeline.DataSegment;
@@ -57,65 +58,53 @@ private static Path getPath(DataSegment segment)
   public void kill(DataSegment segment) throws SegmentLoadingException
   {
     final Path segmentPath = getPath(segment);
-    log.info("killing segment[%s] mapped to path[%s]", segment.getIdentifier(), segmentPath);
+    log.info("Killing segment[%s] mapped to path[%s]", segment.getIdentifier(), segmentPath);
 
     try {
-      String segmentLocation = segmentPath.getName();
+      String filename = segmentPath.getName();
       final FileSystem fs = segmentPath.getFileSystem(config);
-      if (!segmentLocation.endsWith(".zip")) {
+      if (!filename.endsWith(".zip")) {
         throw new SegmentLoadingException("Unknown file type[%s]", segmentPath);
       } else {
 
         if (!fs.exists(segmentPath)) {
-          log.warn("Segment Path [%s] does not exist. It appears to have been deleted already.", segmentPath);
+          log.warn("Segment path [%s] does not exist", segmentPath);
           return;
         }
 
-        String[] zipParts = segmentLocation.split("_");
-        // for segments stored as hdfs://nn1/hdfs_base_directory/data_source_name/interval/version/shardNum_index.zip
-        if (zipParts.length == 2
-            && zipParts[1].equals("index.zip")
-            && StringUtils.isNumeric(zipParts[0])) {
-          if (!fs.delete(segmentPath, false)) {
-            throw new SegmentLoadingException(
-                "Unable to kill segment, failed to delete [%s]",
-                segmentPath.toString()
-            );
-          }
-          Path descriptorPath = new Path(
+        // There are 3 supported path formats:
+        //    - hdfs://nn1/hdfs_base_directory/data_source_name/interval/version/shardNum/index.zip
+        //    - hdfs://nn1/hdfs_base_directory/data_source_name/interval/version/shardNum_index.zip
+        //    - hdfs://nn1/hdfs_base_directory/data_source_name/interval/version/shardNum_UUID_index.zip
+        String[] zipParts = filename.split("_");
+
+        Path descriptorPath = new Path(segmentPath.getParent(), "descriptor.json");
+        if (zipParts.length > 1) {
+          Preconditions.checkState(zipParts.length <= 3 &&
+                                   StringUtils.isNumeric(zipParts[0]) &&
+                                   "index.zip".equals(zipParts[zipParts.length - 1]),
+                                   "Unexpected segmentPath format [%s]", segmentPath
+          );
+
+          descriptorPath = new Path(
               segmentPath.getParent(),
-              io.druid.java.util.common.StringUtils.format("%s_descriptor.json", zipParts[0])
+              io.druid.java.util.common.StringUtils.format(
+                  "%s_%sdescriptor.json",
+                  zipParts[0],
+                  zipParts.length == 2 ? "" : zipParts[1] + "_"
+              )
           );
-          //delete partitionNumber_descriptor.json
-          if (!fs.delete(descriptorPath, false)) {
-            throw new SegmentLoadingException(
-                "Unable to kill segment, failed to delete [%s]",
-                descriptorPath.toString()
-            );
-          }
-          //for segments stored as hdfs://nn1/hdfs_base_directory/data_source_name/interval/version/shardNum_index.zip
-          // max depth to look is 2, i.e version directory and interval.
-          mayBeDeleteParentsUpto(fs, segmentPath, 2);
-
-        } else { //for segments stored as hdfs://nn1/hdfs_base_directory/data_source_name/interval/version/shardNum/
-          // index.zip
-          if (!fs.delete(segmentPath, false)) {
-            throw new SegmentLoadingException(
-                "Unable to kill segment, failed to delete [%s]",
-                segmentPath.toString()
-            );
-          }
-          Path descriptorPath = new Path(segmentPath.getParent(), "descriptor.json");
-          if (!fs.delete(descriptorPath, false)) {
-            throw new SegmentLoadingException(
-                "Unable to kill segment, failed to delete [%s]",
-                descriptorPath.toString()
-            );
-          }
-          //for segments stored as hdfs://nn1/hdfs_base_directory/data_source_name/interval/version/shardNum/index.zip
-          //max depth to look is 3, i.e partition number directory,version directory and interval.
-          mayBeDeleteParentsUpto(fs, segmentPath, 3);
         }
+
+        if (!fs.delete(segmentPath, false)) {
+          throw new SegmentLoadingException("Unable to kill segment, failed to delete [%s]", segmentPath.toString());
+        }
+
+        if (!fs.delete(descriptorPath, false)) {
+          throw new SegmentLoadingException("Unable to kill segment, failed to delete [%s]", descriptorPath.toString());
+        }
+
+        removeEmptyParentDirectories(fs, segmentPath, zipParts.length > 1 ? 2 : 3);
       }
     }
     catch (IOException e) {
@@ -131,11 +120,11 @@ public void killAll() throws IOException
     fs.delete(storageDirectory, true);
   }
 
-  private void mayBeDeleteParentsUpto(final FileSystem fs, final Path segmentPath, final int maxDepthTobeDeleted)
+  private void removeEmptyParentDirectories(final FileSystem fs, final Path segmentPath, final int depth)
   {
     Path path = segmentPath;
     try {
-      for (int i = 1; i <= maxDepthTobeDeleted; i++) {
+      for (int i = 1; i <= depth; i++) {
         path = path.getParent();
         if (fs.listStatus(path).length != 0 || !fs.delete(path, false)) {
           break;
diff --git a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java
index 33a85ac2147..94b41e70e19 100644
--- a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java
+++ b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java
@@ -20,6 +20,7 @@
 package io.druid.storage.hdfs;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.io.ByteSink;
 import com.google.common.io.ByteSource;
@@ -57,11 +58,8 @@
   private final String fullyQualifiedStorageDirectory;
 
   @Inject
-  public HdfsDataSegmentPusher(
-      HdfsDataSegmentPusherConfig config,
-      Configuration hadoopConfig,
-      ObjectMapper jsonMapper
-  ) throws IOException
+  public HdfsDataSegmentPusher(HdfsDataSegmentPusherConfig config, Configuration hadoopConfig, ObjectMapper jsonMapper)
+      throws IOException
   {
     this.config = config;
     this.hadoopConfig = hadoopConfig;
@@ -89,9 +87,11 @@ public String getPathForHadoop()
   }
 
   @Override
-  public DataSegment push(File inDir, DataSegment segment) throws IOException
+  public DataSegment push(final File inDir, final DataSegment segment, final boolean useUniquePath) throws IOException
   {
-    final String storageDir = this.getStorageDir(segment);
+    // For HDFS, useUniquePath does not affect the directory tree but instead affects the filename, which is of the form
+    // '{partitionNum}_index.zip' without unique paths and '{partitionNum}_{UUID}_index.zip' with unique paths.
+    final String storageDir = this.getStorageDir(segment, false);
 
     log.info(
         "Copying segment[%s] to HDFS at location[%s/%s]",
@@ -116,17 +116,20 @@ public DataSegment push(File inDir, DataSegment segment) throws IOException
     final DataSegment dataSegment;
     try (FSDataOutputStream out = fs.create(tmpIndexFile)) {
       size = CompressionUtils.zip(inDir, out);
+      final String uniquePrefix = useUniquePath ? DataSegmentPusher.generateUniquePath() + "_" : "";
       final Path outIndexFile = new Path(StringUtils.format(
-          "%s/%s/%d_index.zip",
+          "%s/%s/%d_%sindex.zip",
           fullyQualifiedStorageDirectory,
           storageDir,
-          segment.getShardSpec().getPartitionNum()
+          segment.getShardSpec().getPartitionNum(),
+          uniquePrefix
       ));
       final Path outDescriptorFile = new Path(StringUtils.format(
-          "%s/%s/%d_descriptor.json",
+          "%s/%s/%d_%sdescriptor.json",
           fullyQualifiedStorageDirectory,
           storageDir,
-          segment.getShardSpec().getPartitionNum()
+          segment.getShardSpec().getPartitionNum(),
+          uniquePrefix
       ));
 
       dataSegment = segment.withLoadSpec(makeLoadSpec(outIndexFile.toUri()))
@@ -167,13 +170,12 @@ private void copyFilesWithChecks(final FileSystem fs, final Path from, final Pat
     if (!HadoopFsWrapper.rename(fs, from, to)) {
       if (fs.exists(to)) {
         log.info(
-            "Unable to rename temp Index file[%s] to final segment path [%s]. "
-            + "It is already pushed by a replica task.",
+            "Unable to rename temp file [%s] to segment path [%s], it may have already been pushed by a replica task.",
             from,
             to
         );
       } else {
-        throw new IOE("Failed to rename temp Index file[%s] and final segment path[%s] is not present.", from, to);
+        throw new IOE("Failed to rename temp file [%s] and final segment path [%s] is not present.", from, to);
       }
     }
   }
@@ -208,8 +210,17 @@ public OutputStream openStream() throws IOException
    */
 
   @Override
-  public String getStorageDir(DataSegment segment)
+  public String getStorageDir(DataSegment segment, boolean useUniquePath)
   {
+    // This is only called by HdfsDataSegmentPusher.push(), which will always set useUniquePath to false since any
+    // 'uniqueness' will be applied not to the directory but to the filename along with the shard number. This is done
+    // to avoid performance issues due to excessive HDFS directories. Hence useUniquePath is ignored here and we
+    // expect it to be false.
+    Preconditions.checkArgument(
+        !useUniquePath,
+        "useUniquePath must be false for HdfsDataSegmentPusher.getStorageDir()"
+    );
+
     return JOINER.join(
         segment.getDataSource(),
         StringUtils.format(
@@ -224,9 +235,10 @@ public String getStorageDir(DataSegment segment)
   @Override
   public String makeIndexPathName(DataSegment dataSegment, String indexName)
   {
+    // This is only called from Hadoop batch which doesn't require unique segment paths so set useUniquePath=false
     return StringUtils.format(
         "./%s/%d_%s",
-        this.getStorageDir(dataSegment),
+        this.getStorageDir(dataSegment, false),
         dataSegment.getShardSpec().getPartitionNum(),
         indexName
     );
diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java b/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java
index df00fdb7a0f..913adb0977b 100644
--- a/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java
+++ b/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java
@@ -36,14 +36,15 @@
   private HadoopFsWrapper() {}
 
   /**
-   * Same as FileSystem.rename(from, to, Options.Rename.NONE) . That is,
-   * it returns "false" when "to" directory already exists. It is different from FileSystem.rename(from, to)
-   * which moves "from" directory inside "to" directory if it already exists.
+   * Same as FileSystem.rename(from, to, Options.Rename). It is different from FileSystem.rename(from, to) which moves
+   * "from" directory inside "to" directory if it already exists.
    *
    * @param from
    * @param to
-   * @return
-   * @throws IOException
+   *
+   * @return true if operation succeeded, false if destination already exists
+   *
+   * @throws IOException if trying to overwrite a non-empty directory
    */
   public static boolean rename(FileSystem fs, Path from, Path to) throws IOException
   {
@@ -51,8 +52,8 @@ public static boolean rename(FileSystem fs, Path from, Path to) throws IOExcepti
       fs.rename(from, to, Options.Rename.NONE);
       return true;
     }
-    catch (IOException ex) {
-      log.warn(ex, "Failed to rename [%s] to [%s].", from, to);
+    catch (FileAlreadyExistsException ex) {
+      log.info(ex, "Destination exists while renaming [%s] to [%s]", from, to);
       return false;
     }
   }
diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentFinderTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentFinderTest.java
index 62612462252..5e052df1e6d 100644
--- a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentFinderTest.java
+++ b/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentFinderTest.java
@@ -25,6 +25,7 @@
 import com.google.common.collect.ImmutableMap;
 import io.druid.java.util.common.IOE;
 import io.druid.java.util.common.Intervals;
+import io.druid.java.util.common.StringUtils;
 import io.druid.segment.TestHelper;
 import io.druid.storage.hdfs.HdfsDataSegmentFinder;
 import io.druid.timeline.DataSegment;
@@ -278,6 +279,30 @@ public void testFindSegmentsFail() throws Exception
     hdfsDataSegmentFinder.findSegments(dataSourceDir.toString(), false);
   }
 
+  @Test
+  public void testPreferNewestSegment() throws Exception
+  {
+    dataSourceDir = new Path(new Path(uriBase), "/usr/replicaDataSource");
+    descriptor1 = new Path(dataSourceDir, StringUtils.format("interval1/v1/%d_%s_%s", 0, "older", DESCRIPTOR_JSON));
+    descriptor2 = new Path(dataSourceDir, StringUtils.format("interval1/v1/%d_%s_%s", 0, "newer", DESCRIPTOR_JSON));
+    indexZip1 = new Path(descriptor1.getParent(), StringUtils.format("%d_%s_%s", 0, "older", INDEX_ZIP));
+    indexZip2 = new Path(descriptor2.getParent(), StringUtils.format("%d_%s_%s", 0, "newer", INDEX_ZIP));
+
+    mapper.writeValue(fs.create(descriptor1), SEGMENT_1);
+    mapper.writeValue(fs.create(descriptor2), SEGMENT_1);
+
+    create(indexZip1);
+    Thread.sleep(1000);
+    create(indexZip2);
+
+    final Set<DataSegment> segments = new HdfsDataSegmentFinder(conf, mapper).findSegments(
+        dataSourceDir.toString(), false
+    );
+
+    Assert.assertEquals(1, segments.size());
+    Assert.assertEquals(indexZip2.toUri().getPath(), segments.iterator().next().getLoadSpec().get("path"));
+  }
+
   private String getDescriptorPath(DataSegment segment)
   {
     final Path indexzip = new Path(String.valueOf(segment.getLoadSpec().get("path")));
diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentKillerTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentKillerTest.java
index bdf499dcd90..dba43507a69 100644
--- a/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentKillerTest.java
+++ b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentKillerTest.java
@@ -26,13 +26,13 @@
 import io.druid.timeline.DataSegment;
 import io.druid.timeline.partition.NoneShardSpec;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.UUID;
 
 /**
  */
@@ -129,7 +129,10 @@ public String getStorageDirectory()
     Path interval1Dir = new Path(dataSourceDir, "intervalNew");
     Path version11Dir = new Path(interval1Dir, "v1");
 
-    makePartitionDirWithIndexWitNewFormat(fs, version11Dir, 3);
+    Assert.assertTrue(fs.mkdirs(version11Dir));
+    fs.createNewFile(new Path(version11Dir, StringUtils.format("%s_index.zip", 3)));
+    fs.createNewFile(new Path(version11Dir, StringUtils.format("%s_descriptor.json", 3)));
+
     killer.kill(getSegmentWithPath(new Path(version11Dir, "3_index.zip").toString()));
 
     Assert.assertFalse(fs.exists(version11Dir));
@@ -141,7 +144,7 @@ public String getStorageDirectory()
   }
 
   @Test
-  public void testKillNonExistingSegment() throws Exception
+  public void testKillForSegmentWithUniquePath() throws Exception
   {
     Configuration config = new Configuration();
     HdfsDataSegmentKiller killer = new HdfsDataSegmentKiller(
@@ -155,28 +158,51 @@ public String getStorageDirectory()
           }
         }
     );
-    killer.kill(getSegmentWithPath(new Path("/xxx/", "index.zip").toString()));
+
+    FileSystem fs = FileSystem.get(config);
+    Path dataSourceDir = new Path("/tmp/dataSourceNew");
+
+    Path interval1Dir = new Path(dataSourceDir, "intervalNew");
+    Path version11Dir = new Path(interval1Dir, "v1");
+    String uuid = UUID.randomUUID().toString().substring(0, 5);
+
+    Assert.assertTrue(fs.mkdirs(version11Dir));
+    fs.createNewFile(new Path(version11Dir, StringUtils.format("%s_%s_index.zip", 3, uuid)));
+    fs.createNewFile(new Path(version11Dir, StringUtils.format("%s_%s_descriptor.json", 3, uuid)));
+
+    killer.kill(getSegmentWithPath(new Path(version11Dir, StringUtils.format("%s_%s_index.zip", 3, uuid)).toString()));
+
+    Assert.assertFalse(fs.exists(version11Dir));
+    Assert.assertFalse(fs.exists(interval1Dir));
+    Assert.assertTrue(fs.exists(dataSourceDir));
+    Assert.assertTrue(fs.exists(new Path("/tmp")));
+    Assert.assertTrue(fs.exists(dataSourceDir));
+    Assert.assertTrue(fs.delete(dataSourceDir, false));
   }
 
-  private void makePartitionDirWithIndex(FileSystem fs, Path path) throws IOException
+  @Test
+  public void testKillNonExistingSegment() throws Exception
   {
-    Assert.assertTrue(fs.mkdirs(path));
-    try (FSDataOutputStream os = fs.create(new Path(path, "index.zip")); FSDataOutputStream oos = fs.create(new Path(
-        path,
-        "descriptor.json"
-    ))) {
-    }
+    Configuration config = new Configuration();
+    HdfsDataSegmentKiller killer = new HdfsDataSegmentKiller(
+        config,
+        new HdfsDataSegmentPusherConfig()
+        {
+          @Override
+          public String getStorageDirectory()
+          {
+            return "/tmp";
+          }
+        }
+    );
+    killer.kill(getSegmentWithPath(new Path("/xxx/", "index.zip").toString()));
   }
 
-  private void makePartitionDirWithIndexWitNewFormat(FileSystem fs, Path path, Integer partitionNumber)
-      throws IOException
+  private void makePartitionDirWithIndex(FileSystem fs, Path path) throws IOException
   {
     Assert.assertTrue(fs.mkdirs(path));
-    try (FSDataOutputStream os = fs.create(new Path(
-        path,
-        StringUtils.format("%s_index.zip", partitionNumber)
-    )); FSDataOutputStream oos = fs.create(new Path(path, StringUtils.format("%s_descriptor.json", partitionNumber)))) {
-    }
+    fs.createNewFile(new Path(path, "index.zip"));
+    fs.createNewFile(new Path(path, "descriptor.json"));
   }
 
   private DataSegment getSegmentWithPath(String path)
diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java
index 95dd85bf2b9..9bb25f90707 100644
--- a/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java
+++ b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java
@@ -127,7 +127,8 @@ public void testPushWithMultipleSegments() throws Exception
     testUsingSchemeForMultipleSegments("file", 3);
   }
 
-  private void testUsingScheme(final String scheme) throws Exception
+  @Test
+  public void testUsingUniqueFilePath() throws Exception
   {
     Configuration conf = new Configuration(true);
 
@@ -142,11 +143,7 @@ private void testUsingScheme(final String scheme) throws Exception
     HdfsDataSegmentPusherConfig config = new HdfsDataSegmentPusherConfig();
     final File storageDirectory = tempFolder.newFolder();
 
-    config.setStorageDirectory(
-        scheme != null
-        ? StringUtils.format("%s://%s", scheme, storageDirectory.getAbsolutePath())
-        : storageDirectory.getAbsolutePath()
-    );
+    config.setStorageDirectory(StringUtils.format("file://%s", storageDirectory.getAbsolutePath()));
     HdfsDataSegmentPusher pusher = new HdfsDataSegmentPusher(config, conf, new DefaultObjectMapper());
 
     DataSegment segmentToPush = new DataSegment(
@@ -161,51 +158,13 @@ private void testUsingScheme(final String scheme) throws Exception
         size
     );
 
-    DataSegment segment = pusher.push(segmentDir, segmentToPush);
-
+    DataSegment segment = pusher.push(segmentDir, segmentToPush, true);
 
-    String indexUri = StringUtils.format(
-        "%s/%s/%d_index.zip",
-        FileSystem.newInstance(conf).makeQualified(new Path(config.getStorageDirectory())).toUri().toString(),
-        pusher.getStorageDir(segmentToPush),
-        segmentToPush.getShardSpec().getPartitionNum()
+    String matcher = ".*/foo/20150101T000000\\.000Z_20160101T000000\\.000Z/0/0_[A-Za-z0-9-]{36}_index\\.zip";
+    Assert.assertTrue(
+        segment.getLoadSpec().get("path").toString(),
+        segment.getLoadSpec().get("path").toString().matches(matcher)
     );
-
-    Assert.assertEquals(segmentToPush.getSize(), segment.getSize());
-    Assert.assertEquals(segmentToPush, segment);
-    Assert.assertEquals(ImmutableMap.of(
-        "type",
-        "hdfs",
-        "path",
-        indexUri
-    ), segment.getLoadSpec());
-    // rename directory after push
-    final String segmentPath = pusher.getStorageDir(segment);
-
-    File indexFile = new File(StringUtils.format(
-        "%s/%s/%d_index.zip",
-        storageDirectory,
-        segmentPath,
-        segment.getShardSpec().getPartitionNum()
-    ));
-    Assert.assertTrue(indexFile.exists());
-    File descriptorFile = new File(StringUtils.format(
-        "%s/%s/%d_descriptor.json",
-        storageDirectory,
-        segmentPath,
-        segment.getShardSpec().getPartitionNum()
-    ));
-    Assert.assertTrue(descriptorFile.exists());
-
-    // push twice will fail and temp dir cleaned
-    File outDir = new File(StringUtils.format("%s/%s", config.getStorageDirectory(), segmentPath));
-    outDir.setReadOnly();
-    try {
-      pusher.push(segmentDir, segmentToPush);
-    }
-    catch (IOException e) {
-      Assert.fail("should not throw exception");
-    }
   }
 
   private void testUsingSchemeForMultipleSegments(final String scheme, final int numberOfSegments) throws Exception
@@ -246,12 +205,12 @@ private void testUsingSchemeForMultipleSegments(final String scheme, final int n
     }
 
     for (int i = 0; i < numberOfSegments; i++) {
-      final DataSegment pushedSegment = pusher.push(segmentDir, segments[i]);
+      final DataSegment pushedSegment = pusher.push(segmentDir, segments[i], false);
 
       String indexUri = StringUtils.format(
           "%s/%s/%d_index.zip",
           FileSystem.newInstance(conf).makeQualified(new Path(config.getStorageDirectory())).toUri().toString(),
-          pusher.getStorageDir(segments[i]),
+          pusher.getStorageDir(segments[i], false),
           segments[i].getShardSpec().getPartitionNum()
       );
 
@@ -264,7 +223,7 @@ private void testUsingSchemeForMultipleSegments(final String scheme, final int n
           indexUri
       ), pushedSegment.getLoadSpec());
       // rename directory after push
-      String segmentPath = pusher.getStorageDir(pushedSegment);
+      String segmentPath = pusher.getStorageDir(pushedSegment, false);
 
       File indexFile = new File(StringUtils.format(
           "%s/%s/%d_index.zip",
@@ -293,7 +252,7 @@ private void testUsingSchemeForMultipleSegments(final String scheme, final int n
           indexUri
       ), fromDescriptorFileDataSegment.getLoadSpec());
       // rename directory after push
-      segmentPath = pusher.getStorageDir(fromDescriptorFileDataSegment);
+      segmentPath = pusher.getStorageDir(fromDescriptorFileDataSegment, false);
 
       indexFile = new File(StringUtils.format(
           "%s/%s/%d_index.zip",
@@ -308,7 +267,7 @@ private void testUsingSchemeForMultipleSegments(final String scheme, final int n
       File outDir = new File(StringUtils.format("%s/%s", config.getStorageDirectory(), segmentPath));
       outDir.setReadOnly();
       try {
-        pusher.push(segmentDir, segments[i]);
+        pusher.push(segmentDir, segments[i], false);
       }
       catch (IOException e) {
         Assert.fail("should not throw exception");
@@ -316,6 +275,87 @@ private void testUsingSchemeForMultipleSegments(final String scheme, final int n
     }
   }
 
+  private void testUsingScheme(final String scheme) throws Exception
+  {
+    Configuration conf = new Configuration(true);
+
+    // Create a mock segment on disk
+    File segmentDir = tempFolder.newFolder();
+    File tmp = new File(segmentDir, "version.bin");
+
+    final byte[] data = new byte[]{0x0, 0x0, 0x0, 0x1};
+    Files.write(data, tmp);
+    final long size = data.length;
+
+    HdfsDataSegmentPusherConfig config = new HdfsDataSegmentPusherConfig();
+    final File storageDirectory = tempFolder.newFolder();
+
+    config.setStorageDirectory(
+        scheme != null
+        ? StringUtils.format("%s://%s", scheme, storageDirectory.getAbsolutePath())
+        : storageDirectory.getAbsolutePath()
+    );
+    HdfsDataSegmentPusher pusher = new HdfsDataSegmentPusher(config, conf, new DefaultObjectMapper());
+
+    DataSegment segmentToPush = new DataSegment(
+        "foo",
+        Intervals.of("2015/2016"),
+        "0",
+        Maps.<String, Object>newHashMap(),
+        Lists.<String>newArrayList(),
+        Lists.<String>newArrayList(),
+        NoneShardSpec.instance(),
+        0,
+        size
+    );
+
+    DataSegment segment = pusher.push(segmentDir, segmentToPush, false);
+
+
+    String indexUri = StringUtils.format(
+        "%s/%s/%d_index.zip",
+        FileSystem.newInstance(conf).makeQualified(new Path(config.getStorageDirectory())).toUri().toString(),
+        pusher.getStorageDir(segmentToPush, false),
+        segmentToPush.getShardSpec().getPartitionNum()
+    );
+
+    Assert.assertEquals(segmentToPush.getSize(), segment.getSize());
+    Assert.assertEquals(segmentToPush, segment);
+    Assert.assertEquals(ImmutableMap.of(
+        "type",
+        "hdfs",
+        "path",
+        indexUri
+    ), segment.getLoadSpec());
+    // rename directory after push
+    final String segmentPath = pusher.getStorageDir(segment, false);
+
+    File indexFile = new File(StringUtils.format(
+        "%s/%s/%d_index.zip",
+        storageDirectory,
+        segmentPath,
+        segment.getShardSpec().getPartitionNum()
+    ));
+    Assert.assertTrue(indexFile.exists());
+    File descriptorFile = new File(StringUtils.format(
+        "%s/%s/%d_descriptor.json",
+        storageDirectory,
+        segmentPath,
+        segment.getShardSpec().getPartitionNum()
+    ));
+    Assert.assertTrue(descriptorFile.exists());
+
+    // push twice will fail and temp dir cleaned
+    File outDir = new File(StringUtils.format("%s/%s", config.getStorageDirectory(), segmentPath));
+    outDir.setReadOnly();
+    try {
+      pusher.push(segmentDir, segmentToPush, false);
+    }
+    catch (IOException e) {
+      Assert.fail("should not throw exception");
+    }
+  }
+
   public static class TestObjectMapper extends ObjectMapper
   {
     public TestObjectMapper()
@@ -371,7 +411,7 @@ public void shouldNotHaveColonsInHdfsStorageDir() throws Exception
         1
     );
 
-    String storageDir = hdfsDataSegmentPusher.getStorageDir(segment);
+    String storageDir = hdfsDataSegmentPusher.getStorageDir(segment, false);
     Assert.assertEquals("something/20111001T000000.000Z_20111002T000000.000Z/brand_new_version", storageDir);
 
   }
diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml
index 5f0a3765d64..d31dfef639b 100644
--- a/extensions-core/histogram/pom.xml
+++ b/extensions-core/histogram/pom.xml
@@ -26,7 +26,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java
index 6711c36f4b1..b9c8d3d3d46 100644
--- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java
+++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java
@@ -93,15 +93,27 @@ public Aggregation toDruidAggregation(
         project,
         aggregateCall.getArgList().get(1)
     );
-    final float probability = ((Number) RexLiteral.value(probabilityArg)).floatValue();
 
+    if (!probabilityArg.isA(SqlKind.LITERAL)) {
+      // Probability must be a literal in order to plan.
+      return null;
+    }
+
+    final float probability = ((Number) RexLiteral.value(probabilityArg)).floatValue();
     final int resolution;
+
     if (aggregateCall.getArgList().size() >= 3) {
       final RexNode resolutionArg = Expressions.fromFieldAccess(
           rowSignature,
           project,
           aggregateCall.getArgList().get(2)
       );
+
+      if (!resolutionArg.isA(SqlKind.LITERAL)) {
+        // Resolution must be a literal in order to plan.
+        return null;
+      }
+
       resolution = ((Number) RexLiteral.value(resolutionArg)).intValue();
     } else {
       resolution = ApproximateHistogram.DEFAULT_HISTOGRAM_SIZE;
diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
index 45197c2c7e0..77f629e5a1a 100644
--- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
+++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
@@ -27,6 +27,7 @@
 import io.druid.java.util.common.guava.Sequences;
 import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
 import io.druid.query.Druids;
+import io.druid.query.QueryDataSource;
 import io.druid.query.aggregation.CountAggregatorFactory;
 import io.druid.query.aggregation.DoubleSumAggregatorFactory;
 import io.druid.query.aggregation.FilteredAggregatorFactory;
@@ -34,9 +35,13 @@
 import io.druid.query.aggregation.histogram.ApproximateHistogramDruidModule;
 import io.druid.query.aggregation.histogram.ApproximateHistogramFoldingAggregatorFactory;
 import io.druid.query.aggregation.histogram.QuantilePostAggregator;
+import io.druid.query.aggregation.post.ArithmeticPostAggregator;
+import io.druid.query.aggregation.post.FieldAccessPostAggregator;
+import io.druid.query.dimension.DefaultDimensionSpec;
 import io.druid.query.expression.TestExprMacroTable;
 import io.druid.query.filter.NotDimFilter;
 import io.druid.query.filter.SelectorDimFilter;
+import io.druid.query.groupby.GroupByQuery;
 import io.druid.query.spec.MultipleIntervalSegmentSpec;
 import io.druid.segment.IndexBuilder;
 import io.druid.segment.QueryableIndex;
@@ -293,4 +298,69 @@ public void testQuantileOnComplexColumn() throws Exception
       );
     }
   }
+
+  @Test
+  public void testQuantileOnInnerQuery() throws Exception
+  {
+    try (final DruidPlanner planner = plannerFactory.createPlanner(null)) {
+      final String sql = "SELECT AVG(x), APPROX_QUANTILE(x, 0.98)\n"
+                         + "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)";
+
+      final PlannerResult plannerResult = planner.plan(sql);
+
+      // Verify results
+      final List<Object[]> results = Sequences.toList(plannerResult.run(), new ArrayList<Object[]>());
+      final List<Object[]> expectedResults = ImmutableList.of(
+          new Object[]{7.0, 8.26386833190918}
+      );
+      Assert.assertEquals(expectedResults.size(), results.size());
+      for (int i = 0; i < expectedResults.size(); i++) {
+        Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
+      }
+
+      // Verify query
+      Assert.assertEquals(
+          GroupByQuery.builder()
+                      .setDataSource(
+                          new QueryDataSource(
+                              GroupByQuery.builder()
+                                          .setDataSource(CalciteTests.DATASOURCE1)
+                                          .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                                          .setGranularity(Granularities.ALL)
+                                          .setDimensions(ImmutableList.of(new DefaultDimensionSpec("dim2", "d0")))
+                                          .setAggregatorSpecs(
+                                              ImmutableList.of(
+                                                  new DoubleSumAggregatorFactory("a0", "m1")
+                                              )
+                                          )
+                                          .setContext(ImmutableMap.of())
+                                          .build()
+                          )
+                      )
+                      .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
+                      .setGranularity(Granularities.ALL)
+                      .setAggregatorSpecs(ImmutableList.of(
+                          new DoubleSumAggregatorFactory("_a0:sum", "a0"),
+                          new CountAggregatorFactory("_a0:count"),
+                          new ApproximateHistogramAggregatorFactory("_a1:agg", "a0", null, null, null, null)
+                      ))
+                      .setPostAggregatorSpecs(
+                          ImmutableList.of(
+                              new ArithmeticPostAggregator(
+                                  "_a0",
+                                  "quotient",
+                                  ImmutableList.of(
+                                      new FieldAccessPostAggregator(null, "_a0:sum"),
+                                      new FieldAccessPostAggregator(null, "_a0:count")
+                                  )
+                              ),
+                              new QuantilePostAggregator("_a1", "_a1:agg", 0.98f)
+                          )
+                      )
+                      .setContext(ImmutableMap.of())
+                      .build(),
+          Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
+      );
+    }
+  }
 }
diff --git a/extensions-core/kafka-eight/pom.xml b/extensions-core/kafka-eight/pom.xml
index 2b94032c587..4a3bd108c3e 100644
--- a/extensions-core/kafka-eight/pom.xml
+++ b/extensions-core/kafka-eight/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml
index 0edc753e9bc..0a9cd6e3fdb 100644
--- a/extensions-core/kafka-extraction-namespace/pom.xml
+++ b/extensions-core/kafka-extraction-namespace/pom.xml
@@ -18,8 +18,7 @@
   ~ 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";>
+<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>
   <groupId>io.druid.extensions</groupId>
   <artifactId>druid-kafka-extraction-namespace</artifactId>
@@ -29,7 +28,7 @@
   <parent>
     <groupId>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml
index 007336e282f..414f951d90d 100644
--- a/extensions-core/kafka-indexing-service/pom.xml
+++ b/extensions-core/kafka-indexing-service/pom.xml
@@ -29,7 +29,7 @@
   <parent>
     <groupId>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java
index d4fee282bde..d7bffedea17 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java
@@ -44,7 +44,7 @@
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.data.input.Committer;
 import io.druid.data.input.InputRow;
 import io.druid.data.input.impl.InputRowParser;
@@ -62,9 +62,11 @@
 import io.druid.indexing.common.task.AbstractTask;
 import io.druid.indexing.common.task.RealtimeIndexTask;
 import io.druid.indexing.common.task.TaskResource;
+import io.druid.indexing.common.task.Tasks;
 import io.druid.indexing.kafka.supervisor.KafkaSupervisor;
 import io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.Intervals;
 import io.druid.java.util.common.StringUtils;
 import io.druid.java.util.common.collect.Utils;
 import io.druid.java.util.common.concurrent.Execs;
@@ -81,9 +83,9 @@
 import io.druid.segment.realtime.FireDepartmentMetrics;
 import io.druid.segment.realtime.RealtimeMetricsMonitor;
 import io.druid.segment.realtime.appenderator.Appenderator;
-import io.druid.segment.realtime.appenderator.AppenderatorDriver;
 import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult;
 import io.druid.segment.realtime.appenderator.Appenderators;
+import io.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
 import io.druid.segment.realtime.appenderator.SegmentIdentifier;
 import io.druid.segment.realtime.appenderator.SegmentsAndMetadata;
 import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
@@ -135,7 +137,6 @@
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionException;
@@ -186,7 +187,7 @@
   private TaskToolbox toolbox;
 
   private volatile Appenderator appenderator = null;
-  private volatile AppenderatorDriver driver = null;
+  private volatile StreamAppenderatorDriver driver = null;
   private volatile FireDepartmentMetrics fireDepartmentMetrics = null;
   private volatile DateTime startTime;
   private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread)
@@ -306,6 +307,12 @@ private static String makeTaskId(String dataSource, int randomBits)
     return Joiner.on("_").join(TYPE, dataSource, suffix);
   }
 
+  @Override
+  public int getPriority()
+  {
+    return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY);
+  }
+
   @Override
   public String getType()
   {
@@ -371,7 +378,7 @@ private void createAndStartPublishExecutor()
                     Joiner.on(", ").join(
                         result.getSegments().stream().map(DataSegment::getIdentifier).collect(Collectors.toList())
                     ),
-                    result.getCommitMetadata()
+                    Preconditions.checkNotNull(result.getCommitMetadata(), "commitMetadata")
                 );
               }
 
@@ -422,9 +429,11 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception
     if (getContext() != null && getContext().get("checkpoints") != null) {
       log.info("Got checkpoints [%s]", (String) getContext().get("checkpoints"));
       final TreeMap<Integer, Map<Integer, Long>> checkpoints = toolbox.getObjectMapper().readValue(
-          (String) getContext().get("checkpoints"), new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
+          (String) getContext().get("checkpoints"),
+          new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
           {
-          });
+          }
+      );
 
       Iterator<Map.Entry<Integer, Map<Integer, Long>>> sequenceOffsets = checkpoints.entrySet().iterator();
       Map.Entry<Integer, Map<Integer, Long>> previous = sequenceOffsets.next();
@@ -511,13 +520,14 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception
       final Object restoredMetadata = driver.startJob();
       if (restoredMetadata == null) {
         // no persist has happened so far
+        // so either this is a brand new task or replacement of a failed task
         Preconditions.checkState(sequences.get(0).startOffsets.entrySet().stream().allMatch(
             partitionOffsetEntry -> Longs.compare(
                 partitionOffsetEntry.getValue(),
                 ioConfig.getStartPartitions()
                         .getPartitionOffsetMap()
                         .get(partitionOffsetEntry.getKey())
-            ) == 0
+            ) >= 0
         ), "Sequence offsets are not compatible with start offsets of task");
         nextOffsets.putAll(sequences.get(0).startOffsets);
       } else {
@@ -544,7 +554,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception
               ioConfig.getStartPartitions().getPartitionOffsetMap().keySet()
           );
         }
-        // sequences size can 0 only when all sequences got published and task stopped before it could finish
+        // sequences size can be 0 only when all sequences got published and task stopped before it could finish
         // which is super rare
         if (sequences.size() == 0 || sequences.get(sequences.size() - 1).isCheckpointed()) {
           this.endOffsets.putAll(sequences.size() == 0
@@ -707,7 +717,7 @@ public void run()
 
                     if (addResult.isOk()) {
                       // If the number of rows in the segment exceeds the threshold after adding a row,
-                      // move the segment out from the active segments of AppenderatorDriver to make a new segment.
+                      // move the segment out from the active segments of BaseAppenderatorDriver to make a new segment.
                       if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) {
                         if (!sequenceToUse.isCheckpointed()) {
                           sequenceToCheckpoint = sequenceToUse;
@@ -745,7 +755,6 @@ public void onFailure(Throwable t)
                         }
                       }
                   );
-
                 }
               }
               catch (ParseException e) {
@@ -848,7 +857,7 @@ public void onFailure(Throwable t)
               Joiner.on(", ").join(
                   handedOff.getSegments().stream().map(DataSegment::getIdentifier).collect(Collectors.toList())
               ),
-              handedOff.getCommitMetadata()
+              Preconditions.checkNotNull(handedOff.getCommitMetadata(), "commitMetadata")
           );
         }
       }
@@ -938,9 +947,9 @@ private TaskStatus runLegacy(final TaskToolbox toolbox) throws Exception
     );
 
     try (
-        final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox);
-        final AppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics);
-        final KafkaConsumer<byte[], byte[]> consumer = newConsumer()
+            final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox);
+            final StreamAppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics);
+            final KafkaConsumer<byte[], byte[]> consumer = newConsumer()
     ) {
       toolbox.getDataSegmentServerAnnouncer().announce();
       toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode);
@@ -1102,7 +1111,7 @@ public void run()
 
                     if (addResult.isOk()) {
                       // If the number of rows in the segment exceeds the threshold after adding a row,
-                      // move the segment out from the active segments of AppenderatorDriver to make a new segment.
+                      // move the segment out from the active segments of BaseAppenderatorDriver to make a new segment.
                       if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) {
                         segmentsToMoveOut.computeIfAbsent(sequenceName, k -> new HashSet<>())
                                          .add(addResult.getSegmentIdentifier());
@@ -1168,7 +1177,7 @@ public void run()
 
       final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> {
         final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue(
-            ((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS),
+            ((Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_NEXT_PARTITIONS),
             KafkaPartitions.class
         );
 
@@ -1228,7 +1237,7 @@ public String apply(DataSegment input)
                     }
                 )
             ),
-            handedOff.getCommitMetadata()
+            Preconditions.checkNotNull(handedOff.getCommitMetadata(), "commitMetadata")
         );
       }
     }
@@ -1267,7 +1276,7 @@ private void checkAndMaybeThrowException()
   }
 
   private void maybePersistAndPublishSequences(Supplier<Committer> committerSupplier)
-      throws ExecutionException, InterruptedException
+      throws InterruptedException
   {
     for (SequenceMetadata sequenceMetadata : sequences) {
       sequenceMetadata.updateAssignments(nextOffsets);
@@ -1791,17 +1800,18 @@ private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox
     );
   }
 
-  private AppenderatorDriver newDriver(
+  private StreamAppenderatorDriver newDriver(
       final Appenderator appenderator,
       final TaskToolbox toolbox,
       final FireDepartmentMetrics metrics
   )
   {
-    return new AppenderatorDriver(
+    return new StreamAppenderatorDriver(
         appenderator,
         new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema),
         toolbox.getSegmentHandoffNotifierFactory(),
         new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
+        toolbox.getDataSegmentKiller(),
         toolbox.getObjectMapper(),
         metrics
     );
@@ -2015,6 +2025,19 @@ private boolean withinMinMaxRecordTime(final InputRow row)
     final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent()
                                             && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp());
 
+    if (!Intervals.ETERNITY.contains(row.getTimestamp())) {
+      final String errorMsg = StringUtils.format(
+          "Encountered row with timestamp that cannot be represented as a long: [%s]",
+          row
+      );
+      log.debug(errorMsg);
+      if (tuningConfig.isReportParseExceptions()) {
+        throw new ParseException(errorMsg);
+      } else {
+        return false;
+      }
+    }
+
     if (log.isDebugEnabled()) {
       if (beforeMinimumMessageTime) {
         log.debug(
@@ -2207,7 +2230,7 @@ public TransactionalSegmentPublisher getPublisher(TaskToolbox toolbox, boolean u
     {
       return (segments, commitMetadata) -> {
         final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue(
-            ((Map) commitMetadata).get(METADATA_PUBLISH_PARTITIONS),
+            ((Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_PUBLISH_PARTITIONS),
             KafkaPartitions.class
         );
 
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java
index 30a06de8c17..6525d127631 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java
@@ -29,11 +29,11 @@
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.FullResponseHandler;
-import com.metamx.http.client.response.FullResponseHolder;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.FullResponseHandler;
+import io.druid.java.util.http.client.response.FullResponseHolder;
 import io.druid.indexing.common.RetryPolicy;
 import io.druid.indexing.common.RetryPolicyConfig;
 import io.druid.indexing.common.RetryPolicyFactory;
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java
index 86939264200..da4d8c369f9 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java
@@ -21,7 +21,7 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.inject.Inject;
-import com.metamx.http.client.HttpClient;
+import io.druid.java.util.http.client.HttpClient;
 import io.druid.guice.annotations.EscalatedGlobal;
 import io.druid.guice.annotations.Json;
 import io.druid.indexing.common.TaskInfoProvider;
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java
index d0b8ea55c24..7cc79f6a98a 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java
@@ -43,9 +43,9 @@
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.indexing.common.TaskInfoProvider;
 import io.druid.indexer.TaskLocation;
 import io.druid.indexing.common.TaskStatus;
@@ -670,7 +670,7 @@ public void handle() throws ExecutionException, InterruptedException, TimeoutExc
         // as when the task starts they are sent existing checkpoints
         Preconditions.checkState(
             checkpoints.size() <= 1,
-            "Got checkpoint request with null as previous check point, however found more than one checkpoints in metadata store"
+            "Got checkpoint request with null as previous check point, however found more than one checkpoints"
         );
         if (checkpoints.size() == 1) {
           log.info("Already checkpointed with dataSourceMetadata [%s]", checkpoints.get(0));
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java
index d11709ecc62..2b540e53710 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java
@@ -25,7 +25,7 @@
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
-import com.metamx.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.guice.annotations.Json;
 import io.druid.indexing.kafka.KafkaIndexTaskClientFactory;
 import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java
index 8d328aadbad..2834cc838a8 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java
@@ -27,10 +27,10 @@
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.FullResponseHandler;
-import com.metamx.http.client.response.FullResponseHolder;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.FullResponseHandler;
+import io.druid.java.util.http.client.response.FullResponseHolder;
 import io.druid.indexing.common.TaskInfoProvider;
 import io.druid.indexer.TaskLocation;
 import io.druid.indexing.common.TaskStatus;
diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
index 298d1490f88..45f3003638f 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
@@ -38,10 +38,10 @@
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.core.NoopEmitter;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.metrics.MonitorScheduler;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.core.NoopEmitter;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.metrics.MonitorScheduler;
 import io.druid.client.cache.CacheConfig;
 import io.druid.client.cache.MapCache;
 import io.druid.data.input.impl.DimensionsSpec;
@@ -155,10 +155,12 @@
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Executors;
@@ -241,6 +243,7 @@ public KafkaIndexTaskTest(boolean isIncrementalHandoffSupported)
         new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2010", "c", "y", 1.0f)),
         new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2011", "d", "y", 1.0f)),
         new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2011", "e", "y", 1.0f)),
+        new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("246140482-04-24T15:36:27.903Z", "x", "z", 1.0f)),
         new ProducerRecord<byte[], byte[]>(topic, 0, null, StringUtils.toUtf8("unparseable")),
         new ProducerRecord<byte[], byte[]>(topic, 0, null, null),
         new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2013", "f", "y", 1.0f)),
@@ -456,7 +459,7 @@ public void testIncrementalHandOff() throws Exception
     // of events fetched across two partitions from Kafka
     final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 0L));
     final KafkaPartitions checkpoint2 = new KafkaPartitions(topic, ImmutableMap.of(0, 4L, 1, 2L));
-    final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 8L, 1, 2L));
+    final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 9L, 1, 2L));
     final KafkaIndexTask task = createTask(
         null,
         new KafkaIOConfig(
@@ -494,7 +497,7 @@ public void testIncrementalHandOff() throws Exception
     // Check metrics
     Assert.assertEquals(8, task.getFireDepartmentMetrics().processed());
     Assert.assertEquals(1, task.getFireDepartmentMetrics().unparseable());
-    Assert.assertEquals(1, task.getFireDepartmentMetrics().thrownAway());
+    Assert.assertEquals(2, task.getFireDepartmentMetrics().thrownAway());
 
     // Check published metadata
     SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
@@ -506,7 +509,7 @@ public void testIncrementalHandOff() throws Exception
     SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 8L, 1, 2L))),
+        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 9L, 1, 2L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -970,7 +973,7 @@ public void testRunConflicting() throws Exception
         new KafkaIOConfig(
             "sequence1",
             new KafkaPartitions(topic, ImmutableMap.of(0, 3L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 8L)),
+            new KafkaPartitions(topic, ImmutableMap.of(0, 9L)),
             kafkaServer.consumerProperties(),
             true,
             false,
@@ -1001,7 +1004,7 @@ public void testRunConflicting() throws Exception
     Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway());
     Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed());
     Assert.assertEquals(1, task2.getFireDepartmentMetrics().unparseable());
-    Assert.assertEquals(1, task2.getFireDepartmentMetrics().thrownAway());
+    Assert.assertEquals(2, task2.getFireDepartmentMetrics().thrownAway());
 
     // Check published segments & metadata, should all be from the first task
     SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
@@ -1039,7 +1042,7 @@ public void testRunConflictingWithoutTransactions() throws Exception
         new KafkaIOConfig(
             "sequence1",
             new KafkaPartitions(topic, ImmutableMap.of(0, 3L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 8L)),
+            new KafkaPartitions(topic, ImmutableMap.of(0, 9L)),
             kafkaServer.consumerProperties(),
             false,
             false,
@@ -1076,7 +1079,7 @@ public void testRunConflictingWithoutTransactions() throws Exception
     Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway());
     Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed());
     Assert.assertEquals(1, task2.getFireDepartmentMetrics().unparseable());
-    Assert.assertEquals(1, task2.getFireDepartmentMetrics().thrownAway());
+    Assert.assertEquals(2, task2.getFireDepartmentMetrics().thrownAway());
 
     // Check published segments & metadata
     SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1);
@@ -1449,7 +1452,7 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception
     Assert.assertEquals(KafkaIndexTask.Status.PAUSED, task.getStatus());
 
     // try again but with resume flag == true
-    newEndOffsets = ImmutableMap.of(0, 6L);
+    newEndOffsets = ImmutableMap.of(0, 7L);
     task.setEndOffsets(newEndOffsets, true, true);
     Assert.assertEquals(newEndOffsets, task.getEndOffsets());
     Assert.assertNotEquals(KafkaIndexTask.Status.PAUSED, task.getStatus());
@@ -1468,7 +1471,7 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception
     // Check metrics
     Assert.assertEquals(4, task.getFireDepartmentMetrics().processed());
     Assert.assertEquals(1, task.getFireDepartmentMetrics().unparseable());
-    Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway());
+    Assert.assertEquals(1, task.getFireDepartmentMetrics().thrownAway());
 
     // Check published metadata
     SegmentDescriptor desc1 = SD(task, "2009/P1D", 0);
@@ -1476,7 +1479,7 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception
     SegmentDescriptor desc3 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 6L))),
+        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 7L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1556,6 +1559,72 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva
     }
   }
 
+  @Test(timeout = 60_000L)
+  public void testRunContextSequenceAheadOfStartingOffsets() throws Exception
+  {
+    // This tests the case when a replacement task is created in place of a failed test
+    // which has done some incremental handoffs, thus the context will contain starting
+    // sequence offsets from which the task should start reading and ignore the start offsets
+    if (!isIncrementalHandoffSupported) {
+      return;
+    }
+    // Insert data
+    try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
+      for (ProducerRecord<byte[], byte[]> record : records) {
+        kafkaProducer.send(record).get();
+      }
+    }
+
+    final TreeMap<Integer, Map<Integer, Long>> sequences = new TreeMap<>();
+    // Here the sequence number is 1 meaning that one incremental handoff was done by the failed task
+    // and this task should start reading from offset 2 for partition 0
+    sequences.put(1, ImmutableMap.of(0, 2L));
+    final Map<String, Object> context = new HashMap<>();
+    context.put("checkpoints", objectMapper.writerWithType(new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
+    {
+    }).writeValueAsString(sequences));
+
+    final KafkaIndexTask task = createTask(
+        null,
+        new KafkaIOConfig(
+            "sequence0",
+            // task should ignore these and use sequence info sent in the context
+            new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
+            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            kafkaServer.consumerProperties(),
+            true,
+            false,
+            null,
+            null,
+            false
+        ),
+        context
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    // Check metrics
+    Assert.assertEquals(3, task.getFireDepartmentMetrics().processed());
+    Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable());
+    Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertEquals(
+        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+  }
+
   private ListenableFuture<TaskStatus> runTask(final Task task)
   {
     try {
@@ -1614,6 +1683,15 @@ private KafkaIndexTask createTask(
     return createTask(taskId, DATA_SCHEMA, ioConfig);
   }
 
+  private KafkaIndexTask createTask(
+      final String taskId,
+      final KafkaIOConfig ioConfig,
+      final Map<String, Object> context
+  )
+  {
+    return createTask(taskId, DATA_SCHEMA, ioConfig, context);
+  }
+
   private KafkaIndexTask createTask(
       final String taskId,
       final DataSchema dataSchema,
@@ -1650,6 +1728,45 @@ private KafkaIndexTask createTask(
     return task;
   }
 
+
+  private KafkaIndexTask createTask(
+      final String taskId,
+      final DataSchema dataSchema,
+      final KafkaIOConfig ioConfig,
+      final Map<String, Object> context
+  )
+  {
+    final KafkaTuningConfig tuningConfig = new KafkaTuningConfig(
+        1000,
+        maxRowsPerSegment,
+        new Period("P1Y"),
+        null,
+        null,
+        null,
+        true,
+        reportParseExceptions,
+        handoffConditionTimeout,
+        resetOffsetAutomatically,
+        null
+    );
+    if (isIncrementalHandoffSupported) {
+      context.put(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true);
+    }
+
+    final KafkaIndexTask task = new KafkaIndexTask(
+        taskId,
+        null,
+        cloneDataSchema(dataSchema),
+        tuningConfig,
+        ioConfig,
+        context,
+        null,
+        null
+    );
+    task.setPollRetryMs(POLL_RETRY_MS);
+    return task;
+  }
+
   private static DataSchema cloneDataSchema(final DataSchema dataSchema)
   {
     return new DataSchema(
@@ -1884,9 +2001,9 @@ private File getSegmentDirectory()
 
   private List<String> readSegmentColumn(final String column, final SegmentDescriptor descriptor) throws IOException
   {
-    File indexZip = new File(
+    File indexBasePath = new File(
         StringUtils.format(
-            "%s/%s/%s_%s/%s/%d/index.zip",
+            "%s/%s/%s_%s/%s/%d",
             getSegmentDirectory(),
             DATA_SCHEMA.getDataSource(),
             descriptor.getInterval().getStart(),
@@ -1895,6 +2012,7 @@ private File getSegmentDirectory()
             descriptor.getPartitionNumber()
         )
     );
+
     File outputLocation = new File(
         directory,
         StringUtils.format(
@@ -1907,7 +2025,7 @@ private File getSegmentDirectory()
     );
     outputLocation.mkdir();
     CompressionUtils.unzip(
-        Files.asByteSource(indexZip),
+        Files.asByteSource(new File(indexBasePath.listFiles()[0], "index.zip")),
         outputLocation,
         Predicates.<Throwable>alwaysFalse(),
         false
diff --git a/extensions-core/lookups-cached-global/pom.xml b/extensions-core/lookups-cached-global/pom.xml
index 4ce26c7a803..fe8908564e7 100644
--- a/extensions-core/lookups-cached-global/pom.xml
+++ b/extensions-core/lookups-cached-global/pom.xml
@@ -18,8 +18,7 @@
   ~ 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";>
+<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>
   <groupId>io.druid.extensions</groupId>
   <artifactId>druid-lookups-cached-global</artifactId>
@@ -29,7 +28,7 @@
   <parent>
     <groupId>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java
index 38fcce978aa..1f1d3e3f813 100644
--- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java
+++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java
@@ -22,8 +22,9 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.concurrent.ConcurrentAwaitableCounter;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.guice.LazySingleton;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.StringUtils;
@@ -51,11 +52,11 @@
  * // cacheState could be either NoCache or VersionedCache.
  * if (cacheState instanceof NoCache) {
  *   // the cache is not yet created, or already closed
- * } else if (cacheState instanceof VersionedCache) {
+ * } else {
  *   Map<String, String> cache = ((VersionedCache) cacheState).getCache(); // use the cache
  *   // Although VersionedCache implements AutoCloseable, versionedCache shouldn't be manually closed
  *   // when obtained from entry.getCacheState(). If the namespace updates should be ceased completely,
- *   // entry.close() (see below) should be called, it will close the last VersionedCache itself.
+ *   // entry.close() (see below) should be called, it will close the last VersionedCache as well.
  *   // On scheduled updates, outdated VersionedCaches are also closed automatically.
  * }
  * ...
@@ -105,14 +106,16 @@ public CacheState getCacheState()
       return impl.updaterFuture;
     }
 
+    @VisibleForTesting
     public void awaitTotalUpdates(int totalUpdates) throws InterruptedException
     {
-      impl.updateCounter.awaitTotalUpdates(totalUpdates);
+      impl.updateCounter.awaitCount(totalUpdates);
     }
 
+    @VisibleForTesting
     void awaitNextUpdates(int nextUpdates) throws InterruptedException
     {
-      impl.updateCounter.awaitNextUpdates(nextUpdates);
+      impl.updateCounter.awaitNextIncrements(nextUpdates);
     }
 
     /**
@@ -145,7 +148,7 @@ public String toString()
     private final Future<?> updaterFuture;
     private final Cleaner entryCleaner;
     private final CacheGenerator<T> cacheGenerator;
-    private final UpdateCounter updateCounter = new UpdateCounter();
+    private final ConcurrentAwaitableCounter updateCounter = new ConcurrentAwaitableCounter();
     private final CountDownLatch startLatch = new CountDownLatch(1);
 
     private EntryImpl(final T namespace, final Entry<T> entry, final CacheGenerator<T> cacheGenerator)
@@ -276,7 +279,7 @@ private CacheState swapCacheState(VersionedCache newVersionedCache)
           return lastCacheState;
         }
       } while (!cacheStateHolder.compareAndSet(lastCacheState, newVersionedCache));
-      updateCounter.update();
+      updateCounter.increment();
       return lastCacheState;
     }
 
@@ -485,7 +488,7 @@ public Entry scheduleAndWait(ExtractionNamespace namespace, long waitForFirstRun
     log.debug("Scheduled new %s", entry);
     boolean success = false;
     try {
-      success = entry.impl.updateCounter.awaitFirstUpdate(waitForFirstRunMs, TimeUnit.MILLISECONDS);
+      success = entry.impl.updateCounter.awaitFirstIncrement(waitForFirstRunMs, TimeUnit.MILLISECONDS);
       if (success) {
         return entry;
       } else {
diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java
index 65f646d5af6..49047294c9c 100644
--- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java
+++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManager.java
@@ -22,7 +22,7 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.metamx.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.java.util.common.concurrent.ExecutorServices;
 import io.druid.java.util.common.lifecycle.Lifecycle;
 import io.druid.java.util.common.logger.Logger;
diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java
index e591f8da90d..ab069001e39 100644
--- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java
+++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OffHeapNamespaceExtractionCacheManager.java
@@ -21,8 +21,8 @@
 
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.java.util.common.lifecycle.Lifecycle;
 import io.druid.java.util.common.logger.Logger;
 import io.druid.server.lookup.namespace.NamespaceExtractionConfig;
diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java
index 6a5e015f252..d538c8f2da8 100644
--- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java
+++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java
@@ -21,8 +21,8 @@
 
 import com.google.common.primitives.Chars;
 import com.google.inject.Inject;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.lifecycle.Lifecycle;
 import io.druid.java.util.common.logger.Logger;
diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/UpdateCounter.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/UpdateCounter.java
deleted file mode 100644
index ed229b45f89..00000000000
--- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/UpdateCounter.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to Metamarkets Group Inc. (Metamarkets) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. Metamarkets 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 io.druid.server.lookup.namespace.cache;
-
-import java.util.concurrent.Phaser;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-final class UpdateCounter
-{
-  /**
-   * Max {@link Phaser}'s phase, specified in it's javadoc. Then it wraps to zero.
-   */
-  private static final int MAX_PHASE = Integer.MAX_VALUE;
-
-  private final Phaser phaser = new Phaser(1);
-
-  void update()
-  {
-    phaser.arrive();
-  }
-
-  void awaitTotalUpdates(int totalUpdates) throws InterruptedException
-  {
-    totalUpdates &= MAX_PHASE;
-    int currentUpdates = phaser.getPhase();
-    checkNotTerminated(currentUpdates);
-    while (comparePhases(totalUpdates, currentUpdates) > 0) {
-      currentUpdates = phaser.awaitAdvanceInterruptibly(currentUpdates);
-      checkNotTerminated(currentUpdates);
-    }
-  }
-
-  private static int comparePhases(int phase1, int phase2)
-  {
-    int diff = (phase1 - phase2) & MAX_PHASE;
-    if (diff == 0) {
-      return 0;
-    }
-    return diff < MAX_PHASE / 2 ? 1 : -1;
-  }
-
-  private void checkNotTerminated(int phase)
-  {
-    if (phase < 0) {
-      throw new IllegalStateException("Phaser[" + phaser + "] unexpectedly terminated.");
-    }
-  }
-
-  void awaitNextUpdates(int nextUpdates) throws InterruptedException
-  {
-    if (nextUpdates <= 0) {
-      throw new IllegalArgumentException("nextUpdates is not positive: " + nextUpdates);
-    }
-    if (nextUpdates > MAX_PHASE / 4) {
-      throw new UnsupportedOperationException("Couldn't wait for so many updates: " + nextUpdates);
-    }
-    awaitTotalUpdates(phaser.getPhase() + nextUpdates);
-  }
-
-  boolean awaitFirstUpdate(long timeout, TimeUnit unit) throws InterruptedException
-  {
-    try {
-      phaser.awaitAdvanceInterruptibly(0, timeout, unit);
-      return true;
-    }
-    catch (TimeoutException e) {
-      return false;
-    }
-  }
-}
diff --git a/extensions-core/lookups-cached-single/pom.xml b/extensions-core/lookups-cached-single/pom.xml
index 9b4ea5a5ea3..a2c335d53fb 100644
--- a/extensions-core/lookups-cached-single/pom.xml
+++ b/extensions-core/lookups-cached-single/pom.xml
@@ -18,8 +18,7 @@
   ~ 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";>
+<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>
   <groupId>io.druid.extensions</groupId>
   <artifactId>druid-lookups-cached-single</artifactId>
@@ -29,7 +28,7 @@
   <parent>
     <groupId>io.druid</groupId>
     <artifactId>druid</artifactId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/extensions-core/mysql-metadata-storage/pom.xml b/extensions-core/mysql-metadata-storage/pom.xml
index 950a985ce8f..1187cf0d606 100644
--- a/extensions-core/mysql-metadata-storage/pom.xml
+++ b/extensions-core/mysql-metadata-storage/pom.xml
@@ -28,7 +28,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLConnector.java b/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLConnector.java
index 3079ad427ea..910f397f447 100644
--- a/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLConnector.java
+++ b/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLConnector.java
@@ -19,6 +19,7 @@
 
 package io.druid.metadata.storage.mysql;
 
+import com.google.common.base.Joiner;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 import com.google.inject.Inject;
@@ -36,6 +37,7 @@
 import org.skife.jdbi.v2.tweak.HandleCallback;
 import org.skife.jdbi.v2.util.BooleanMapper;
 
+import java.io.File;
 import java.sql.SQLException;
 
 public class MySQLConnector extends SQLMetadataConnector
@@ -48,7 +50,11 @@
   private final DBI dbi;
 
   @Inject
-  public MySQLConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables)
+  public MySQLConnector(
+      Supplier<MetadataStorageConnectorConfig> config,
+      Supplier<MetadataStorageTablesConfig> dbTables,
+      MySQLConnectorConfig connectorConfig
+  )
   {
     super(config, dbTables);
 
@@ -57,6 +63,68 @@ public MySQLConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<
     // so we need to help JDBC find the driver
     datasource.setDriverClassLoader(getClass().getClassLoader());
     datasource.setDriverClassName("com.mysql.jdbc.Driver");
+    datasource.addConnectionProperty("useSSL", String.valueOf(connectorConfig.isUseSSL()));
+    if (connectorConfig.isUseSSL()) {
+      log.info("SSL is enabled on this MySQL connection. ");
+
+      datasource.addConnectionProperty(
+          "verifyServerCertificate",
+          String.valueOf(connectorConfig.isVerifyServerCertificate())
+      );
+      if (connectorConfig.isVerifyServerCertificate()) {
+        log.info("Server certificate verification is enabled. ");
+
+        if (connectorConfig.getTrustCertificateKeyStoreUrl() != null) {
+          datasource.addConnectionProperty(
+              "trustCertificateKeyStoreUrl",
+              new File(connectorConfig.getTrustCertificateKeyStoreUrl()).toURI().toString()
+          );
+        }
+        if (connectorConfig.getTrustCertificateKeyStoreType() != null) {
+          datasource.addConnectionProperty(
+              "trustCertificateKeyStoreType",
+              connectorConfig.getTrustCertificateKeyStoreType()
+          );
+        }
+        if (connectorConfig.getTrustCertificateKeyStorePassword() == null) {
+          log.warn(
+              "Trust store password is empty. Ensure that the trust store has been configured with an empty password.");
+        } else {
+          datasource.addConnectionProperty(
+              "trustCertificateKeyStorePassword",
+              connectorConfig.getTrustCertificateKeyStorePassword()
+          );
+        }
+      }
+      if (connectorConfig.getClientCertificateKeyStoreUrl() != null) {
+        datasource.addConnectionProperty(
+            "clientCertificateKeyStoreUrl",
+            new File(connectorConfig.getClientCertificateKeyStoreUrl()).toURI().toString()
+        );
+      }
+      if (connectorConfig.getClientCertificateKeyStoreType() != null) {
+        datasource.addConnectionProperty(
+            "clientCertificateKeyStoreType",
+            connectorConfig.getClientCertificateKeyStoreType()
+        );
+      }
+      if (connectorConfig.getClientCertificateKeyStorePassword() != null) {
+        datasource.addConnectionProperty(
+            "clientCertificateKeyStorePassword",
+            connectorConfig.getClientCertificateKeyStorePassword()
+        );
+      }
+      Joiner joiner = Joiner.on(",").skipNulls();
+      if (connectorConfig.getEnabledSSLCipherSuites() != null) {
+        datasource.addConnectionProperty(
+            "enabledSSLCipherSuites",
+            joiner.join(connectorConfig.getEnabledSSLCipherSuites())
+        );
+      }
+      if (connectorConfig.getEnabledTLSProtocols() != null) {
+        datasource.addConnectionProperty("enabledTLSProtocols", joiner.join(connectorConfig.getEnabledTLSProtocols()));
+      }
+    }
 
     // use double-quotes for quoting columns, so we can write SQL that works with most databases
     datasource.setConnectionInitSqls(ImmutableList.of("SET sql_mode='ANSI_QUOTES'"));
@@ -97,9 +165,9 @@ public boolean tableExists(Handle handle, String tableName)
   {
     // ensure database defaults to utf8, otherwise bail
     boolean isUtf8 = handle
-                         .createQuery("SELECT @@character_set_database = 'utf8'")
-                         .map(BooleanMapper.FIRST)
-                         .first();
+        .createQuery("SELECT @@character_set_database = 'utf8'")
+        .map(BooleanMapper.FIRST)
+        .first();
 
     if (!isUtf8) {
       throw new ISE(
diff --git a/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLConnectorConfig.java b/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLConnectorConfig.java
new file mode 100644
index 00000000000..77fc9dcd60b
--- /dev/null
+++ b/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLConnectorConfig.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.metadata.storage.mysql;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.druid.metadata.PasswordProvider;
+
+import java.util.List;
+
+public class MySQLConnectorConfig
+{
+  @JsonProperty
+  private boolean useSSL = false;
+
+  @JsonProperty
+  private String trustCertificateKeyStoreUrl;
+
+  @JsonProperty
+  private String trustCertificateKeyStoreType;
+
+  @JsonProperty("trustCertificateKeyStorePassword")
+  private PasswordProvider trustCertificateKeyStorePasswordProvider;
+
+  @JsonProperty
+  private String clientCertificateKeyStoreUrl;
+
+  @JsonProperty
+  private String clientCertificateKeyStoreType;
+
+  @JsonProperty("clientCertificateKeyStorePassword")
+  private PasswordProvider clientCertificateKeyStorePasswordProvider;
+
+  @JsonProperty
+  private List<String> enabledSSLCipherSuites;
+
+  @JsonProperty
+  private List<String> enabledTLSProtocols;
+
+  @JsonProperty
+  private boolean verifyServerCertificate = false;
+
+  public boolean isUseSSL()
+  {
+    return useSSL;
+  }
+
+  public String getTrustCertificateKeyStoreUrl()
+  {
+    return trustCertificateKeyStoreUrl;
+  }
+
+  public String getTrustCertificateKeyStoreType()
+  {
+    return trustCertificateKeyStoreType;
+  }
+
+  public String getTrustCertificateKeyStorePassword()
+  {
+    return trustCertificateKeyStorePasswordProvider == null ? null : trustCertificateKeyStorePasswordProvider.getPassword();
+  }
+
+  public String getClientCertificateKeyStoreUrl()
+  {
+    return clientCertificateKeyStoreUrl;
+  }
+
+  public String getClientCertificateKeyStoreType()
+  {
+    return clientCertificateKeyStoreType;
+  }
+
+  public String getClientCertificateKeyStorePassword()
+  {
+    return clientCertificateKeyStorePasswordProvider == null ? null : clientCertificateKeyStorePasswordProvider.getPassword();
+  }
+
+  public List<String> getEnabledSSLCipherSuites()
+  {
+    return enabledSSLCipherSuites;
+  }
+
+  public List<String> getEnabledTLSProtocols()
+  {
+    return enabledTLSProtocols;
+  }
+
+  public boolean isVerifyServerCertificate()
+  {
+    return verifyServerCertificate;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "MySQLConnectorConfig{" +
+           "useSSL='" + useSSL + '\'' +
+           ", clientCertificateKeyStoreUrl='" + clientCertificateKeyStoreUrl + '\'' +
+           ", clientCertificateKeyStoreType='" + clientCertificateKeyStoreType + '\'' +
+           ", verifyServerCertificate='" + verifyServerCertificate + '\'' +
+           ", trustCertificateKeyStoreUrl='" + trustCertificateKeyStoreUrl + '\'' +
+           ", trustCertificateKeyStoreType='" + trustCertificateKeyStoreType + '\'' +
+           ", enabledSSLCipherSuites=" + enabledSSLCipherSuites +
+           ", enabledTLSProtocols=" + enabledTLSProtocols +
+           '}';
+  }
+}
diff --git a/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java b/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java
index 211d5ca0b0e..4ba1cb8177b 100644
--- a/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java
+++ b/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java
@@ -23,6 +23,7 @@
 import com.google.common.collect.ImmutableList;
 import com.google.inject.Binder;
 import com.google.inject.Key;
+import io.druid.guice.JsonConfigProvider;
 import io.druid.guice.LazySingleton;
 import io.druid.guice.PolyBind;
 import io.druid.guice.SQLMetadataStorageDruidModule;
@@ -56,6 +57,8 @@ public void configure(Binder binder)
   {
     super.configure(binder);
 
+    JsonConfigProvider.bind(binder, "druid.metadata.mysql.ssl", MySQLConnectorConfig.class);
+
     PolyBind
         .optionBinder(binder, Key.get(MetadataStorageProvider.class))
         .addBinding(TYPE)
diff --git a/extensions-core/postgresql-metadata-storage/pom.xml b/extensions-core/postgresql-metadata-storage/pom.xml
index d4625a6499c..f62f0c58237 100644
--- a/extensions-core/postgresql-metadata-storage/pom.xml
+++ b/extensions-core/postgresql-metadata-storage/pom.xml
@@ -28,7 +28,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml
index e1a14d412b6..0d9c8da4c19 100644
--- a/extensions-core/protobuf-extensions/pom.xml
+++ b/extensions-core/protobuf-extensions/pom.xml
@@ -16,9 +16,7 @@
   ~ 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";>
+<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>
 
   <groupId>io.druid.extensions</groupId>
@@ -29,7 +27,7 @@
   <parent>
     <artifactId>druid</artifactId>
     <groupId>io.druid</groupId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/extensions-core/s3-extensions/pom.xml b/extensions-core/s3-extensions/pom.xml
index c08208c81e1..8e16183f57a 100644
--- a/extensions-core/s3-extensions/pom.xml
+++ b/extensions-core/s3-extensions/pom.xml
@@ -16,8 +16,7 @@
   ~ 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";>
+<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>
 
     <groupId>io.druid.extensions</groupId>
@@ -28,7 +27,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
@@ -45,15 +44,15 @@
             <version>${project.parent.version}</version>
             <scope>provided</scope>
         </dependency>
-
         <dependency>
-            <groupId>net.java.dev.jets3t</groupId>
-            <artifactId>jets3t</artifactId>
+            <groupId>io.druid</groupId>
+            <artifactId>java-util</artifactId>
+            <version>${project.parent.version}</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
-            <groupId>com.metamx</groupId>
-            <artifactId>java-util</artifactId>
+            <groupId>net.java.dev.jets3t</groupId>
+            <artifactId>jets3t</artifactId>
             <scope>provided</scope>
         </dependency>
         <dependency>
diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentFinder.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentFinder.java
index d6d773640e8..3f4529ec99a 100644
--- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentFinder.java
+++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentFinder.java
@@ -21,9 +21,8 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Throwables;
-import com.google.common.collect.Sets;
 import com.google.inject.Inject;
-
+import io.druid.java.util.common.Pair;
 import io.druid.java.util.common.logger.Logger;
 import io.druid.segment.loading.DataSegmentFinder;
 import io.druid.segment.loading.SegmentLoadingException;
@@ -35,9 +34,11 @@
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 public class S3DataSegmentFinder implements DataSegmentFinder
 {
@@ -62,7 +63,7 @@ public S3DataSegmentFinder(
   @Override
   public Set<DataSegment> findSegments(String workingDirPath, boolean updateDescriptor) throws SegmentLoadingException
   {
-    final Set<DataSegment> segments = Sets.newHashSet();
+    final Map<String, Pair<DataSegment, Long>> timestampedSegments = new HashMap<>();
 
     try {
       Iterator<StorageObject> objectsIterator = S3Utils.storageObjectsIterator(
@@ -103,7 +104,12 @@ public S3DataSegmentFinder(
                   s3Client.putObject(config.getBucket(), newDescJsonObject);
                 }
               }
-              segments.add(dataSegment);
+
+              DataSegmentFinder.putInMapRetainingNewest(
+                  timestampedSegments,
+                  dataSegment,
+                  indexObject.getLastModifiedDate() == null ? 0 : indexObject.getLastModifiedDate().getTime()
+              );
             }
           } else {
             throw new SegmentLoadingException(
@@ -124,6 +130,6 @@ public S3DataSegmentFinder(
       Throwables.propagateIfInstanceOf(e, SegmentLoadingException.class);
       Throwables.propagate(e);
     }
-    return segments;
+    return timestampedSegments.values().stream().map(x -> x.lhs).collect(Collectors.toSet());
   }
 }
diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java
index 983837865e8..643b30407d9 100644
--- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java
+++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java
@@ -72,8 +72,11 @@ public DataSegment move(DataSegment segment, Map<String, Object> targetLoadSpec)
       final String targetS3Bucket = MapUtils.getString(targetLoadSpec, "bucket");
       final String targetS3BaseKey = MapUtils.getString(targetLoadSpec, "baseKey");
 
-      final String targetS3Path = S3Utils.constructSegmentPath(targetS3BaseKey, DataSegmentPusher.getDefaultStorageDir(segment));
-      String targetS3DescriptorPath = S3Utils.descriptorPathForSegmentPath(targetS3Path);
+      final String targetS3Path = S3Utils.constructSegmentPath(
+          targetS3BaseKey,
+          DataSegmentPusher.getDefaultStorageDir(segment, false)
+      );
+      final String targetS3DescriptorPath = S3Utils.descriptorPathForSegmentPath(targetS3Path);
 
       if (targetS3Bucket.isEmpty()) {
         throw new SegmentLoadingException("Target S3 bucket is not specified");
diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java
index 1100287cb96..eb5c4a9c10d 100644
--- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java
+++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java
@@ -24,9 +24,9 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
 import io.druid.java.util.common.CompressionUtils;
 import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.segment.SegmentUtils;
 import io.druid.segment.loading.DataSegmentPusher;
 import io.druid.timeline.DataSegment;
@@ -88,9 +88,10 @@ public String getPathForHadoop(String dataSource)
   }
 
   @Override
-  public DataSegment push(final File indexFilesDir, final DataSegment inSegment) throws IOException
+  public DataSegment push(final File indexFilesDir, final DataSegment inSegment, final boolean useUniquePath)
+      throws IOException
   {
-    final String s3Path = S3Utils.constructSegmentPath(config.getBaseKey(), getStorageDir(inSegment));
+    final String s3Path = S3Utils.constructSegmentPath(config.getBaseKey(), getStorageDir(inSegment, useUniquePath));
 
     log.info("Copying segment[%s] to S3 at location[%s]", inSegment.getIdentifier(), s3Path);
 
@@ -105,21 +106,10 @@ public DataSegment push(final File indexFilesDir, final DataSegment inSegment) t
             public DataSegment call() throws Exception
             {
               S3Object toPush = new S3Object(zipOutFile);
-
-              final String outputBucket = config.getBucket();
-              final String s3DescriptorPath = S3Utils.descriptorPathForSegmentPath(s3Path);
-
-              toPush.setBucketName(outputBucket);
-              toPush.setKey(s3Path);
-              if (!config.getDisableAcl()) {
-                toPush.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL);
-              }
-
-              log.info("Pushing %s.", toPush);
-              s3Client.putObject(outputBucket, toPush);
+              putObject(config.getBucket(), s3Path, toPush);
 
               final DataSegment outSegment = inSegment.withSize(indexSize)
-                                                      .withLoadSpec(makeLoadSpec(outputBucket, toPush.getKey()))
+                                                      .withLoadSpec(makeLoadSpec(config.getBucket(), toPush.getKey()))
                                                       .withBinaryVersion(SegmentUtils.getVersionFromDir(indexFilesDir));
 
               File descriptorFile = File.createTempFile("druid", "descriptor.json");
@@ -127,14 +117,12 @@ public DataSegment call() throws Exception
               // runtime, and because Guava deletes methods over time, that causes incompatibilities.
               Files.write(descriptorFile.toPath(), jsonMapper.writeValueAsBytes(outSegment));
               S3Object descriptorObject = new S3Object(descriptorFile);
-              descriptorObject.setBucketName(outputBucket);
-              descriptorObject.setKey(s3DescriptorPath);
-              if (!config.getDisableAcl()) {
-                descriptorObject.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL);
-              }
 
-              log.info("Pushing %s", descriptorObject);
-              s3Client.putObject(outputBucket, descriptorObject);
+              putObject(
+                  config.getBucket(),
+                  S3Utils.descriptorPathForSegmentPath(s3Path),
+                  descriptorObject
+              );
 
               log.info("Deleting zipped index File[%s]", zipOutFile);
               zipOutFile.delete();
@@ -164,7 +152,6 @@ public DataSegment call() throws Exception
 
   /**
    * Any change in loadSpec need to be reflected {@link io.druid.indexer.JobHelper#getURIFromSegment()}
-   *
    */
   @SuppressWarnings("JavadocReference")
   private Map<String, Object> makeLoadSpec(String bucket, String key)
@@ -180,4 +167,17 @@ public DataSegment call() throws Exception
         config.isUseS3aSchema() ? "s3a" : "s3n"
     );
   }
+
+  private void putObject(String bucketName, String path, S3Object object) throws ServiceException
+  {
+    object.setBucketName(bucketName);
+    object.setKey(path);
+    if (!config.getDisableAcl()) {
+      object.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL);
+    }
+
+    log.info("Pushing %s.", object);
+
+    s3Client.putObject(bucketName, object);
+  }
 }
diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java
index f0df427ccb2..55ad817003d 100644
--- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java
+++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java
@@ -124,7 +124,6 @@
   private String indexZip4_1;
 
 
-
   @BeforeClass
   public static void setUpStatic()
   {
@@ -210,31 +209,51 @@ public void testFindSegments() throws Exception
     final String serializedSegment4_0 = mapper.writeValueAsString(updatedSegment4_0);
     final String serializedSegment4_1 = mapper.writeValueAsString(updatedSegment4_1);
 
-    Assert.assertNotEquals(serializedSegment1,
-                           IOUtils.toString(mockS3Client.getObject(bucket, descriptor1).getDataInputStream()));
-    Assert.assertNotEquals(serializedSegment2,
-                           IOUtils.toString(mockS3Client.getObject(bucket, descriptor2).getDataInputStream()));
-    Assert.assertNotEquals(serializedSegment3,
-                           IOUtils.toString(mockS3Client.getObject(bucket, descriptor3).getDataInputStream()));
-    Assert.assertNotEquals(serializedSegment4_0,
-                           IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_0).getDataInputStream()));
-    Assert.assertNotEquals(serializedSegment4_1,
-                           IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_1).getDataInputStream()));
+    Assert.assertNotEquals(
+        serializedSegment1,
+        IOUtils.toString(mockS3Client.getObject(bucket, descriptor1).getDataInputStream())
+    );
+    Assert.assertNotEquals(
+        serializedSegment2,
+        IOUtils.toString(mockS3Client.getObject(bucket, descriptor2).getDataInputStream())
+    );
+    Assert.assertNotEquals(
+        serializedSegment3,
+        IOUtils.toString(mockS3Client.getObject(bucket, descriptor3).getDataInputStream())
+    );
+    Assert.assertNotEquals(
+        serializedSegment4_0,
+        IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_0).getDataInputStream())
+    );
+    Assert.assertNotEquals(
+        serializedSegment4_1,
+        IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_1).getDataInputStream())
+    );
 
     final Set<DataSegment> segments2 = s3DataSegmentFinder.findSegments("", true);
 
     Assert.assertEquals(segments, segments2);
 
-    Assert.assertEquals(serializedSegment1,
-                           IOUtils.toString(mockS3Client.getObject(bucket, descriptor1).getDataInputStream()));
-    Assert.assertEquals(serializedSegment2,
-                           IOUtils.toString(mockS3Client.getObject(bucket, descriptor2).getDataInputStream()));
-    Assert.assertEquals(serializedSegment3,
-                           IOUtils.toString(mockS3Client.getObject(bucket, descriptor3).getDataInputStream()));
-    Assert.assertEquals(serializedSegment4_0,
-                           IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_0).getDataInputStream()));
-    Assert.assertEquals(serializedSegment4_1,
-                           IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_1).getDataInputStream()));
+    Assert.assertEquals(
+        serializedSegment1,
+        IOUtils.toString(mockS3Client.getObject(bucket, descriptor1).getDataInputStream())
+    );
+    Assert.assertEquals(
+        serializedSegment2,
+        IOUtils.toString(mockS3Client.getObject(bucket, descriptor2).getDataInputStream())
+    );
+    Assert.assertEquals(
+        serializedSegment3,
+        IOUtils.toString(mockS3Client.getObject(bucket, descriptor3).getDataInputStream())
+    );
+    Assert.assertEquals(
+        serializedSegment4_0,
+        IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_0).getDataInputStream())
+    );
+    Assert.assertEquals(
+        serializedSegment4_1,
+        IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_1).getDataInputStream())
+    );
   }
 
   @Test(expected = SegmentLoadingException.class)
@@ -268,9 +287,7 @@ public void testFindSegmentsWithworkingDirPath() throws SegmentLoadingException
   public void testFindSegmentsUpdateLoadSpec() throws Exception
   {
     config.setBucket("amazing");
-    final DataSegment segmentMissingLoadSpec = DataSegment.builder(SEGMENT_1)
-        .loadSpec(ImmutableMap.of())
-        .build();
+    final DataSegment segmentMissingLoadSpec = DataSegment.builder(SEGMENT_1).loadSpec(ImmutableMap.of()).build();
     final S3DataSegmentFinder s3DataSegmentFinder = new S3DataSegmentFinder(mockS3Client, config, mapper);
     final String segmentPath = baseKey + "/interval_missing_load_spec/v1/1/";
     final String descriptorPath = S3Utils.descriptorPathForSegmentPath(segmentPath);
@@ -304,6 +321,32 @@ public void testFindSegmentsUpdateLoadSpec() throws Exception
     Assert.assertEquals(indexPath, testLoadSpec.get("key"));
   }
 
+  @Test
+  public void testPreferNewestSegment() throws Exception
+  {
+    baseKey = "replicaDataSource";
+
+    config = new S3DataSegmentPusherConfig();
+    config.setBucket(bucket);
+    config.setBaseKey(baseKey);
+
+    descriptor1 = S3Utils.descriptorPathForSegmentPath(baseKey + "/interval10/v1/0/older/");
+    descriptor2 = S3Utils.descriptorPathForSegmentPath(baseKey + "/interval10/v1/0/newer/");
+
+    indexZip1 = S3Utils.indexZipForSegmentPath(descriptor1);
+    indexZip2 = S3Utils.indexZipForSegmentPath(descriptor2);
+
+    mockS3Client.putObject(bucket, new S3Object(descriptor1, mapper.writeValueAsString(SEGMENT_1)));
+    mockS3Client.putObject(bucket, new S3Object(indexZip1, "dummy"));
+    mockS3Client.putObject(bucket, new S3Object(descriptor2, mapper.writeValueAsString(SEGMENT_1)));
+    mockS3Client.putObject(bucket, new S3Object(indexZip2, "dummy"));
+
+    final S3DataSegmentFinder s3DataSegmentFinder = new S3DataSegmentFinder(mockS3Client, config, mapper);
+    final Set<DataSegment> segments = s3DataSegmentFinder.findSegments("", false);
+
+    Assert.assertEquals(1, segments.size());
+  }
+
   private String getDescriptorPath(DataSegment segment)
   {
     return S3Utils.descriptorPathForSegmentPath(String.valueOf(segment.getLoadSpec().get("key")));
diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java
index 32818b17e48..841f10e1c66 100644
--- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java
+++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java
@@ -64,6 +64,20 @@ public void setValue(T value)
 
   @Test
   public void testPush() throws Exception
+  {
+    testPushInternal(false, "key/foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/index\\.zip");
+  }
+
+  @Test
+  public void testPushUseUniquePath() throws Exception
+  {
+    testPushInternal(
+        true,
+        "key/foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/[A-Za-z0-9-]{36}/index\\.zip"
+    );
+  }
+
+  private void testPushInternal(boolean useUniquePath, String matcher) throws Exception
   {
     RestS3Service s3Client = EasyMock.createStrictMock(RestS3Service.class);
 
@@ -113,14 +127,15 @@ public S3Object answer() throws Throwable
         size
     );
 
-    DataSegment segment = pusher.push(tempFolder.getRoot(), segmentToPush);
+    DataSegment segment = pusher.push(tempFolder.getRoot(), segmentToPush, useUniquePath);
 
     Assert.assertEquals(segmentToPush.getSize(), segment.getSize());
     Assert.assertEquals(1, (int) segment.getBinaryVersion());
     Assert.assertEquals("bucket", segment.getLoadSpec().get("bucket"));
-    Assert.assertEquals(
-        "key/foo/2015-01-01T00:00:00.000Z_2016-01-01T00:00:00.000Z/0/0/index.zip",
-        segment.getLoadSpec().get("key"));
+    Assert.assertTrue(
+        segment.getLoadSpec().get("key").toString(),
+        segment.getLoadSpec().get("key").toString().matches(matcher)
+    );
     Assert.assertEquals("s3_zip", segment.getLoadSpec().get("type"));
 
     // Verify that the pushed S3Object contains the correct data
diff --git a/extensions-core/simple-client-sslcontext/pom.xml b/extensions-core/simple-client-sslcontext/pom.xml
index 9277db3a552..f03946338e6 100644
--- a/extensions-core/simple-client-sslcontext/pom.xml
+++ b/extensions-core/simple-client-sslcontext/pom.xml
@@ -1,11 +1,9 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<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";>
+<project xmlns="http://maven.apache.org/POM/4.0.0"; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd";>
   <parent>
     <artifactId>druid</artifactId>
     <groupId>io.druid</groupId>
-    <version>0.12.0-SNAPSHOT</version>
+    <version>0.12.1-rc2</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
@@ -35,8 +33,9 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>com.metamx</groupId>
+      <groupId>io.druid</groupId>
       <artifactId>java-util</artifactId>
+      <version>${project.parent.version}</version>
       <scope>provided</scope>
     </dependency>
   </dependencies>
diff --git a/extensions-core/simple-client-sslcontext/src/main/java/io/druid/https/SSLContextProvider.java b/extensions-core/simple-client-sslcontext/src/main/java/io/druid/https/SSLContextProvider.java
index ccdacf1b733..ee53c9278e0 100644
--- a/extensions-core/simple-client-sslcontext/src/main/java/io/druid/https/SSLContextProvider.java
+++ b/extensions-core/simple-client-sslcontext/src/main/java/io/druid/https/SSLContextProvider.java
@@ -22,7 +22,7 @@
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
 import com.google.inject.Provider;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.TrustManagerFactory;
diff --git a/extensions-core/stats/pom.xml b/extensions-core/stats/pom.xml
index e6dd75e687d..098ba088914 100644
--- a/extensions-core/stats/pom.xml
+++ b/extensions-core/stats/pom.xml
@@ -29,7 +29,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
diff --git a/hll/pom.xml b/hll/pom.xml
index 653b2ca63b3..f2952b1a304 100644
--- a/hll/pom.xml
+++ b/hll/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
     </parent>
 
     <artifactId>druid-hll</artifactId>
diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml
index 9ab3867d3bf..aafe8d19166 100644
--- a/indexing-hadoop/pom.xml
+++ b/indexing-hadoop/pom.xml
@@ -26,7 +26,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
     </parent>
 
     <dependencies>
diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java
index c9111d00cd3..a9726bb2db1 100644
--- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java
+++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java
@@ -64,6 +64,7 @@
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.InvalidJobConfException;
 import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.Partitioner;
@@ -210,9 +211,17 @@ public boolean run()
 
       boolean success = job.waitForCompletion(true);
 
-      Counter invalidRowCount = job.getCounters()
-                                   .findCounter(HadoopDruidIndexerConfig.IndexJobCounters.INVALID_ROW_COUNTER);
-      jobStats.setInvalidRowCount(invalidRowCount.getValue());
+      Counters counters = job.getCounters();
+      if (counters == null) {
+        log.info("No counters found for job [%s]", job.getJobName());
+      } else {
+        Counter invalidRowCount = counters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.INVALID_ROW_COUNTER);
+        if (invalidRowCount != null) {
+          jobStats.setInvalidRowCount(invalidRowCount.getValue());
+        } else {
+          log.info("No invalid row counter found for job [%s]", job.getJobName());
+        }
+      }
 
       return success;
     }
@@ -258,6 +267,7 @@ private static IncrementalIndex makeIncrementalIndex(
 
     private AggregatorFactory[] aggregators;
     private AggregatorFactory[] combiningAggs;
+    private Map<String, InputRowSerde.IndexSerdeTypeHelper> typeHelperMap;
 
     @Override
     protected void setup(Context context)
@@ -269,6 +279,11 @@ protected void setup(Context context)
       for (int i = 0; i < aggregators.length; ++i) {
         combiningAggs[i] = aggregators[i].getCombiningFactory();
       }
+      typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema()
+                                                           .getDataSchema()
+                                                           .getParser()
+                                                           .getParseSpec()
+                                                           .getDimensionsSpec());
     }
 
     @Override
@@ -299,9 +314,9 @@ protected void innerMap(
       // and they contain the columns as they show up in the segment after ingestion, not what you would see in raw
       // data
       byte[] serializedInputRow = inputRow instanceof SegmentInputRow ?
-                                  InputRowSerde.toBytes(inputRow, combiningAggs, reportParseExceptions)
+                                  InputRowSerde.toBytes(typeHelperMap, inputRow, combiningAggs, reportParseExceptions)
                                                                       :
-                                  InputRowSerde.toBytes(inputRow, aggregators, reportParseExceptions);
+                                  InputRowSerde.toBytes(typeHelperMap, inputRow, aggregators, reportParseExceptions);
 
       context.write(
           new SortableBytes(
@@ -322,6 +337,7 @@ protected void innerMap(
     private HadoopDruidIndexerConfig config;
     private AggregatorFactory[] aggregators;
     private AggregatorFactory[] combiningAggs;
+    private Map<String, InputRowSerde.IndexSerdeTypeHelper> typeHelperMap;
 
     @Override
     protected void setup(Context context)
@@ -334,6 +350,11 @@ protected void setup(Context context)
       for (int i = 0; i < aggregators.length; ++i) {
         combiningAggs[i] = aggregators[i].getCombiningFactory();
       }
+      typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema()
+                                                           .getDataSchema()
+                                                           .getParser()
+                                                           .getParseSpec()
+                                                           .getDimensionsSpec());
     }
 
     @Override
@@ -350,11 +371,11 @@ protected void reduce(
         SortableBytes keyBytes = SortableBytes.fromBytesWritable(key);
         Bucket bucket = Bucket.fromGroupKey(keyBytes.getGroupKey()).lhs;
         IncrementalIndex index = makeIncrementalIndex(bucket, combiningAggs, config, null, null);
-        index.add(InputRowSerde.fromBytes(first.getBytes(), aggregators));
+        index.add(InputRowSerde.fromBytes(typeHelperMap, first.getBytes(), aggregators));
 
         while (iter.hasNext()) {
           context.progress();
-          InputRow value = InputRowSerde.fromBytes(iter.next().getBytes(), aggregators);
+          InputRow value = InputRowSerde.fromBytes(typeHelperMap, iter.next().getBytes(), aggregators);
 
           if (!index.canAppendRow()) {
             dimOrder.addAll(index.getDimensionOrder());
@@ -381,10 +402,13 @@ private void flushIndexToContextAndClose(BytesWritable key, IncrementalIndex ind
         context.progress();
         Row row = rows.next();
         InputRow inputRow = getInputRowFromRow(row, dimensions);
+
         // reportParseExceptions is true as any unparseable data is already handled by the mapper.
+        byte[] serializedRow = InputRowSerde.toBytes(typeHelperMap, inputRow, combiningAggs, true);
+
         context.write(
             key,
-            new BytesWritable(InputRowSerde.toBytes(inputRow, combiningAggs, true))
+            new BytesWritable(serializedRow)
         );
       }
       index.close();
@@ -479,6 +503,7 @@ public void setConf(Configuration config)
     private List<String> metricNames = Lists.newArrayList();
     private AggregatorFactory[] aggregators;
     private AggregatorFactory[] combiningAggs;
+    private Map<String, InputRowSerde.IndexSerdeTypeHelper> typeHelperMap;
 
     protected ProgressIndicator makeProgressIndicator(final Context context)
     {
@@ -530,6 +555,11 @@ protected void setup(Context context)
         metricNames.add(aggregators[i].getName());
         combiningAggs[i] = aggregators[i].getCombiningFactory();
       }
+      typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema()
+                                                           .getDataSchema()
+                                                           .getParser()
+                                                           .getParseSpec()
+                                                           .getDimensionsSpec());
     }
 
     @Override
@@ -597,7 +627,7 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor)
         for (final BytesWritable bw : values) {
           context.progress();
 
-          final InputRow inputRow = index.formatRow(InputRowSerde.fromBytes(bw.getBytes(), aggregators));
+          final InputRow inputRow = index.formatRow(InputRowSerde.fromBytes(typeHelperMap, bw.getBytes(), aggregators));
           int numRows = index.add(inputRow);
 
           ++lineCount;
diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java
index 672f7713374..cd1dd531604 100644
--- a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java
+++ b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java
@@ -22,17 +22,24 @@
 import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import com.google.common.io.ByteArrayDataInput;
 import com.google.common.io.ByteArrayDataOutput;
 import com.google.common.io.ByteStreams;
+
 import io.druid.data.input.InputRow;
 import io.druid.data.input.MapBasedInputRow;
+import io.druid.data.input.Rows;
+import io.druid.data.input.impl.DimensionSchema;
+import io.druid.data.input.impl.DimensionsSpec;
 import io.druid.java.util.common.IAE;
 import io.druid.java.util.common.StringUtils;
 import io.druid.java.util.common.logger.Logger;
 import io.druid.java.util.common.parsers.ParseException;
 import io.druid.query.aggregation.Aggregator;
 import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.segment.DimensionHandlerUtils;
 import io.druid.segment.VirtualColumns;
+import io.druid.segment.column.ValueType;
 import io.druid.segment.incremental.IncrementalIndex;
 import io.druid.segment.serde.ComplexMetricSerde;
 import io.druid.segment.serde.ComplexMetrics;
@@ -49,7 +56,165 @@
 {
   private static final Logger log = new Logger(InputRowSerde.class);
 
-  public static final byte[] toBytes(final InputRow row, AggregatorFactory[] aggs, boolean reportParseExceptions)
+  private static final IndexSerdeTypeHelper STRING_HELPER = new StringIndexSerdeTypeHelper();
+  private static final IndexSerdeTypeHelper LONG_HELPER = new LongIndexSerdeTypeHelper();
+  private static final IndexSerdeTypeHelper FLOAT_HELPER = new FloatIndexSerdeTypeHelper();
+  private static final IndexSerdeTypeHelper DOUBLE_HELPER = new DoubleIndexSerdeTypeHelper();
+
+  public interface IndexSerdeTypeHelper<T>
+  {
+    ValueType getType();
+
+    void serialize(ByteArrayDataOutput out, Object value, boolean reportParseExceptions);
+
+    T deserialize(ByteArrayDataInput in);
+  }
+
+  public static Map<String, IndexSerdeTypeHelper> getTypeHelperMap(DimensionsSpec dimensionsSpec)
+  {
+    Map<String, IndexSerdeTypeHelper> typeHelperMap = Maps.newHashMap();
+    for (DimensionSchema dimensionSchema : dimensionsSpec.getDimensions()) {
+      IndexSerdeTypeHelper typeHelper;
+      switch (dimensionSchema.getValueType()) {
+        case STRING:
+          typeHelper = STRING_HELPER;
+          break;
+        case LONG:
+          typeHelper = LONG_HELPER;
+          break;
+        case FLOAT:
+          typeHelper = FLOAT_HELPER;
+          break;
+        case DOUBLE:
+          typeHelper = DOUBLE_HELPER;
+          break;
+        default:
+          throw new IAE("Invalid type: [%s]", dimensionSchema.getValueType());
+      }
+      typeHelperMap.put(dimensionSchema.getName(), typeHelper);
+    }
+    return typeHelperMap;
+  }
+
+  public static class StringIndexSerdeTypeHelper implements IndexSerdeTypeHelper<List<String>>
+  {
+    @Override
+    public ValueType getType()
+    {
+      return ValueType.STRING;
+    }
+
+    @Override
+    public void serialize(ByteArrayDataOutput out, Object value, boolean reportParseExceptions)
+    {
+      List<String> values = Rows.objectToStrings(value);
+      try {
+        writeStringArray(values, out);
+      }
+      catch (IOException ioe) {
+        throw new RuntimeException(ioe);
+      }
+    }
+
+    @Override
+    public List<String> deserialize(ByteArrayDataInput in)
+    {
+      try {
+        return readStringArray(in);
+      }
+      catch (IOException ioe) {
+        throw new RuntimeException(ioe);
+      }
+    }
+  }
+
+  public static class LongIndexSerdeTypeHelper implements IndexSerdeTypeHelper<Long>
+  {
+    @Override
+    public ValueType getType()
+    {
+      return ValueType.LONG;
+    }
+
+    @Override
+    public void serialize(ByteArrayDataOutput out, Object value, boolean reportParseExceptions)
+    {
+      Long ret = DimensionHandlerUtils.convertObjectToLong(value, reportParseExceptions);
+      if (ret == null) {
+        // remove null -> zero conversion when https://github.com/druid-io/druid/pull/5278 series of patches is merged
+        // we'll also need to change the serialized encoding so that it can represent numeric nulls
+        ret = DimensionHandlerUtils.ZERO_LONG;
+      }
+      out.writeLong(ret);
+    }
+
+    @Override
+    public Long deserialize(ByteArrayDataInput in)
+    {
+      return in.readLong();
+    }
+  }
+
+  public static class FloatIndexSerdeTypeHelper implements IndexSerdeTypeHelper<Float>
+  {
+    @Override
+    public ValueType getType()
+    {
+      return ValueType.FLOAT;
+    }
+
+    @Override
+    public void serialize(ByteArrayDataOutput out, Object value, boolean reportParseExceptions)
+    {
+      Float ret = DimensionHandlerUtils.convertObjectToFloat(value, reportParseExceptions);
+      if (ret == null) {
+        // remove null -> zero conversion when https://github.com/druid-io/druid/pull/5278 series of patches is merged
+        // we'll also need to change the serialized encoding so that it can represent numeric nulls
+        ret = DimensionHandlerUtils.ZERO_FLOAT;
+      }
+      out.writeFloat(ret);
+    }
+
+    @Override
+    public Float deserialize(ByteArrayDataInput in)
+    {
+      return in.readFloat();
+    }
+  }
+
+  public static class DoubleIndexSerdeTypeHelper implements IndexSerdeTypeHelper<Double>
+  {
+    @Override
+    public ValueType getType()
+    {
+      return ValueType.DOUBLE;
+    }
+
+    @Override
+    public void serialize(ByteArrayDataOutput out, Object value, boolean reportParseExceptions)
+    {
+      Double ret = DimensionHandlerUtils.convertObjectToDouble(value, reportParseExceptions);
+      if (ret == null) {
+        // remove null -> zero conversion when https://github.com/druid-io/druid/pull/5278 series of patches is merged
+        // we'll also need to change the serialized encoding so that it can represent numeric nulls
+        ret = DimensionHandlerUtils.ZERO_DOUBLE;
+      }
+      out.writeDouble(ret);
+    }
+
+    @Override
+    public Double deserialize(ByteArrayDataInput in)
+    {
+      return in.readDouble();
+    }
+  }
+
+  public static final byte[] toBytes(
+      final Map<String, IndexSerdeTypeHelper> typeHelperMap,
+      final InputRow row,
+      AggregatorFactory[] aggs,
+      boolean reportParseExceptions
+  )
   {
     try {
       ByteArrayDataOutput out = ByteStreams.newDataOutput();
@@ -63,9 +228,12 @@
       WritableUtils.writeVInt(out, dimList.size());
       if (dimList != null) {
         for (String dim : dimList) {
-          List<String> dimValues = row.getDimension(dim);
+          IndexSerdeTypeHelper typeHelper = typeHelperMap.get(dim);
+          if (typeHelper == null) {
+            typeHelper = STRING_HELPER;
+          }
           writeString(dim, out);
-          writeStringArray(dimValues, out);
+          typeHelper.serialize(out, row.getRaw(dim), reportParseExceptions);
         }
       }
 
@@ -176,10 +344,14 @@ private static String readString(DataInput in) throws IOException
     return values;
   }
 
-  public static final InputRow fromBytes(byte[] data, AggregatorFactory[] aggs)
+  public static final InputRow fromBytes(
+      final Map<String, IndexSerdeTypeHelper> typeHelperMap,
+      byte[] data,
+      AggregatorFactory[] aggs
+  )
   {
     try {
-      DataInput in = ByteStreams.newDataInput(data);
+      ByteArrayDataInput in = ByteStreams.newDataInput(data);
 
       //Read timestamp
       long timestamp = in.readLong();
@@ -192,14 +364,25 @@ public static final InputRow fromBytes(byte[] data, AggregatorFactory[] aggs)
       for (int i = 0; i < dimNum; i++) {
         String dimension = readString(in);
         dimensions.add(dimension);
-        List<String> dimensionValues = readStringArray(in);
-        if (dimensionValues == null) {
+
+        IndexSerdeTypeHelper typeHelper = typeHelperMap.get(dimension);
+        if (typeHelper == null) {
+          typeHelper = STRING_HELPER;
+        }
+        Object dimValues = typeHelper.deserialize(in);
+        if (dimValues == null) {
           continue;
         }
-        if (dimensionValues.size() == 1) {
-          event.put(dimension, dimensionValues.get(0));
+
+        if (typeHelper.getType() == ValueType.STRING) {
+          List<String> dimensionValues = (List<String>) dimValues;
+          if (dimensionValues.size() == 1) {
+            event.put(dimension, dimensionValues.get(0));
+          } else {
+            event.put(dimension, dimensionValues);
+          }
         } else {
-          event.put(dimension, dimensionValues);
+          event.put(dimension, dimValues);
         }
       }
 
diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java
index 7ce678045c7..8975c01d86a 100644
--- a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java
+++ b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java
@@ -91,6 +91,7 @@ public static Path distributedClassPath(Path base)
   {
     return new Path(base, "classpath");
   }
+
   public static final String INDEX_ZIP = "index.zip";
   public static final String DESCRIPTOR_JSON = "descriptor.json";
 
@@ -568,8 +569,10 @@ public static Path makeFileNamePath(
       DataSegmentPusher dataSegmentPusher
   )
   {
-    return new Path(prependFSIfNullScheme(fs, basePath),
-                    dataSegmentPusher.makeIndexPathName(segmentTemplate, baseFileName));
+    return new Path(
+        prependFSIfNullScheme(fs, basePath),
+        dataSegmentPusher.makeIndexPathName(segmentTemplate, baseFileName)
+    );
   }
 
   public static Path makeTmpPath(
@@ -582,9 +585,10 @@ public static Path makeTmpPath(
   {
     return new Path(
         prependFSIfNullScheme(fs, basePath),
-        StringUtils.format("./%s.%d",
-                           dataSegmentPusher.makeIndexPathName(segmentTemplate, JobHelper.INDEX_ZIP),
-                           taskAttemptID.getId()
+        StringUtils.format(
+            "./%s.%d",
+            dataSegmentPusher.makeIndexPathName(segmentTemplate, JobHelper.INDEX_ZIP),
+            taskAttemptID.getId()
         )
     );
   }
diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java
index b4856a925fc..a93459f6163 100644
--- a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java
+++ b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java
@@ -29,7 +29,6 @@
 import io.druid.indexer.JobHelper;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.logger.Logger;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -65,7 +64,7 @@
   @Override
   public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException
   {
-    Configuration conf = context.getConfiguration();
+    JobConf conf = new JobConf(context.getConfiguration());
 
     String segmentsStr = Preconditions.checkNotNull(
         conf.get(CONF_INPUT_SEGMENTS),
@@ -89,7 +88,7 @@
       for (WindowedDataSegment segment : segments) {
         totalSize += segment.getSegment().getSize();
       }
-      int mapTask = ((JobConf) conf).getNumMapTasks();
+      int mapTask = conf.getNumMapTasks();
       if (mapTask > 0) {
         maxSize = totalSize / mapTask;
       }
@@ -116,11 +115,10 @@ public int compare(WindowedDataSegment s1, WindowedDataSegment s2)
     List<WindowedDataSegment> list = new ArrayList<>();
     long size = 0;
 
-    JobConf dummyConf = new JobConf();
     org.apache.hadoop.mapred.InputFormat fio = supplier.get();
     for (WindowedDataSegment segment : segments) {
       if (size + segment.getSegment().getSize() > maxSize && size > 0) {
-        splits.add(toDataSourceSplit(list, fio, dummyConf));
+        splits.add(toDataSourceSplit(list, fio, conf));
         list = Lists.newArrayList();
         size = 0;
       }
@@ -130,7 +128,7 @@ public int compare(WindowedDataSegment s1, WindowedDataSegment s2)
     }
 
     if (list.size() > 0) {
-      splits.add(toDataSourceSplit(list, fio, dummyConf));
+      splits.add(toDataSourceSplit(list, fio, conf));
     }
 
     logger.info("Number of splits [%d]", splits.size());
@@ -214,14 +212,14 @@ private DatasourceInputSplit toDataSourceSplit(
                   try {
                     return Arrays.stream(split.getLocations());
                   }
-                  catch (final IOException e) {
+                  catch (final Exception e) {
                     logger.error(e, "Exception getting locations");
                     return Stream.empty();
                   }
                 }
             );
           }
-          catch (final IOException e) {
+          catch (final Exception e) {
             logger.error(e, "Exception getting splits");
             return Stream.empty();
           }
diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java
index 9eaf75349ed..9eb75e27b77 100644
--- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java
+++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java
@@ -25,6 +25,7 @@
 import io.druid.data.input.InputRow;
 import io.druid.data.input.MapBasedInputRow;
 import io.druid.data.input.impl.DimensionsSpec;
+import io.druid.data.input.impl.StringDimensionSchema;
 import io.druid.data.input.impl.StringInputRowParser;
 import io.druid.data.input.impl.TimeAndDimsParseSpec;
 import io.druid.data.input.impl.TimestampSpec;
@@ -144,6 +145,17 @@ public void testMultipleRowsMerged() throws Exception
     );
     BytesWritable key = keySortableBytes.toBytesWritable();
 
+    DimensionsSpec dimensionsSpec = new DimensionsSpec(
+        Arrays.asList(
+            new StringDimensionSchema("host"),
+            new StringDimensionSchema("keywords")
+        ),
+        null,
+        null
+    );
+
+    Map<String, InputRowSerde.IndexSerdeTypeHelper> typeHelperMap = InputRowSerde.getTypeHelperMap(dimensionsSpec);
+
     InputRow row1 = new MapBasedInputRow(
         timestamp,
         ImmutableList.<String>of("keywords"),
@@ -163,8 +175,8 @@ public void testMultipleRowsMerged() throws Exception
         )
     );
     List<BytesWritable> rows = Lists.newArrayList(
-        new BytesWritable(InputRowSerde.toBytes(row1, aggregators, true)),
-        new BytesWritable(InputRowSerde.toBytes(row2, aggregators, true))
+        new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row1, aggregators, true)),
+        new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row2, aggregators, true))
     );
 
     Reducer.Context context = EasyMock.createNiceMock(Reducer.Context.class);
@@ -183,7 +195,7 @@ public void testMultipleRowsMerged() throws Exception
 
     Assert.assertTrue(captureKey.getValue() == key);
 
-    InputRow capturedRow = InputRowSerde.fromBytes(captureVal.getValue().getBytes(), aggregators);
+    InputRow capturedRow = InputRowSerde.fromBytes(typeHelperMap, captureVal.getValue().getBytes(), aggregators);
     Assert.assertEquals(Arrays.asList("host", "keywords"), capturedRow.getDimensions());
     Assert.assertEquals(ImmutableList.of(), capturedRow.getDimension("host"));
     Assert.assertEquals(Arrays.asList("bar", "foo"), capturedRow.getDimension("keywords"));
@@ -228,9 +240,21 @@ public void testMultipleRowsNotMerged() throws Exception
             "visited", 5
         )
     );
+
+    DimensionsSpec dimensionsSpec = new DimensionsSpec(
+        Arrays.asList(
+            new StringDimensionSchema("host"),
+            new StringDimensionSchema("keywords")
+        ),
+        null,
+        null
+    );
+
+    Map<String, InputRowSerde.IndexSerdeTypeHelper> typeHelperMap = InputRowSerde.getTypeHelperMap(dimensionsSpec);
+
     List<BytesWritable> rows = Lists.newArrayList(
-        new BytesWritable(InputRowSerde.toBytes(row1, aggregators, true)),
-        new BytesWritable(InputRowSerde.toBytes(row2, aggregators, true))
+        new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row1, aggregators, true)),
+        new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row2, aggregators, true))
     );
 
     Reducer.Context context = EasyMock.createNiceMock(Reducer.Context.class);
@@ -253,7 +277,7 @@ public void testMultipleRowsNotMerged() throws Exception
     Assert.assertTrue(captureKey1.getValue() == key);
     Assert.assertTrue(captureKey2.getValue() == key);
 
-    InputRow capturedRow1 = InputRowSerde.fromBytes(captureVal1.getValue().getBytes(), aggregators);
+    InputRow capturedRow1 = InputRowSerde.fromBytes(typeHelperMap, captureVal1.getValue().getBytes(), aggregators);
     Assert.assertEquals(Arrays.asList("host", "keywords"), capturedRow1.getDimensions());
     Assert.assertEquals(Collections.singletonList("host1"), capturedRow1.getDimension("host"));
     Assert.assertEquals(Arrays.asList("bar", "foo"), capturedRow1.getDimension("keywords"));
@@ -264,7 +288,7 @@ public void testMultipleRowsNotMerged() throws Exception
         0.001
     );
 
-    InputRow capturedRow2 = InputRowSerde.fromBytes(captureVal2.getValue().getBytes(), aggregators);
+    InputRow capturedRow2 = InputRowSerde.fromBytes(typeHelperMap, captureVal2.getValue().getBytes(), aggregators);
     Assert.assertEquals(Arrays.asList("host", "keywords"), capturedRow2.getDimensions());
     Assert.assertEquals(Collections.singletonList("host2"), capturedRow2.getDimension("host"));
     Assert.assertEquals(Arrays.asList("bar", "foo"), capturedRow2.getDimension("keywords"));
diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java
index 67a9b5abf8a..71609e42dd3 100644
--- a/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java
+++ b/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java
@@ -20,9 +20,14 @@
 package io.druid.indexer;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
 import io.druid.data.input.InputRow;
 import io.druid.data.input.MapBasedInputRow;
+import io.druid.data.input.impl.DimensionsSpec;
+import io.druid.data.input.impl.DoubleDimensionSchema;
+import io.druid.data.input.impl.FloatDimensionSchema;
+import io.druid.data.input.impl.LongDimensionSchema;
+import io.druid.data.input.impl.StringDimensionSchema;
 import io.druid.hll.HyperLogLogCollector;
 import io.druid.jackson.AggregatorsModule;
 import io.druid.java.util.common.parsers.ParseException;
@@ -35,8 +40,11 @@
 import io.druid.segment.ColumnSelectorFactory;
 import org.easymock.EasyMock;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -53,17 +61,22 @@
     new AggregatorsModule(); //registers ComplexMetric serde for hyperUnique
   }
 
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
   public InputRowSerdeTest()
   {
     this.timestamp = System.currentTimeMillis();
-    this.dims = ImmutableList.of("dim_non_existing", "d1", "d2");
-    this.event = ImmutableMap.<String, Object>of(
-        "d1", "d1v",
-        "d2", ImmutableList.of("d2v1", "d2v2"),
-        "m1", 5.0f,
-        "m2", 100L,
-        "m3", "m3v"
-    );
+    this.dims = ImmutableList.of("dim_non_existing", "d1", "d2", "d3", "d4", "d5");
+    this.event = Maps.newHashMap();
+    event.put("d1", "d1v");
+    event.put("d2", ImmutableList.of("d2v1", "d2v2"));
+    event.put("d3", 200L);
+    event.put("d4", 300.1f);
+    event.put("d5", 400.5d);
+    event.put("m1", 5.0f);
+    event.put("m2", 100L);
+    event.put("m3", "m3v");
   }
 
   @Test
@@ -99,14 +112,29 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory)
         }
     };
 
-    byte[] data = InputRowSerde.toBytes(in, aggregatorFactories, false); // Ignore Unparseable aggregator
-    InputRow out = InputRowSerde.fromBytes(data, aggregatorFactories);
+    DimensionsSpec dimensionsSpec = new DimensionsSpec(
+        Arrays.asList(
+            new StringDimensionSchema("d1"),
+            new StringDimensionSchema("d2"),
+            new LongDimensionSchema("d3"),
+            new FloatDimensionSchema("d4"),
+            new DoubleDimensionSchema("d5")
+        ),
+        null,
+        null
+    );
+
+    byte[] data = InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories, false); // Ignore Unparseable aggregator
+    InputRow out = InputRowSerde.fromBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), data, aggregatorFactories);
 
     Assert.assertEquals(timestamp, out.getTimestampFromEpoch());
     Assert.assertEquals(dims, out.getDimensions());
     Assert.assertEquals(Collections.EMPTY_LIST, out.getDimension("dim_non_existing"));
     Assert.assertEquals(ImmutableList.of("d1v"), out.getDimension("d1"));
     Assert.assertEquals(ImmutableList.of("d2v1", "d2v2"), out.getDimension("d2"));
+    Assert.assertEquals(200L, out.getRaw("d3"));
+    Assert.assertEquals(300.1f, out.getRaw("d4"));
+    Assert.assertEquals(400.5d, out.getRaw("d5"));
 
     Assert.assertEquals(0.0f, out.getMetric("agg_non_existing").floatValue(), 0.00001);
     Assert.assertEquals(5.0f, out.getMetric("m1out").floatValue(), 0.00001);
@@ -117,7 +145,7 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory)
     EasyMock.verify(mockedAggregator);
   }
 
-  @Test(expected = ParseException.class)
+  @Test
   public void testThrowParseExceptions()
   {
     InputRow in = new MapBasedInputRow(
@@ -133,7 +161,66 @@ public void testThrowParseExceptions()
         new LongSumAggregatorFactory("unparseable", "m3") // Unparseable from String to Long
     };
 
-    InputRowSerde.toBytes(in, aggregatorFactories, true);
+    DimensionsSpec dimensionsSpec = new DimensionsSpec(
+        Arrays.asList(
+            new StringDimensionSchema("d1"),
+            new StringDimensionSchema("d2"),
+            new LongDimensionSchema("d3"),
+            new FloatDimensionSchema("d4"),
+            new DoubleDimensionSchema("d5")
+        ),
+        null,
+        null
+    );
 
+    expectedException.expect(ParseException.class);
+    expectedException.expectMessage("Encountered parse error for aggregator[unparseable]");
+    InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories, true);
+  }
+
+  @Test
+  public void testDimensionParseExceptions()
+  {
+    InputRow in = new MapBasedInputRow(
+        timestamp,
+        dims,
+        event
+    );
+    AggregatorFactory[] aggregatorFactories = new AggregatorFactory[]{
+        new LongSumAggregatorFactory("m2out", "m2")
+    };
+
+    expectedException.expect(ParseException.class);
+    expectedException.expectMessage("could not convert value [d1v] to long");
+    DimensionsSpec dimensionsSpec = new DimensionsSpec(
+        Arrays.asList(
+            new LongDimensionSchema("d1")
+        ),
+        null,
+        null
+    );
+    InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories, true);
+
+    expectedException.expect(ParseException.class);
+    expectedException.expectMessage("could not convert value [d1v] to float");
+    dimensionsSpec = new DimensionsSpec(
+        Arrays.asList(
+            new FloatDimensionSchema("d1")
+        ),
+        null,
+        null
+    );
+    InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories, true);
+
+    expectedException.expect(ParseException.class);
+    expectedException.expectMessage("could not convert value [d1v] to double");
+    dimensionsSpec = new DimensionsSpec(
+        Arrays.asList(
+            new DoubleDimensionSchema("d1")
+        ),
+        null,
+        null
+    );
+    InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories, true);
   }
 }
diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml
index eedb98c9487..a058fcf2f75 100644
--- a/indexing-service/pom.xml
+++ b/indexing-service/pom.xml
@@ -26,7 +26,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
     </parent>
 
     <dependencies>
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java
index 3db6e0f7b90..dd132769192 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java
@@ -27,8 +27,8 @@
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Multimaps;
 import com.google.inject.Provider;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.metrics.MonitorScheduler;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.metrics.MonitorScheduler;
 import io.druid.client.cache.Cache;
 import io.druid.client.cache.CacheConfig;
 import io.druid.discovery.DataNodeService;
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java
index 43d2abea594..c17b23fe210 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java
@@ -23,8 +23,8 @@
 import com.google.common.base.Preconditions;
 import com.google.inject.Inject;
 import com.google.inject.Provider;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.metrics.MonitorScheduler;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.metrics.MonitorScheduler;
 import io.druid.client.cache.Cache;
 import io.druid.client.cache.CacheConfig;
 import io.druid.discovery.DataNodeService;
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java
index 7bce2b6e89b..0e688ecbfd8 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java
@@ -19,7 +19,7 @@
 
 package io.druid.indexing.common.actions;
 
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.indexing.common.task.Task;
 import io.druid.indexing.overlord.TaskStorage;
 import io.druid.java.util.common.ISE;
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java
index b2e3c5f23c3..9c1c7e82399 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java
@@ -21,7 +21,7 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Throwables;
-import com.metamx.http.client.response.FullResponseHolder;
+import io.druid.java.util.http.client.response.FullResponseHolder;
 import io.druid.discovery.DruidLeaderClient;
 import io.druid.indexing.common.RetryPolicy;
 import io.druid.indexing.common.RetryPolicyFactory;
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
index 2e33d5e15c0..7c55939f4d5 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
@@ -24,7 +24,7 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.google.common.collect.ImmutableSet;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.indexing.common.task.Task;
 import io.druid.indexing.overlord.CriticalAction;
 import io.druid.java.util.common.ISE;
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java
index 044cfebd00f..dad3ca51646 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java
@@ -24,7 +24,7 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.google.common.collect.ImmutableSet;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.indexing.common.task.Task;
 import io.druid.indexing.overlord.CriticalAction;
 import io.druid.java.util.common.ISE;
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java
index c22275def06..acd6eb5a647 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java
@@ -23,12 +23,12 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.google.common.collect.ImmutableSet;
-import com.metamx.emitter.service.ServiceMetricEvent;
 import io.druid.indexing.common.task.Task;
 import io.druid.indexing.overlord.CriticalAction;
 import io.druid.indexing.overlord.DataSourceMetadata;
 import io.druid.indexing.overlord.SegmentPublishResult;
 import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.query.DruidMetrics;
 import io.druid.timeline.DataSegment;
 
@@ -127,22 +127,21 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throws
       throw new RuntimeException(e);
     }
 
+    // Emit metrics
+    final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
+        .setDimension(DruidMetrics.DATASOURCE, task.getDataSource())
+        .setDimension(DruidMetrics.TASK_TYPE, task.getType());
+
     if (retVal.isSuccess()) {
-      // Emit metrics
-      final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
-          .setDimension(DruidMetrics.DATASOURCE, task.getDataSource())
-          .setDimension(DruidMetrics.TASK_TYPE, task.getType());
-
-      if (retVal.isSuccess()) {
-        toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1));
-      } else {
-        toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1));
-      }
-
-      for (DataSegment segment : retVal.getSegments()) {
-        metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString());
-        toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize()));
-      }
+      toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1));
+    } else {
+      toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1));
+    }
+
+    // getSegments() should return an empty set if announceHistoricalSegments() failed
+    for (DataSegment segment : retVal.getSegments()) {
+      metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString());
+      toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize()));
     }
 
     return retVal;
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java
index 59ca2294215..4d08465b99e 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java
@@ -20,7 +20,7 @@
 package io.druid.indexing.common.actions;
 
 import com.google.inject.Inject;
-import com.metamx.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
 import io.druid.indexing.overlord.TaskLockbox;
 import io.druid.indexing.overlord.supervisor.SupervisorManager;
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java
index b6e2a26d88f..4364f50f40a 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java
@@ -199,7 +199,7 @@ public void finishJob()
                                                      .withDimensions(ImmutableList.copyOf(mappedSegment.getAvailableDimensions()))
                                                      .withBinaryVersion(SegmentUtils.getVersionFromDir(fileToUpload));
 
-          dataSegmentPusher.push(fileToUpload, segmentToUpload);
+          dataSegmentPusher.push(fileToUpload, segmentToUpload, false);
 
           log.info(
               "Uploaded segment[%s]",
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java
index 96f3889982b..5e94285cd9f 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java
@@ -51,6 +51,7 @@
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.JodaUtils;
 import io.druid.java.util.common.Pair;
+import io.druid.java.util.common.RE;
 import io.druid.java.util.common.granularity.NoneGranularity;
 import io.druid.java.util.common.guava.Comparators;
 import io.druid.java.util.common.jackson.JacksonUtils;
@@ -78,6 +79,7 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -189,26 +191,35 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception
           jsonMapper
       );
 
-      indexTaskSpec = new IndexTask(
-          getId(),
-          getGroupId(),
-          getTaskResource(),
-          getDataSource(),
-          ingestionSpec,
-          getContext()
-      );
-    }
-
-    if (indexTaskSpec.getIngestionSchema() == null) {
-      log.info("Cannot find segments for interval");
+      if (ingestionSpec != null) {
+        indexTaskSpec = new IndexTask(
+            getId(),
+            getGroupId(),
+            getTaskResource(),
+            getDataSource(),
+            ingestionSpec,
+            getContext()
+        );
+      }
     }
 
-    final String json = jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(indexTaskSpec);
-    log.info("Generated compaction task details: " + json);
+    if (indexTaskSpec == null) {
+      log.warn("Failed to generate compaction spec");
+      return TaskStatus.failure(getId());
+    } else {
+      final String json = jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(indexTaskSpec);
+      log.info("Generated compaction task details: " + json);
 
-    return indexTaskSpec.run(toolbox);
+      return indexTaskSpec.run(toolbox);
+    }
   }
 
+  /**
+   * Generate {@link IndexIngestionSpec} from input segments.
+   *
+   * @return null if input segments don't exist. Otherwise, a generated ingestionSpec.
+   */
+  @Nullable
   @VisibleForTesting
   static IndexIngestionSpec createIngestionSchema(
       TaskToolbox toolbox,
@@ -281,12 +292,22 @@ private static DataSchema createDataSchema(
       throws IOException, SegmentLoadingException
   {
     // find metadata for interval
-    final List<QueryableIndex> queryableIndices = loadSegments(timelineSegments, segmentFileMap, indexIO);
+    final List<Pair<QueryableIndex, DataSegment>> queryableIndexAndSegments = loadSegments(
+        timelineSegments,
+        segmentFileMap,
+        indexIO
+    );
 
     // find merged aggregators
-    final List<AggregatorFactory[]> aggregatorFactories = queryableIndices
+    for (Pair<QueryableIndex, DataSegment> pair : queryableIndexAndSegments) {
+      final QueryableIndex index = pair.lhs;
+      if (index.getMetadata() == null) {
+        throw new RE("Index metadata doesn't exist for segment[%s]", pair.rhs.getIdentifier());
+      }
+    }
+    final List<AggregatorFactory[]> aggregatorFactories = queryableIndexAndSegments
         .stream()
-        .map(index -> index.getMetadata().getAggregators())
+        .map(pair -> pair.lhs.getMetadata().getAggregators()) // We have already done null check on index.getMetadata()
         .collect(Collectors.toList());
     final AggregatorFactory[] mergedAggregators = AggregatorFactory.mergeAggregators(aggregatorFactories);
 
@@ -296,7 +317,11 @@ private static DataSchema createDataSchema(
 
     // find granularity spec
     // set rollup only if rollup is set for all segments
-    final boolean rollup = queryableIndices.stream().allMatch(index -> index.getMetadata().isRollup());
+    final boolean rollup = queryableIndexAndSegments.stream().allMatch(pair -> {
+      // We have already checked getMetadata() doesn't return null
+      final Boolean isRollup = pair.lhs.getMetadata().isRollup();
+      return isRollup != null && isRollup;
+    });
     final GranularitySpec granularitySpec = new ArbitraryGranularitySpec(
         new NoneGranularity(),
         rollup,
@@ -305,7 +330,7 @@ private static DataSchema createDataSchema(
 
     // find unique dimensions
     final DimensionsSpec finalDimensionsSpec = dimensionsSpec == null ?
-                                               createDimensionsSpec(queryableIndices) :
+                                               createDimensionsSpec(queryableIndexAndSegments) :
                                                dimensionsSpec;
     final InputRowParser parser = new NoopInputRowParser(new TimeAndDimsParseSpec(null, finalDimensionsSpec));
 
@@ -319,7 +344,7 @@ private static DataSchema createDataSchema(
     );
   }
 
-  private static DimensionsSpec createDimensionsSpec(List<QueryableIndex> queryableIndices)
+  private static DimensionsSpec createDimensionsSpec(List<Pair<QueryableIndex, DataSegment>> queryableIndices)
   {
     final BiMap<String, Integer> uniqueDims = HashBiMap.create();
     final Map<String, DimensionSchema> dimensionSchemaMap = new HashMap<>();
@@ -329,9 +354,24 @@ private static DimensionsSpec createDimensionsSpec(List<QueryableIndex> queryabl
     // Dimensions are extracted from the recent segments to olders because recent segments are likely to be queried more
     // frequently, and thus the performance should be optimized for recent ones rather than old ones.
 
-    // timelineSegments are sorted in order of interval
+    // timelineSegments are sorted in order of interval, but we do a sanity check here.
+    final Comparator<Interval> intervalComparator = Comparators.intervalsByStartThenEnd();
+    for (int i = 0; i < queryableIndices.size() - 1; i++) {
+      final Interval shouldBeSmaller = queryableIndices.get(i).lhs.getDataInterval();
+      final Interval shouldBeLarger = queryableIndices.get(i + 1).lhs.getDataInterval();
+      Preconditions.checkState(
+          intervalComparator.compare(shouldBeSmaller, shouldBeLarger) <= 0,
+          "QueryableIndexes are not sorted! Interval[%s] of segment[%s] is laster than interval[%s] of segment[%s]",
+          shouldBeSmaller,
+          queryableIndices.get(i).rhs.getIdentifier(),
+          shouldBeLarger,
+          queryableIndices.get(i + 1).rhs.getIdentifier()
+      );
+    }
+
     int index = 0;
-    for (QueryableIndex queryableIndex : Lists.reverse(queryableIndices)) {
+    for (Pair<QueryableIndex, DataSegment> pair : Lists.reverse(queryableIndices)) {
+      final QueryableIndex queryableIndex = pair.lhs;
       final Map<String, DimensionHandler> dimensionHandlerMap = queryableIndex.getDimensionHandlers();
 
       for (String dimension : queryableIndex.getAvailableDimensions()) {
@@ -376,23 +416,22 @@ private static DimensionsSpec createDimensionsSpec(List<QueryableIndex> queryabl
     return new DimensionsSpec(dimensionSchemas, null, null);
   }
 
-  private static List<QueryableIndex> loadSegments(
+  private static List<Pair<QueryableIndex, DataSegment>> loadSegments(
       List<TimelineObjectHolder<String, DataSegment>> timelineSegments,
       Map<DataSegment, File> segmentFileMap,
       IndexIO indexIO
   ) throws IOException
   {
-    final List<QueryableIndex> segments = new ArrayList<>();
+    final List<Pair<QueryableIndex, DataSegment>> segments = new ArrayList<>();
 
     for (TimelineObjectHolder<String, DataSegment> timelineSegment : timelineSegments) {
       final PartitionHolder<DataSegment> partitionHolder = timelineSegment.getObject();
       for (PartitionChunk<DataSegment> chunk : partitionHolder) {
         final DataSegment segment = chunk.getObject();
-        segments.add(
-            indexIO.loadIndex(
-                Preconditions.checkNotNull(segmentFileMap.get(segment), "File for segment %s", segment.getIdentifier())
-            )
+        final QueryableIndex queryableIndex = indexIO.loadIndex(
+            Preconditions.checkNotNull(segmentFileMap.get(segment), "File for segment %s", segment.getIdentifier())
         );
+        segments.add(Pair.of(queryableIndex, segment));
       }
     }
 
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java
index 4605995ee61..460225af1fe 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java
@@ -440,7 +440,8 @@ private void convertSegment(TaskToolbox toolbox) throws SegmentLoadingException,
         // Appending to the version makes a new version that inherits most comparability parameters of the original
         // version, but is "newer" than said original version.
         DataSegment updatedSegment = segment.withVersion(StringUtils.format("%s_v%s", segment.getVersion(), outVersion));
-        updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment);
+
+        updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment, false);
 
         actionClient.submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment)));
       } else {
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java
index 1a9bade78c9..1d88733f952 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java
@@ -24,11 +24,9 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Function;
 import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -36,7 +34,6 @@
 import com.google.common.hash.HashFunction;
 import com.google.common.hash.Hashing;
 import com.google.common.util.concurrent.ListenableFuture;
-import io.druid.data.input.Committer;
 import io.druid.data.input.Firehose;
 import io.druid.data.input.FirehoseFactory;
 import io.druid.data.input.InputRow;
@@ -57,7 +54,6 @@
 import io.druid.java.util.common.guava.Comparators;
 import io.druid.java.util.common.logger.Logger;
 import io.druid.java.util.common.parsers.ParseException;
-import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
 import io.druid.query.DruidMetrics;
 import io.druid.segment.IndexSpec;
 import io.druid.segment.indexing.DataSchema;
@@ -71,15 +67,15 @@
 import io.druid.segment.realtime.RealtimeMetricsMonitor;
 import io.druid.segment.realtime.appenderator.Appenderator;
 import io.druid.segment.realtime.appenderator.AppenderatorConfig;
-import io.druid.segment.realtime.appenderator.AppenderatorDriver;
+import io.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
 import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult;
 import io.druid.segment.realtime.appenderator.Appenderators;
+import io.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
 import io.druid.segment.realtime.appenderator.SegmentAllocator;
 import io.druid.segment.realtime.appenderator.SegmentIdentifier;
 import io.druid.segment.realtime.appenderator.SegmentsAndMetadata;
 import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
-import io.druid.segment.realtime.plumber.Committers;
-import io.druid.segment.realtime.plumber.NoopSegmentHandoffNotifierFactory;
+import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
 import io.druid.timeline.DataSegment;
 import io.druid.timeline.partition.HashBasedNumberedShardSpec;
 import io.druid.timeline.partition.NoneShardSpec;
@@ -284,16 +280,15 @@ private static String findVersion(Map<Interval, String> versions, Interval inter
   private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig)
   {
     Preconditions.checkState(
-        !(tuningConfig.isForceGuaranteedRollup() &&
-          (tuningConfig.isForceExtendableShardSpecs() || ioConfig.isAppendToExisting())),
-        "Perfect rollup cannot be guaranteed with extendable shardSpecs"
+        !tuningConfig.isForceGuaranteedRollup() || !ioConfig.isAppendToExisting(),
+        "Perfect rollup cannot be guaranteed when appending to existing dataSources"
     );
     return tuningConfig.isForceGuaranteedRollup();
   }
 
   private static boolean isExtendableShardSpecs(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig)
   {
-    return !isGuaranteedRollup(ioConfig, tuningConfig);
+    return tuningConfig.isForceExtendableShardSpecs() || ioConfig.isAppendToExisting();
   }
 
   /**
@@ -546,7 +541,7 @@ private static ShardSpecs createShardSpecsFromInput(
   }
 
   /**
-   * This method reads input data row by row and adds the read row to a proper segment using {@link AppenderatorDriver}.
+   * This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}.
    * If there is no segment for the row, a new one is created.  Segments can be published in the middle of reading inputs
    * if one of below conditions are satisfied.
    *
@@ -555,7 +550,7 @@ private static ShardSpecs createShardSpecsFromInput(
    * If the number of rows in a segment exceeds {@link IndexTuningConfig#targetPartitionSize}
    * </li>
    * <li>
-   * If the number of rows added to {@link AppenderatorDriver} so far exceeds {@link IndexTuningConfig#maxTotalRows}
+   * If the number of rows added to {@link BaseAppenderatorDriver} so far exceeds {@link IndexTuningConfig#maxTotalRows}
    * </li>
    * </ul>
    *
@@ -590,11 +585,7 @@ private boolean generateAndPublishSegments(
 
     final IndexIOConfig ioConfig = ingestionSchema.getIOConfig();
     final IndexTuningConfig tuningConfig = ingestionSchema.tuningConfig;
-    final long publishTimeout = tuningConfig.getPublishTimeout();
-    final long maxRowsInAppenderator = tuningConfig.getMaxTotalRows();
-    final int maxRowsInSegment = tuningConfig.getTargetPartitionSize() == null
-                                 ? Integer.MAX_VALUE
-                                 : tuningConfig.getTargetPartitionSize();
+    final long pushTimeout = tuningConfig.getPushTimeout();
     final boolean isGuaranteedRollup = isGuaranteedRollup(ioConfig, tuningConfig);
 
     final SegmentAllocator segmentAllocator;
@@ -644,7 +635,12 @@ private boolean generateAndPublishSegments(
         }
 
         final int partitionNum = counters.computeIfAbsent(interval, x -> new AtomicInteger()).getAndIncrement();
-        return new SegmentIdentifier(getDataSource(), interval, findVersion(versions, interval), new NumberedShardSpec(partitionNum, 0));
+        return new SegmentIdentifier(
+            getDataSource(),
+            interval,
+            findVersion(versions, interval),
+            new NumberedShardSpec(partitionNum, 0)
+        );
       };
     }
 
@@ -654,97 +650,73 @@ private boolean generateAndPublishSegments(
     };
 
     try (
-        final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig);
-        final AppenderatorDriver driver = newDriver(
-            appenderator,
-            toolbox,
-            segmentAllocator,
-            fireDepartmentMetrics
-        );
-        final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir)
+            final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig);
+            final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator);
+            final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir)
     ) {
-      final Supplier<Committer> committerSupplier = Committers.supplierFromFirehose(firehose);
+      driver.startJob();
 
-      if (driver.startJob() != null) {
-        driver.clear();
-      }
-
-      try {
-        while (firehose.hasMore()) {
-          try {
-            final InputRow inputRow = firehose.nextRow();
+      while (firehose.hasMore()) {
+        try {
+          final InputRow inputRow = firehose.nextRow();
 
-            if (inputRow == null) {
-              fireDepartmentMetrics.incrementThrownAway();
-              continue;
-            }
+          if (inputRow == null) {
+            fireDepartmentMetrics.incrementThrownAway();
+            continue;
+          }
 
-            final Optional<Interval> optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp());
-            if (!optInterval.isPresent()) {
-              fireDepartmentMetrics.incrementThrownAway();
-              continue;
-            }
+          final Optional<Interval> optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp());
+          if (!optInterval.isPresent()) {
+            fireDepartmentMetrics.incrementThrownAway();
+            continue;
+          }
 
-            final String sequenceName;
-
-            if (isGuaranteedRollup) {
-              // Sequence name is based solely on the shardSpec, and there will only be one segment per sequence.
-              final Interval interval = optInterval.get();
-              final ShardSpec shardSpec = shardSpecs.getShardSpec(interval, inputRow);
-              sequenceName = Appenderators.getSequenceName(interval, findVersion(versions, interval), shardSpec);
-            } else {
-              // Segments are created as needed, using a single sequence name. They may be allocated from the overlord
-              // (in append mode) or may be created on our own authority (in overwrite mode).
-              sequenceName = getId();
-            }
-            final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName, committerSupplier);
-
-            if (addResult.isOk()) {
-              // incremental segment publishment is allowed only when rollup don't have to be perfect.
-              if (!isGuaranteedRollup &&
-                  (addResult.getNumRowsInSegment() >= maxRowsInSegment ||
-                   addResult.getTotalNumRowsInAppenderator() >= maxRowsInAppenderator)) {
-                // There can be some segments waiting for being published even though any rows won't be added to them.
-                // If those segments are not published here, the available space in appenderator will be kept to be small
-                // which makes the size of segments smaller.
-                final SegmentsAndMetadata published = awaitPublish(
-                    driver.publishAll(
-                        publisher,
-                        committerSupplier.get()
-                    ),
-                    publishTimeout
-                );
-                // Even though IndexTask uses NoopHandoffNotifier which does nothing for segment handoff,
-                // the below code is needed to update the total number of rows added to the appenderator so far.
-                // See AppenderatorDriver.registerHandoff() and Appenderator.drop().
-                // A hard-coded timeout is used here because the below get() is expected to return immediately.
-                driver.registerHandoff(published).get(30, TimeUnit.SECONDS);
-              }
-            } else {
-              throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp());
-            }
+          final String sequenceName;
 
-            fireDepartmentMetrics.incrementProcessed();
+          if (isGuaranteedRollup) {
+            // Sequence name is based solely on the shardSpec, and there will only be one segment per sequence.
+            final Interval interval = optInterval.get();
+            final ShardSpec shardSpec = shardSpecs.getShardSpec(interval, inputRow);
+            sequenceName = Appenderators.getSequenceName(interval, findVersion(versions, interval), shardSpec);
+          } else {
+            // Segments are created as needed, using a single sequence name. They may be allocated from the overlord
+            // (in append mode) or may be created on our own authority (in overwrite mode).
+            sequenceName = getId();
           }
-          catch (ParseException e) {
-            if (tuningConfig.isReportParseExceptions()) {
-              throw e;
-            } else {
-              fireDepartmentMetrics.incrementUnparseable();
+          final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName);
+
+          if (addResult.isOk()) {
+            // incremental segment publishment is allowed only when rollup don't have to be perfect.
+            if (!isGuaranteedRollup &&
+                (exceedMaxRowsInSegment(addResult.getNumRowsInSegment(), tuningConfig) ||
+                 exceedMaxRowsInAppenderator(addResult.getTotalNumRowsInAppenderator(), tuningConfig))) {
+              // There can be some segments waiting for being published even though any rows won't be added to them.
+              // If those segments are not published here, the available space in appenderator will be kept to be small
+              // which makes the size of segments smaller.
+              final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout);
+              log.info("Pushed segments[%s]", pushed.getSegments());
             }
+          } else {
+            throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp());
+          }
+
+          fireDepartmentMetrics.incrementProcessed();
+        }
+        catch (ParseException e) {
+          if (tuningConfig.isReportParseExceptions()) {
+            throw e;
+          } else {
+            fireDepartmentMetrics.incrementUnparseable();
           }
         }
-      }
-      finally {
-        driver.persist(committerSupplier.get());
       }
 
+      final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout);
+      log.info("Pushed segments[%s]", pushed.getSegments());
+
       final SegmentsAndMetadata published = awaitPublish(
-          driver.publishAll(
-              publisher,
-              committerSupplier.get()
-          ),
-          publishTimeout
+          driver.publishAll(publisher),
+          pushTimeout
       );
 
       if (published == null) {
@@ -755,14 +727,7 @@ private boolean generateAndPublishSegments(
             "Published segments[%s]", Joiner.on(", ").join(
                 Iterables.transform(
                     published.getSegments(),
-                    new Function<DataSegment, String>()
-                    {
-                      @Override
-                      public String apply(DataSegment input)
-                      {
-                        return input.getIdentifier();
-                      }
-                    }
+                    DataSegment::getIdentifier
                 )
             )
         );
@@ -774,11 +739,24 @@ public String apply(DataSegment input)
     }
   }
 
+  private static boolean exceedMaxRowsInSegment(int numRowsInSegment, IndexTuningConfig indexTuningConfig)
+  {
+    // maxRowsInSegment should be null if numShards is set in indexTuningConfig
+    final Integer maxRowsInSegment = indexTuningConfig.getTargetPartitionSize();
+    return maxRowsInSegment != null && maxRowsInSegment <= numRowsInSegment;
+  }
+
+  private static boolean exceedMaxRowsInAppenderator(long numRowsInAppenderator, IndexTuningConfig indexTuningConfig)
+  {
+    // maxRowsInAppenderator should be null if numShards is set in indexTuningConfig
+    final Long maxRowsInAppenderator = indexTuningConfig.getMaxTotalRows();
+    return maxRowsInAppenderator != null && maxRowsInAppenderator <= numRowsInAppenderator;
+  }
+
   private static SegmentsAndMetadata awaitPublish(
       ListenableFuture<SegmentsAndMetadata> publishFuture,
       long publishTimeout
-  )
-      throws ExecutionException, InterruptedException, TimeoutException
+  ) throws ExecutionException, InterruptedException, TimeoutException
   {
     if (publishTimeout == 0) {
       return publishFuture.get();
@@ -805,20 +783,17 @@ private static Appenderator newAppenderator(
     );
   }
 
-  private static AppenderatorDriver newDriver(
+  private static BatchAppenderatorDriver newDriver(
       final Appenderator appenderator,
       final TaskToolbox toolbox,
-      final SegmentAllocator segmentAllocator,
-      final FireDepartmentMetrics metrics
+      final SegmentAllocator segmentAllocator
   )
   {
-    return new AppenderatorDriver(
+    return new BatchAppenderatorDriver(
         appenderator,
         segmentAllocator,
-        new NoopSegmentHandoffNotifierFactory(), // don't wait for handoff since we don't serve queries
         new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
-        toolbox.getObjectMapper(),
-        metrics
+        toolbox.getDataSegmentKiller()
     );
   }
 
@@ -950,21 +925,33 @@ public boolean isAppendToExisting()
     private static final boolean DEFAULT_FORCE_EXTENDABLE_SHARD_SPECS = false;
     private static final boolean DEFAULT_GUARANTEE_ROLLUP = false;
     private static final boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = false;
-    private static final long DEFAULT_PUBLISH_TIMEOUT = 0;
+    private static final long DEFAULT_PUSH_TIMEOUT = 0;
 
     static final int DEFAULT_TARGET_PARTITION_SIZE = 5000000;
 
     private final Integer targetPartitionSize;
     private final int maxRowsInMemory;
-    private final int maxTotalRows;
+    private final Long maxTotalRows;
     private final Integer numShards;
     private final IndexSpec indexSpec;
     private final File basePersistDirectory;
     private final int maxPendingPersists;
+
+    /**
+     * This flag is to force to always use an extendableShardSpec (like {@link NumberedShardSpec} even if
+     * {@link #forceGuaranteedRollup} is set.
+     */
     private final boolean forceExtendableShardSpecs;
+
+    /**
+     * This flag is to force _perfect rollup mode_. {@link IndexTask} will scan the whole input data twice to 1) figure
+     * out proper shard specs for each segment and 2) generate segments. Note that perfect rollup mode basically assumes
+     * that no more data will be appended in the future. As a result, in perfect rollup mode, {@link NoneShardSpec} and
+     * {@link HashBasedNumberedShardSpec} are used for a single shard and two or shards, respectively.
+     */
     private final boolean forceGuaranteedRollup;
     private final boolean reportParseExceptions;
-    private final long publishTimeout;
+    private final long pushTimeout;
     @Nullable
     private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
 
@@ -972,7 +959,7 @@ public boolean isAppendToExisting()
     public IndexTuningConfig(
         @JsonProperty("targetPartitionSize") @Nullable Integer targetPartitionSize,
         @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory,
-        @JsonProperty("maxTotalRows") @Nullable Integer maxTotalRows,
+        @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows,
         @JsonProperty("rowFlushBoundary") @Nullable Integer rowFlushBoundary_forBackCompatibility, // DEPRECATED
         @JsonProperty("numShards") @Nullable Integer numShards,
         @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec,
@@ -982,7 +969,8 @@ public IndexTuningConfig(
         @JsonProperty("forceExtendableShardSpecs") @Nullable Boolean forceExtendableShardSpecs,
         @JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup,
         @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions,
-        @JsonProperty("publishTimeout") @Nullable Long publishTimeout,
+        @JsonProperty("publishTimeout") @Nullable Long publishTimeout, // deprecated
+        @JsonProperty("pushTimeout") @Nullable Long pushTimeout,
         @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
     )
     {
@@ -996,7 +984,7 @@ public IndexTuningConfig(
           forceExtendableShardSpecs,
           forceGuaranteedRollup,
           reportParseExceptions,
-          publishTimeout,
+          pushTimeout != null ? pushTimeout : publishTimeout,
           null,
           segmentWriteOutMediumFactory
       );
@@ -1010,14 +998,14 @@ private IndexTuningConfig()
     private IndexTuningConfig(
         @Nullable Integer targetPartitionSize,
         @Nullable Integer maxRowsInMemory,
-        @Nullable Integer maxTotalRows,
+        @Nullable Long maxTotalRows,
         @Nullable Integer numShards,
         @Nullable IndexSpec indexSpec,
         @Nullable Integer maxPendingPersists,
         @Nullable Boolean forceExtendableShardSpecs,
         @Nullable Boolean forceGuaranteedRollup,
         @Nullable Boolean reportParseExceptions,
-        @Nullable Long publishTimeout,
+        @Nullable Long pushTimeout,
         @Nullable File basePersistDirectory,
         @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
     )
@@ -1027,15 +1015,9 @@ private IndexTuningConfig(
           "targetPartitionSize and numShards cannot both be set"
       );
 
-      this.targetPartitionSize = numShards != null && !numShards.equals(-1)
-                                 ? null
-                                 : (targetPartitionSize == null || targetPartitionSize.equals(-1)
-                                    ? DEFAULT_TARGET_PARTITION_SIZE
-                                    : targetPartitionSize);
+      this.targetPartitionSize = initializeTargetPartitionSize(numShards, targetPartitionSize);
       this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory;
-      this.maxTotalRows = maxTotalRows == null
-                          ? DEFAULT_MAX_TOTAL_ROWS
-                          : maxTotalRows;
+      this.maxTotalRows = initializeMaxTotalRows(numShards, maxTotalRows);
       this.numShards = numShards == null || numShards.equals(-1) ? null : numShards;
       this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec;
       this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists;
@@ -1046,17 +1028,32 @@ private IndexTuningConfig(
       this.reportParseExceptions = reportParseExceptions == null
                                    ? DEFAULT_REPORT_PARSE_EXCEPTIONS
                                    : reportParseExceptions;
-      this.publishTimeout = publishTimeout == null ? DEFAULT_PUBLISH_TIMEOUT : publishTimeout;
+      this.pushTimeout = pushTimeout == null ? DEFAULT_PUSH_TIMEOUT : pushTimeout;
       this.basePersistDirectory = basePersistDirectory;
 
-      Preconditions.checkArgument(
-          !(this.forceExtendableShardSpecs && this.forceGuaranteedRollup),
-          "Perfect rollup cannot be guaranteed with extendable shardSpecs"
-      );
-
       this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
     }
 
+    private static Integer initializeTargetPartitionSize(Integer numShards, Integer targetPartitionSize)
+    {
+      if (numShards == null || numShards == -1) {
+        return targetPartitionSize == null || targetPartitionSize.equals(-1)
+               ? DEFAULT_TARGET_PARTITION_SIZE
+               : targetPartitionSize;
+      } else {
+        return null;
+      }
+    }
+
+    private static Long initializeMaxTotalRows(Integer numShards, Long maxTotalRows)
+    {
+      if (numShards == null || numShards == -1) {
+        return maxTotalRows == null ? DEFAULT_MAX_TOTAL_ROWS : maxTotalRows;
+      } else {
+        return null;
+      }
+    }
+
     public IndexTuningConfig withBasePersistDirectory(File dir)
     {
       return new IndexTuningConfig(
@@ -1069,7 +1066,7 @@ public IndexTuningConfig withBasePersistDirectory(File dir)
           forceExtendableShardSpecs,
           forceGuaranteedRollup,
           reportParseExceptions,
-          publishTimeout,
+          pushTimeout,
           dir,
           segmentWriteOutMediumFactory
       );
@@ -1089,7 +1086,7 @@ public int getMaxRowsInMemory()
     }
 
     @JsonProperty
-    public int getMaxTotalRows()
+    public Long getMaxTotalRows()
     {
       return maxTotalRows;
     }
@@ -1150,9 +1147,9 @@ public boolean isReportParseExceptions()
     }
 
     @JsonProperty
-    public long getPublishTimeout()
+    public long getPushTimeout()
     {
-      return publishTimeout;
+      return pushTimeout;
     }
 
     @Override
@@ -1180,12 +1177,12 @@ public boolean equals(Object o)
       }
       IndexTuningConfig that = (IndexTuningConfig) o;
       return maxRowsInMemory == that.maxRowsInMemory &&
-             maxTotalRows == that.maxTotalRows &&
+             Objects.equals(maxTotalRows, that.maxTotalRows) &&
              maxPendingPersists == that.maxPendingPersists &&
              forceExtendableShardSpecs == that.forceExtendableShardSpecs &&
              forceGuaranteedRollup == that.forceGuaranteedRollup &&
              reportParseExceptions == that.reportParseExceptions &&
-             publishTimeout == that.publishTimeout &&
+             pushTimeout == that.pushTimeout &&
              Objects.equals(targetPartitionSize, that.targetPartitionSize) &&
              Objects.equals(numShards, that.numShards) &&
              Objects.equals(indexSpec, that.indexSpec) &&
@@ -1207,7 +1204,7 @@ public int hashCode()
           forceExtendableShardSpecs,
           forceGuaranteedRollup,
           reportParseExceptions,
-          publishTimeout,
+          pushTimeout,
           segmentWriteOutMediumFactory
       );
     }
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java
index 8a6cb91dea5..454899f5c77 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java
@@ -34,9 +34,9 @@
 import com.google.common.collect.Ordering;
 import com.google.common.collect.Sets;
 import com.google.common.hash.Hashing;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.indexing.common.TaskLock;
 import io.druid.indexing.common.TaskStatus;
 import io.druid.indexing.common.TaskToolbox;
@@ -185,7 +185,8 @@ public String apply(DataSegment input)
       long uploadStart = System.currentTimeMillis();
 
       // Upload file
-      final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, mergedSegment);
+
+      final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, mergedSegment, false);
 
       emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart));
       emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize()));
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java
index 0a3a8b48547..2a6b3b4723a 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java
@@ -28,7 +28,7 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.primitives.Ints;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.data.input.Committer;
 import io.druid.data.input.Firehose;
 import io.druid.data.input.FirehoseFactory;
diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java
index b6516c9a47e..61c245c57d9 100644
--- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java
+++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java
@@ -30,7 +30,7 @@
 import com.google.common.collect.HashBiMap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.data.input.Firehose;
 import io.druid.data.input.FirehoseFactory;
 import io.druid.data.input.impl.InputRowParser;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java
index ee103420207..7509c8cd815 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java
@@ -41,7 +41,7 @@
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.java.util.common.concurrent.Execs;
 import io.druid.guice.annotations.Self;
 import io.druid.indexer.TaskLocation;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java
index 479b92e92f3..011aaa5d167 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java
@@ -28,7 +28,7 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.indexing.common.TaskLock;
 import io.druid.indexing.common.TaskStatus;
 import io.druid.indexing.common.actions.TaskAction;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java
index c3e26bd5fac..53f5e227a5b 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java
@@ -43,12 +43,12 @@
 import com.google.common.util.concurrent.ListeningScheduledExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.SettableFuture;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.InputStreamResponseHandler;
-import com.metamx.http.client.response.StatusResponseHandler;
-import com.metamx.http.client.response.StatusResponseHolder;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.InputStreamResponseHandler;
+import io.druid.java.util.http.client.response.StatusResponseHandler;
+import io.druid.java.util.http.client.response.StatusResponseHolder;
 import io.druid.java.util.common.concurrent.Execs;
 import io.druid.concurrent.LifecycleLock;
 import io.druid.curator.CuratorUtils;
@@ -1229,6 +1229,11 @@ private void taskComplete(
   @Override
   public Collection<Worker> markWorkersLazy(Predicate<ImmutableWorkerInfo> isLazyWorker, int maxWorkers)
   {
+    // skip the lock and bail early if we should not mark any workers lazy (e.g. number
+    // of current workers is at or below the minNumWorkers of autoscaler config)
+    if (maxWorkers < 1) {
+      return Collections.emptyList();
+    }
     // status lock is used to prevent any tasks being assigned to the worker while we mark it lazy
     synchronized (statusLock) {
       Iterator<String> iterator = zkWorkers.keySet().iterator();
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java
index 5e0b165c0ca..44e7d501576 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java
@@ -22,7 +22,7 @@
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Supplier;
 import com.google.inject.Inject;
-import com.metamx.http.client.HttpClient;
+import io.druid.java.util.http.client.HttpClient;
 import io.druid.curator.cache.PathChildrenCacheFactory;
 import io.druid.guice.annotations.EscalatedGlobal;
 import io.druid.indexing.overlord.autoscaling.NoopProvisioningStrategy;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java
index ef478256d57..5a6c2a29773 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java
@@ -31,7 +31,7 @@
 import com.google.common.collect.Ordering;
 import com.google.common.collect.Sets;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.indexing.common.TaskLock;
 import io.druid.indexing.common.TaskLockType;
 import io.druid.indexing.common.task.Task;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java
index 4fa12b6decf..f1d14e10f2b 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java
@@ -22,8 +22,8 @@
 import com.google.common.base.Optional;
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.client.indexing.IndexingService;
 import io.druid.curator.discovery.ServiceAnnouncer;
 import io.druid.discovery.DruidLeaderSelector;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java
index a360f5c677b..56b5b2f3710 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java
@@ -33,9 +33,9 @@
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.indexing.common.TaskStatus;
 import io.druid.indexing.common.actions.TaskActionClientFactory;
 import io.druid.indexing.common.task.Task;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java
index 2a6d917d4d7..8067c96edbd 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java
@@ -19,7 +19,7 @@
 
 package io.druid.indexing.overlord;
 
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.indexer.TaskLocation;
 import io.druid.indexing.common.TaskStatus;
 import io.druid.java.util.common.Pair;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java
index b7d7008b093..d24d7e4d801 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java
@@ -29,9 +29,9 @@
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
 import io.druid.java.util.common.concurrent.Execs;
 import io.druid.concurrent.TaskThreadPriority;
 import io.druid.guice.annotations.Self;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/AbstractWorkerProvisioningStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/AbstractWorkerProvisioningStrategy.java
index ce53188416b..5ab74b5dd12 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/AbstractWorkerProvisioningStrategy.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/AbstractWorkerProvisioningStrategy.java
@@ -20,7 +20,7 @@
 package io.druid.indexing.overlord.autoscaling;
 
 import com.google.common.base.Supplier;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.indexing.overlord.WorkerTaskRunner;
 import io.druid.java.util.common.DateTimes;
 import io.druid.java.util.common.granularity.PeriodGranularity;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/NoopAutoScaler.java b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/NoopAutoScaler.java
index a83dc35f175..a4330c50df5 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/NoopAutoScaler.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/NoopAutoScaler.java
@@ -19,7 +19,7 @@
 
 package io.druid.indexing.overlord.autoscaling;
 
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 
 import io.druid.java.util.common.UOE;
 
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerProvisioningStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerProvisioningStrategy.java
index 476f6fdcb9c..916833b24aa 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerProvisioningStrategy.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerProvisioningStrategy.java
@@ -30,8 +30,8 @@
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.inject.Inject;
-import com.metamx.common.concurrent.ScheduledExecutors;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.common.concurrent.ScheduledExecutors;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.indexing.common.task.Task;
 import io.druid.indexing.overlord.ImmutableWorkerInfo;
 import io.druid.indexing.overlord.WorkerTaskRunner;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleWorkerProvisioningStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleWorkerProvisioningStrategy.java
index 8ac414f5720..c5d11ff8081 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleWorkerProvisioningStrategy.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleWorkerProvisioningStrategy.java
@@ -29,8 +29,8 @@
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.inject.Inject;
-import com.metamx.common.concurrent.ScheduledExecutors;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.common.concurrent.ScheduledExecutors;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.indexing.overlord.ImmutableWorkerInfo;
 import io.druid.indexing.overlord.TaskRunnerWorkItem;
 import io.druid.indexing.overlord.WorkerTaskRunner;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java
index 60ef3f24734..479253a2b59 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java
@@ -36,7 +36,7 @@
 import com.google.common.base.Function;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.Lists;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.indexing.overlord.autoscaling.AutoScaler;
 import io.druid.indexing.overlord.autoscaling.AutoScalingData;
 import io.druid.indexing.overlord.autoscaling.SimpleWorkerProvisioningConfig;
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorManager.java
index 1dd4d76d762..00f2c92c4b7 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorManager.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorManager.java
@@ -23,7 +23,7 @@
 import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.indexing.overlord.DataSourceMetadata;
 import io.druid.java.util.common.Pair;
 import io.druid.java.util.common.lifecycle.LifecycleStart;
@@ -71,6 +71,7 @@ public boolean createOrUpdateAndStartSupervisor(SupervisorSpec spec)
     Preconditions.checkState(started, "SupervisorManager not started");
     Preconditions.checkNotNull(spec, "spec");
     Preconditions.checkNotNull(spec.getId(), "spec.getId()");
+    Preconditions.checkNotNull(spec.getDataSources(), "spec.getDatasources()");
 
     synchronized (lock) {
       Preconditions.checkState(started, "SupervisorManager not started");
@@ -197,7 +198,7 @@ private boolean possiblyStopAndRemoveSupervisorInternal(String id, boolean write
     }
 
     if (writeTombstone) {
-      metadataSupervisorManager.insert(id, new NoopSupervisorSpec()); // where NoopSupervisorSpec is a tombstone
+      metadataSupervisorManager.insert(id, new NoopSupervisorSpec(null, pair.rhs.getDataSources())); // where NoopSupervisorSpec is a tombstone
     }
     pair.lhs.stop(true);
     supervisors.remove(id);
@@ -232,7 +233,7 @@ private boolean createAndStartSupervisorInternal(SupervisorSpec spec, boolean pe
     catch (Exception e) {
       // Supervisor creation or start failed write tombstone only when trying to start a new supervisor
       if (persistSpec) {
-        metadataSupervisorManager.insert(id, new NoopSupervisorSpec());
+        metadataSupervisorManager.insert(id, new NoopSupervisorSpec(null, spec.getDataSources()));
       }
       Throwables.propagate(e);
     }
diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java
index c23e028e6f1..09196df401c 100644
--- a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java
+++ b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java
@@ -22,10 +22,9 @@
 import com.google.common.base.Function;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.inject.Inject;
 import com.sun.jersey.spi.container.ResourceFilters;
@@ -49,6 +48,7 @@
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
@@ -61,6 +61,20 @@
 @Path("/druid/indexer/v1/supervisor")
 public class SupervisorResource
 {
+  private static final Function<VersionedSupervisorSpec, Iterable<ResourceAction>> SPEC_DATASOURCE_READ_RA_GENERATOR =
+      supervisorSpec -> {
+        if (supervisorSpec.getSpec() == null) {
+          return null;
+        }
+        if (supervisorSpec.getSpec().getDataSources() == null) {
+          return new ArrayList<>();
+        }
+        return Iterables.transform(
+            supervisorSpec.getSpec().getDataSources(),
+            AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR
+        );
+      };
+
   private final TaskMaster taskMaster;
   private final AuthConfig authConfig;
   private final AuthorizerMapper authorizerMapper;
@@ -216,26 +230,14 @@ public Response specGetAllHistory(@Context final HttpServletRequest req)
           @Override
           public Response apply(final SupervisorManager manager)
           {
-            final Map<String, List<VersionedSupervisorSpec>> supervisorHistory = manager.getSupervisorHistory();
-
-            final Set<String> authorizedSupervisorIds = filterAuthorizedSupervisorIds(
-                req,
-                manager,
-                supervisorHistory.keySet()
-            );
-
-            final Map<String, List<VersionedSupervisorSpec>> authorizedSupervisorHistory = Maps.filterKeys(
-                supervisorHistory,
-                new Predicate<String>()
-                {
-                  @Override
-                  public boolean apply(String id)
-                  {
-                    return authorizedSupervisorIds.contains(id);
-                  }
-                }
-            );
-            return Response.ok(supervisorHistory).build();
+            return Response.ok(
+                AuthorizationUtils.filterAuthorizedResources(
+                    req,
+                    manager.getSupervisorHistory(),
+                    SPEC_DATASOURCE_READ_RA_GENERATOR,
+                    authorizerMapper
+                )
+            ).build();
           }
         }
     );
@@ -244,8 +246,9 @@ public boolean apply(String id)
   @GET
   @Path("/{id}/history")
   @Produces(MediaType.APPLICATION_JSON)
-  @ResourceFilters(SupervisorResourceFilter.class)
-  public Response specGetHistory(@PathParam("id") final String id)
+  public Response specGetHistory(
+      @Context final HttpServletRequest req,
+      @PathParam("id") final String id)
   {
     return asLeaderWithSupervisorManager(
         new Function<SupervisorManager, Response>()
@@ -253,23 +256,32 @@ public Response specGetHistory(@PathParam("id") final String id)
           @Override
           public Response apply(SupervisorManager manager)
           {
-            Map<String, List<VersionedSupervisorSpec>> history = manager.getSupervisorHistory();
-            if (history.containsKey(id)) {
-              return Response.ok(history.get(id)).build();
-            } else {
-              return Response.status(Response.Status.NOT_FOUND)
-                             .entity(
-                                 ImmutableMap.of(
-                                     "error",
-                                     StringUtils.format(
-                                         "No history for [%s] (history available for %s)",
-                                         id,
-                                         history.keySet()
-                                     )
-                                 )
-                             )
-                             .build();
+            Map<String, List<VersionedSupervisorSpec>> supervisorHistory = manager.getSupervisorHistory();
+            Iterable<VersionedSupervisorSpec> historyForId = supervisorHistory.get(id);
+            if (historyForId != null) {
+              final List<VersionedSupervisorSpec> authorizedHistoryForId =
+                  Lists.newArrayList(
+                      AuthorizationUtils.filterAuthorizedResources(
+                          req,
+                          historyForId,
+                          SPEC_DATASOURCE_READ_RA_GENERATOR,
+                          authorizerMapper
+                      )
+                  );
+              if (authorizedHistoryForId.size() > 0) {
+                return Response.ok(authorizedHistoryForId).build();
+              }
             }
+
+            return Response.status(Response.Status.NOT_FOUND)
+                           .entity(
+                               ImmutableMap.of(
+                                   "error",
+                                   StringUtils.format("No history for [%s].", id)
+                               )
+                           )
+                           .build();
+
           }
         }
     );
diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java
index 5d470102f13..74f5c335754 100644
--- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java
+++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java
@@ -26,7 +26,7 @@
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.indexer.TaskLocation;
 import io.druid.indexing.common.TaskStatus;
 import io.druid.indexing.common.task.Task;
diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java
index 144c3512b90..d06830eaf6d 100644
--- a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java
+++ b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java
@@ -26,7 +26,7 @@
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.inject.Inject;
-import com.metamx.emitter.EmittingLogger;
+import io.druid.java.util.emitter.EmittingLogger;
 import io.druid.java.util.common.concurrent.Execs;
 import io.druid.indexing.common.TaskStatus;
 import io.druid.indexing.common.actions.TaskActionClientFactory;
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java
index 1542b33aef6..b52a3a351d0 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java
@@ -21,8 +21,8 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.metrics.MonitorScheduler;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.metrics.MonitorScheduler;
 import io.druid.client.cache.Cache;
 import io.druid.client.cache.CacheConfig;
 import io.druid.indexing.common.actions.TaskActionClientFactory;
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java
index 7543b0def3d..b5f79d678ac 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java
@@ -20,8 +20,8 @@
 package io.druid.indexing.common.actions;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.FullResponseHolder;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.FullResponseHolder;
 import io.druid.discovery.DruidLeaderClient;
 import io.druid.indexing.common.RetryPolicyConfig;
 import io.druid.indexing.common.RetryPolicyFactory;
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java
index b7e82324739..cef4c48bfee 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java
@@ -25,8 +25,8 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.indexing.common.TaskLock;
 import io.druid.indexing.common.task.NoopTask;
 import io.druid.indexing.common.task.Task;
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java
index 7bccdbab3d8..2db8255fa9c 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java
@@ -26,6 +26,7 @@
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import io.druid.data.input.FirehoseFactory;
 import io.druid.data.input.impl.DimensionSchema;
@@ -69,8 +70,8 @@
 import io.druid.segment.column.Column;
 import io.druid.segment.column.ColumnBuilder;
 import io.druid.segment.column.ValueType;
-import io.druid.segment.data.CompressionStrategy;
 import io.druid.segment.data.CompressionFactory.LongEncodingStrategy;
+import io.druid.segment.data.CompressionStrategy;
 import io.druid.segment.data.ListIndexed;
 import io.druid.segment.data.RoaringBitmapSerdeFactory;
 import io.druid.segment.incremental.IncrementalIndex;
@@ -84,6 +85,7 @@
 import org.hamcrest.CoreMatchers;
 import org.joda.time.Interval;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
@@ -125,10 +127,12 @@
   private static Map<String, AggregatorFactory> AGGREGATORS;
   private static List<DataSegment> SEGMENTS;
   private static ObjectMapper objectMapper = setupInjectablesInObjectMapper(new DefaultObjectMapper());
-  private static TaskToolbox toolbox;
+  private static Map<DataSegment, File> segmentMap;
+
+  private TaskToolbox toolbox;
 
   @BeforeClass
-  public static void setup()
+  public static void setupClass()
   {
     DIMENSIONS = new HashMap<>();
     AGGREGATORS = new HashMap<>();
@@ -161,7 +165,7 @@ public static void setup()
     AGGREGATORS.put("agg_3", new FloatFirstAggregatorFactory("agg_3", "float_dim_3"));
     AGGREGATORS.put("agg_4", new DoubleLastAggregatorFactory("agg_4", "double_dim_4"));
 
-    final Map<DataSegment, File> segmentMap = new HashMap<>(5);
+    segmentMap = new HashMap<>(5);
     for (int i = 0; i < 5; i++) {
       final Interval segmentInterval = Intervals.of(StringUtils.format("2017-0%d-01/2017-0%d-01", (i + 1), (i + 2)));
       segmentMap.put(
@@ -180,7 +184,11 @@ public static void setup()
       );
     }
     SEGMENTS = new ArrayList<>(segmentMap.keySet());
+  }
 
+  @Before
+  public void setup()
+  {
     toolbox = new TestTaskToolbox(
         new TestTaskActionClient(new ArrayList<>(segmentMap.keySet())),
         new TestIndexIO(objectMapper, segmentMap),
@@ -227,7 +235,7 @@ private static IndexTuningConfig createTuningConfig()
     return new IndexTuningConfig(
         5000000,
         500000,
-        1000000,
+        1000000L,
         null,
         null,
         new IndexSpec(
@@ -241,6 +249,7 @@ private static IndexTuningConfig createTuningConfig()
         false,
         true,
         false,
+        null,
         100L,
         null
     );
@@ -390,6 +399,24 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio
     );
   }
 
+  @Test
+  public void testMissingMetadata() throws IOException, SegmentLoadingException
+  {
+    expectedException.expect(RuntimeException.class);
+    expectedException.expectMessage(CoreMatchers.startsWith("Index metadata doesn't exist for segment"));
+
+    final TestIndexIO indexIO = (TestIndexIO) toolbox.getIndexIO();
+    indexIO.removeMetadata(Iterables.getFirst(indexIO.getQueryableIndexMap().keySet(), null));
+    final List<DataSegment> segments = new ArrayList<>(SEGMENTS);
+    CompactionTask.createIngestionSchema(
+        toolbox,
+        new SegmentProvider(segments),
+        null,
+        TUNING_CONFIG,
+        objectMapper
+    );
+  }
+
   private static DimensionsSpec getExpectedDimensionsSpecForAutoGeneration()
   {
     return new DimensionsSpec(
@@ -575,7 +602,7 @@ private static void assertIngestionSchema(
         }
 
         final Metadata metadata = new Metadata();
-        metadata.setAggregators(aggregatorFactories.toArray(new AggregatorFactory[aggregatorFactories.size()]));
+        metadata.setAggregators(aggregatorFactories.toArray(new AggregatorFactory[0]));
         metadata.setRollup(false);
 
         queryableIndexMap.put(
@@ -598,6 +625,31 @@ public QueryableIndex loadIndex(File file) throws IOException
     {
       return queryableIndexMap.get(file);
     }
+
+    void removeMetadata(File file)
+    {
+      final SimpleQueryableIndex index = (SimpleQueryableIndex) queryableIndexMap.get(file);
+      if (index != null) {
+        queryableIndexMap.put(
+            file,
+            new SimpleQueryableIndex(
+                index.getDataInterval(),
+                index.getColumnNames(),
+                index.getAvailableDimensions(),
+                index.getBitmapFactoryForDimensions(),
+                index.getColumns(),
+                index.getFileMapper(),
+                null,
+                index.getDimensionHandlers()
+            )
+        );
+      }
+    }
+
+    Map<File, QueryableIndex> getQueryableIndexMap()
+    {
+      return queryableIndexMap;
+    }
   }
 
   private static Column createColumn(DimensionSchema dimensionSchema)
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java
index 79310c9771d..7113efa6aa5 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java
@@ -57,14 +57,15 @@
 import io.druid.segment.IndexMergerV9;
 import io.druid.segment.IndexSpec;
 import io.druid.segment.indexing.DataSchema;
-import io.druid.segment.transform.ExpressionTransform;
-import io.druid.segment.transform.TransformSpec;
 import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
 import io.druid.segment.indexing.granularity.GranularitySpec;
 import io.druid.segment.indexing.granularity.UniformGranularitySpec;
+import io.druid.segment.loading.DataSegmentKiller;
 import io.druid.segment.loading.DataSegmentPusher;
 import io.druid.segment.realtime.appenderator.SegmentIdentifier;
 import io.druid.segment.realtime.firehose.LocalFirehoseFactory;
+import io.druid.segment.transform.ExpressionTransform;
+import io.druid.segment.transform.TransformSpec;
 import io.druid.timeline.DataSegment;
 import io.druid.timeline.partition.HashBasedNumberedShardSpec;
 import io.druid.timeline.partition.NoneShardSpec;
@@ -191,7 +192,7 @@ public void testForceExtendableShardSpecs() throws Exception
             tmpDir,
             null,
             null,
-            createTuningConfig(2, null, true, false),
+            createTuningConfig(2, null, true, true),
             false
         ),
         null
@@ -581,7 +582,7 @@ public void testWithSmallMaxTotalRows() throws Exception
                 Granularities.MINUTE,
                 null
             ),
-            createTuningConfig(2, 2, 2, null, false, false, true),
+            createTuningConfig(2, 2, 2L, null, false, false, true),
             false
         ),
         null
@@ -623,7 +624,7 @@ public void testPerfectRollup() throws Exception
                 true,
                 null
             ),
-            createTuningConfig(3, 2, 2, null, false, true, true),
+            createTuningConfig(3, 2, 2L, null, false, true, true),
             false
         ),
         null
@@ -664,7 +665,7 @@ public void testBestEffortRollup() throws Exception
                 true,
                 null
             ),
-            createTuningConfig(3, 2, 2, null, false, false, true),
+            createTuningConfig(3, 2, 2L, null, false, false, true),
             false
         ),
         null
@@ -1006,7 +1007,7 @@ public String getPathForHadoop()
       }
 
       @Override
-      public DataSegment push(File file, DataSegment segment) throws IOException
+      public DataSegment push(File file, DataSegment segment, boolean useUniquePath)
       {
         segments.add(segment);
         return segment;
@@ -1019,12 +1020,27 @@ public DataSegment push(File file, DataSegment segment) throws IOException
       }
     };
 
+    final DataSegmentKiller killer = new DataSegmentKiller()
+    {
+      @Override
+      public void kill(DataSegment segment)
+      {
+
+      }
+
+      @Override
+      public void killAll()
+      {
+
+      }
+    };
+
     final TaskToolbox box = new TaskToolbox(
         null,
         actionClient,
         null,
         pusher,
-        null,
+        killer,
         null,
         null,
         null,
@@ -1128,7 +1144,7 @@ private static IndexTuningConfig createTuningConfig(
   private static IndexTuningConfig createTuningConfig(
       Integer targetPartitionSize,
       Integer maxRowsInMemory,
-      Integer maxTotalRows,
+      Long maxTotalRows,
       Integer numShards,
       boolean forceExtendableShardSpecs,
       boolean forceGuaranteedRollup,
@@ -1148,6 +1164,7 @@ private static IndexTuningConfig createTuningConfig(
         forceGuaranteedRollup,
         reportParseException,
         null,
+        null,
         null
     );
   }
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java
index 521bba77a36..bea1c613145 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java
@@ -30,10 +30,10 @@
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.core.NoopEmitter;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.metrics.MonitorScheduler;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.core.NoopEmitter;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.metrics.MonitorScheduler;
 import io.druid.client.cache.CacheConfig;
 import io.druid.client.cache.MapCache;
 import io.druid.data.input.Firehose;
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java
index 63389a804ba..50932dbb231 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java
@@ -185,7 +185,8 @@ public void testRun() throws Exception
                 return null;
               }
             },
-            new NoopServiceEmitter(), new DataSegmentPusher()
+            new NoopServiceEmitter(),
+            new DataSegmentPusher()
             {
               @Deprecated
               @Override
@@ -201,12 +202,13 @@ public String getPathForHadoop()
               }
 
               @Override
-              public DataSegment push(File file, DataSegment segment) throws IOException
+              public DataSegment push(File file, DataSegment segment, boolean useUniquePath)
               {
                 // the merged segment is pushed to storage
                 segments.add(segment);
                 return segment;
               }
+
               @Override
               public Map<String, Object> makeLoadSpec(URI finalIndexZipFilePath)
               {
diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java
index b0484c88223..dd94f9508de 100644
--- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java
@@ -190,7 +190,7 @@ public void testIndexTaskSerde() throws Exception
                 jsonMapper
             ),
             new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true),
-            new IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, false, null, null, null)
+            new IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, false, null, null, null, null)
         ),
         null
     );
@@ -253,7 +253,7 @@ public void testIndexTaskwithResourceSerde() throws Exception
                 jsonMapper
             ),
             new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true),
-            new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null)
+            new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null, null)
         ),
         null
     );
diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
index 624ddc4cc42..9767ebf513a 100644
--- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
@@ -31,7 +31,6 @@
 import com.google.common.io.Files;
 import com.google.inject.Binder;
 import com.google.inject.Module;
-import com.metamx.emitter.service.ServiceEmitter;
 import io.druid.data.input.InputRow;
 import io.druid.data.input.impl.DimensionsSpec;
 import io.druid.data.input.impl.InputRowParser;
@@ -61,6 +60,7 @@
 import io.druid.java.util.common.JodaUtils;
 import io.druid.java.util.common.StringUtils;
 import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.math.expr.ExprMacroTable;
 import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
 import io.druid.query.aggregation.DoubleSumAggregatorFactory;
@@ -184,7 +184,8 @@
       }
 
       @Override
-      public List<DataSegment> getUsedSegmentsForIntervals(String dataSource, List<Interval> interval) throws IOException
+      public List<DataSegment> getUsedSegmentsForIntervals(String dataSource, List<Interval> interval)
+          throws IOException
       {
         return ImmutableList.copyOf(segmentSet);
       }
@@ -249,7 +250,7 @@ public String getPathForHadoop()
           }
 
           @Override
-          public DataSegment push(File file, DataSegment segment) throws IOException
+          public DataSegment push(File file, DataSegment segment, boolean useUniquePath)
           {
             return segment;
           }
@@ -537,7 +538,11 @@ public void simpleFirehoseReadingTest() throws IOException
         Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray());
         Assert.assertArrayEquals(new String[]{DIM_VALUE}, row.getDimension(DIM_NAME).toArray());
         Assert.assertEquals(METRIC_LONG_VALUE.longValue(), row.getMetric(METRIC_LONG_NAME));
-        Assert.assertEquals(METRIC_FLOAT_VALUE, row.getMetric(METRIC_FLOAT_NAME).floatValue(), METRIC_FLOAT_VALUE * 0.0001);
+        Assert.assertEquals(
+            METRIC_FLOAT_VALUE,
+            row.getMetric(METRIC_FLOAT_NAME).floatValue(),
+            METRIC_FLOAT_VALUE * 0.0001
+        );
         ++rowcount;
       }
     }
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java
index c2a199f872d..be311c3a39d 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java
@@ -28,8 +28,8 @@
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ListenableFuture;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.indexer.TaskState;
 import io.druid.indexing.common.IndexingServiceCondition;
 import io.druid.indexing.common.TaskStatus;
@@ -550,6 +550,24 @@ public boolean apply(ImmutableWorkerInfo input)
     Assert.assertEquals(1, remoteTaskRunner.getLazyWorkers().size());
   }
 
+  @Test
+  public void testFindLazyWorkerNotRunningAnyTaskButWithZeroMaxWorkers() throws Exception
+  {
+    doSetup();
+    Collection<Worker> lazyworkers = remoteTaskRunner.markWorkersLazy(
+        new Predicate<ImmutableWorkerInfo>()
+        {
+          @Override
+          public boolean apply(ImmutableWorkerInfo input)
+          {
+            return true;
+          }
+        }, 0
+    );
+    Assert.assertEquals(0, lazyworkers.size());
+    Assert.assertEquals(0, remoteTaskRunner.getLazyWorkers().size());
+  }
+
   @Test
   public void testWorkerZKReconnect() throws Exception
   {
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTestUtils.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTestUtils.java
index 7bcdff1530a..cc432881407 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTestUtils.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTestUtils.java
@@ -24,7 +24,7 @@
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 import com.google.common.base.Throwables;
-import com.metamx.http.client.HttpClient;
+import io.druid.java.util.http.client.HttpClient;
 import io.druid.common.guava.DSuppliers;
 import io.druid.curator.PotentiallyGzippedCompressionProvider;
 import io.druid.curator.cache.PathChildrenCacheFactory;
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java
index e408bd28c2d..e583a85b267 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java
@@ -34,10 +34,6 @@
 import com.google.common.collect.Lists;
 import com.google.common.collect.Ordering;
 import com.google.common.util.concurrent.MoreExecutors;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.metrics.Monitor;
-import com.metamx.metrics.MonitorScheduler;
 import io.druid.client.cache.MapCache;
 import io.druid.data.input.Firehose;
 import io.druid.data.input.FirehoseFactory;
@@ -82,6 +78,10 @@
 import io.druid.java.util.common.StringUtils;
 import io.druid.java.util.common.granularity.Granularities;
 import io.druid.java.util.common.guava.Comparators;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.metrics.Monitor;
+import io.druid.java.util.metrics.MonitorScheduler;
 import io.druid.metadata.DerbyMetadataStorageActionHandlerFactory;
 import io.druid.metadata.TestDerbyConnector;
 import io.druid.query.QueryRunnerFactoryConglomerate;
@@ -485,7 +485,7 @@ public String getPathForHadoop(String dataSource)
       }
 
       @Override
-      public DataSegment push(File file, DataSegment segment) throws IOException
+      public DataSegment push(File file, DataSegment segment, boolean useUniquePath)
       {
         pushedSegments++;
         return segment;
@@ -527,8 +527,11 @@ private TaskToolboxFactory setUpTaskToolboxFactory(
     Preconditions.checkNotNull(emitter);
 
     taskLockbox = new TaskLockbox(taskStorage);
-    tac = new LocalTaskActionClientFactory(taskStorage, new TaskActionToolbox(taskLockbox, mdc, emitter, EasyMock.createMock(
-        SupervisorManager.class)));
+    tac = new LocalTaskActionClientFactory(
+        taskStorage,
+        new TaskActionToolbox(taskLockbox, mdc, emitter, EasyMock.createMock(
+            SupervisorManager.class))
+    );
     File tmpDir = temporaryFolder.newFolder();
     taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null);
 
@@ -671,7 +674,7 @@ public void testIndexTask() throws Exception
                 mapper
             ),
             new IndexIOConfig(new MockFirehoseFactory(false), false),
-            new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null)
+            new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null, null)
         ),
         null
     );
@@ -729,7 +732,7 @@ public void testIndexTaskFailure() throws Exception
                 mapper
             ),
             new IndexIOConfig(new MockExceptionalFirehoseFactory(), false),
-            new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null)
+            new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null, null)
         ),
         null
     );
@@ -1034,7 +1037,7 @@ public String getPathForHadoop()
       }
 
       @Override
-      public DataSegment push(File file, DataSegment dataSegment) throws IOException
+      public DataSegment push(File file, DataSegment dataSegment, boolean useUniquePath)
       {
         throw new RuntimeException("FAILURE");
       }
@@ -1094,7 +1097,22 @@ public void testResumeTasks() throws Exception
                 mapper
             ),
             new IndexIOConfig(new MockFirehoseFactory(false), false),
-            new IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null, null, null, null)
+            new IndexTuningConfig(
+                10000,
+                10,
+                null,
+                null,
+                null,
+                indexSpec,
+                null,
+                false,
+                null,
+                null,
+                null,
+                null,
+                null,
+                null
+            )
         ),
         null
     );
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java
index 11266b60def..33dd21076e2 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java
@@ -21,8 +21,8 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.Iterables;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.indexing.common.TaskLock;
 import io.druid.indexing.common.TaskLockType;
 import io.druid.indexing.common.TaskStatus;
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java
index e6afde25818..426eb89378d 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java
@@ -23,9 +23,9 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.emitter.service.ServiceEventBuilder;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceEventBuilder;
 import io.druid.common.guava.DSuppliers;
 import io.druid.java.util.common.concurrent.Execs;
 import io.druid.indexer.TaskLocation;
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java
index f917d359c64..2e7294f5239 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java
@@ -24,9 +24,9 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.emitter.service.ServiceEventBuilder;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceEventBuilder;
 import io.druid.common.guava.DSuppliers;
 import io.druid.java.util.common.concurrent.Execs;
 import io.druid.indexer.TaskLocation;
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java
index 78242e8f860..22b9e512aaa 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java
@@ -120,8 +120,8 @@ public Access authorize(AuthenticationResult authenticationResult, Resource reso
 
   public void expectAuthorizationTokenCheck()
   {
-    AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid", null);
 
+    AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid", null, null);
     EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).anyTimes();
     EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT))
             .andReturn(authenticationResult)
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java
index b88556cf4fc..9d95c65afd0 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java
@@ -26,8 +26,8 @@
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.MoreExecutors;
-import com.metamx.emitter.EmittingLogger;
-import com.metamx.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
 import io.druid.curator.PotentiallyGzippedCompressionProvider;
 import io.druid.curator.discovery.NoopServiceAnnouncer;
 import io.druid.discovery.DruidLeaderSelector;
@@ -131,7 +131,7 @@ public void setUp() throws Exception
     req = EasyMock.createMock(HttpServletRequest.class);
     EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).anyTimes();
     EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
-        new AuthenticationResult("druid", "druid", null)
+        new AuthenticationResult("druid", "druid", null, null)
     ).anyTimes();
     req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
     EasyMock.expectLastCall().anyTimes();
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorManagerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorManagerTest.java
index c22dde9eb1f..8415b919b72 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorManagerTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorManagerTest.java
@@ -35,6 +35,7 @@
 import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
@@ -295,7 +296,7 @@ public Supervisor createSupervisor()
     @Override
     public List<String> getDataSources()
     {
-      return null;
+      return new ArrayList<>();
     }
 
   }
diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java
index 2c34ef83a28..cb72a192205 100644
--- a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java
@@ -19,6 +19,7 @@
 
 package io.druid.indexing.overlord.supervisor;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -28,9 +29,13 @@
 import io.druid.indexing.overlord.DataSourceMetadata;
 import io.druid.indexing.overlord.TaskMaster;
 import io.druid.java.util.common.DateTimes;
+import io.druid.server.security.Access;
+import io.druid.server.security.Action;
 import io.druid.server.security.AuthConfig;
-import io.druid.server.security.AuthTestUtils;
 import io.druid.server.security.AuthenticationResult;
+import io.druid.server.security.Authorizer;
+import io.druid.server.security.AuthorizerMapper;
+import io.druid.server.security.Resource;
 import org.easymock.Capture;
 import org.easymock.EasyMock;
 import org.easymock.EasyMockRunner;
@@ -43,6 +48,7 @@
 
 import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.core.Response;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -64,13 +70,40 @@
   @Before
   public void setUp() throws Exception
   {
-    supervisorResource = new SupervisorResource(taskMaster, new AuthConfig(), AuthTestUtils.TEST_AUTHORIZER_MAPPER);
+    supervisorResource = new SupervisorResource(
+        taskMaster,
+        new AuthConfig(),
+        new AuthorizerMapper(null) {
+          @Override
+          public Authorizer getAuthorizer(String name)
+          {
+            return new Authorizer()
+            {
+              @Override
+              public Access authorize(
+                  AuthenticationResult authenticationResult, Resource resource, Action action
+              )
+              {
+                if (authenticationResult.getIdentity().equals("druid")) {
+                  return Access.OK;
+                } else {
+                  if (resource.getName().equals("datasource2")) {
+                    return new Access(false, "not authorized.");
+                  } else {
+                    return Access.OK;
+                  }
+                }
+              }
+            };
+          }
+        }
+    );
   }
 
   @Test
   public void testSpecPost() throws Exception
   {
-    SupervisorSpec spec = new TestSupervisorSpec("my-id", null) {
+    SupervisorSpec spec = new TestSupervisorSpec("my-id", null, null) {
 
       @Override
       public List<String> getDataSources()
@@ -83,7 +116,7 @@ public void testSpecPost() throws Exception
     EasyMock.expect(supervisorManager.createOrUpdateAndStartSupervisor(spec)).andReturn(true);
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
-        new AuthenticationResult("druid", "druid", null)
+        new AuthenticationResult("druid", "druid", null, null)
     ).atLeastOnce();
     request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
     EasyMock.expectLastCall().anyTimes();
@@ -109,7 +142,7 @@ public void testSpecPost() throws Exception
   public void testSpecGetAll() throws Exception
   {
     Set<String> supervisorIds = ImmutableSet.of("id1", "id2");
-    SupervisorSpec spec1 = new TestSupervisorSpec("id1", null) {
+    SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, null) {
 
       @Override
       public List<String> getDataSources()
@@ -117,7 +150,7 @@ public void testSpecGetAll() throws Exception
         return Lists.newArrayList("datasource1");
       }
     };
-    SupervisorSpec spec2 = new TestSupervisorSpec("id2", null) {
+    SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, null) {
 
       @Override
       public List<String> getDataSources()
@@ -132,7 +165,7 @@ public void testSpecGetAll() throws Exception
     EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2));
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
-        new AuthenticationResult("druid", "druid", null)
+        new AuthenticationResult("druid", "druid", null, null)
     ).atLeastOnce();
     request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
     EasyMock.expectLastCall().anyTimes();
@@ -157,7 +190,7 @@ public void testSpecGetAll() throws Exception
   @Test
   public void testSpecGet() throws Exception
   {
-    SupervisorSpec spec = new TestSupervisorSpec("my-id", null);
+    SupervisorSpec spec = new TestSupervisorSpec("my-id", null, null);
 
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(2);
     EasyMock.expect(supervisorManager.getSupervisorSpec("my-id")).andReturn(Optional.of(spec));
@@ -255,33 +288,186 @@ public void testShutdown() throws Exception
   @Test
   public void testSpecGetAllHistory() throws Exception
   {
+    List<VersionedSupervisorSpec> versions1 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id1", null, Arrays.asList("datasource1")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id1", null, Arrays.asList("datasource1")),
+            "v2"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, Arrays.asList("datasource1")),
+            "tombstone"
+        )
+    );
+    List<VersionedSupervisorSpec> versions2 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v2"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, Arrays.asList("datasource2")),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v3"
+        )
+    );
+    List<VersionedSupervisorSpec> versions3 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource3")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, null),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource3")),
+            "v2"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, null),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource3")),
+            "v3"
+        )
+    );
     Map<String, List<VersionedSupervisorSpec>> history = Maps.newHashMap();
-    history.put("id1", null);
-    history.put("id2", null);
+    history.put("id1", versions1);
+    history.put("id2", versions2);
+    history.put("id3", versions3);
 
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(2);
     EasyMock.expect(supervisorManager.getSupervisorHistory()).andReturn(history);
-    SupervisorSpec spec1 = new TestSupervisorSpec("id1", null) {
+    SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, Arrays.asList("datasource1"));
+    SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, Arrays.asList("datasource2"));
+    EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1)).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2)).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
+        new AuthenticationResult("druid", "druid", null, null)
+    ).atLeastOnce();
+    request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
+    EasyMock.expectLastCall().anyTimes();
+    replayAll();
 
-      @Override
-      public List<String> getDataSources()
-      {
-        return Lists.newArrayList("datasource1");
-      }
-    };
-    SupervisorSpec spec2 = new TestSupervisorSpec("id2", null) {
+    Response response = supervisorResource.specGetAllHistory(request);
 
-      @Override
-      public List<String> getDataSources()
-      {
-        return Lists.newArrayList("datasource2");
-      }
-    };
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(history, response.getEntity());
+
+    resetAll();
+
+    EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.<SupervisorManager>absent());
+    replayAll();
+
+    response = supervisorResource.specGetAllHistory(request);
+    verifyAll();
+
+    Assert.assertEquals(503, response.getStatus());
+  }
+
+  @Test
+  public void testSpecGetAllHistoryWithAuthFailureFiltering() throws Exception
+  {
+    List<VersionedSupervisorSpec> versions1 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id1", null, Arrays.asList("datasource1")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id1", null, Arrays.asList("datasource1")),
+            "v2"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, Arrays.asList("datasource1")),
+            "tombstone"
+        )
+    );
+    List<VersionedSupervisorSpec> versions2 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v2"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, Arrays.asList("datasource2")),
+            "tombstone"
+        )
+    );
+    List<VersionedSupervisorSpec> versions3 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v2"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, Arrays.asList("datasource2")),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id3", null, Arrays.asList("datasource3")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, Arrays.asList("datasource3")),
+            "tombstone"
+        )
+    );
+    List<VersionedSupervisorSpec> versions4 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, null),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v2"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, null),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v3"
+        )
+    );
+
+    Map<String, List<VersionedSupervisorSpec>> history = Maps.newHashMap();
+    history.put("id1", versions1);
+    history.put("id2", versions2);
+    history.put("id3", versions3);
+    history.put("id4", versions4);
+
+    EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(2);
+    EasyMock.expect(supervisorManager.getSupervisorHistory()).andReturn(history);
+    SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, Arrays.asList("datasource1"));
+    SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, Arrays.asList("datasource2"));
     EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1)).atLeastOnce();
     EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2)).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
-        new AuthenticationResult("druid", "druid", null)
+        new AuthenticationResult("wronguser", "druid", null, null)
     ).atLeastOnce();
     request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
     EasyMock.expectLastCall().anyTimes();
@@ -289,8 +475,37 @@ public void testSpecGetAllHistory() throws Exception
 
     Response response = supervisorResource.specGetAllHistory(request);
 
+    Map<String, List<VersionedSupervisorSpec>> filteredHistory = Maps.newHashMap();
+    filteredHistory.put("id1", versions1);
+    filteredHistory.put(
+        "id3",
+        ImmutableList.of(
+            new VersionedSupervisorSpec(
+                new TestSupervisorSpec("id3", null, Arrays.asList("datasource3")),
+                "v1"
+            ),
+            new VersionedSupervisorSpec(
+                new NoopSupervisorSpec(null, Arrays.asList("datasource3")),
+                "tombstone"
+            )
+        )
+    );
+    filteredHistory.put(
+        "id4",
+        ImmutableList.of(
+            new VersionedSupervisorSpec(
+                new NoopSupervisorSpec(null, null),
+                "tombstone"
+            ),
+            new VersionedSupervisorSpec(
+                new NoopSupervisorSpec(null, null),
+                "tombstone"
+            )
+        )
+    );
+
     Assert.assertEquals(200, response.getStatus());
-    Assert.assertEquals(history, response.getEntity());
+    Assert.assertEquals(filteredHistory, response.getEntity());
 
     resetAll();
 
@@ -306,33 +521,193 @@ public void testSpecGetAllHistory() throws Exception
   @Test
   public void testSpecGetHistory() throws Exception
   {
-    List<VersionedSupervisorSpec> versions = ImmutableList.of(
-        new VersionedSupervisorSpec(null, "v1"),
-        new VersionedSupervisorSpec(null, "v2")
+    List<VersionedSupervisorSpec> versions1 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id1", null, Arrays.asList("datasource1")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, Arrays.asList("datasource1")),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id1", null, Arrays.asList("datasource1")),
+            "v2"
+        )
+    );
+    List<VersionedSupervisorSpec> versions2 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, Arrays.asList("datasource2")),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v2"
+        )
     );
     Map<String, List<VersionedSupervisorSpec>> history = Maps.newHashMap();
-    history.put("id1", versions);
-    history.put("id2", null);
+    history.put("id1", versions1);
+    history.put("id2", versions2);
 
-    EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(2);
-    EasyMock.expect(supervisorManager.getSupervisorHistory()).andReturn(history).times(2);
+    EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(3);
+    EasyMock.expect(supervisorManager.getSupervisorHistory()).andReturn(history).times(3);
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
+        new AuthenticationResult("druid", "druid", null, null)
+    ).atLeastOnce();
+    request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
+    EasyMock.expectLastCall().anyTimes();
+    replayAll();
+
+    Response response = supervisorResource.specGetHistory(request, "id1");
+
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(versions1, response.getEntity());
+
+    response = supervisorResource.specGetHistory(request, "id2");
+
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(versions2, response.getEntity());
+
+    response = supervisorResource.specGetHistory(request, "id3");
+
+    Assert.assertEquals(404, response.getStatus());
+
+    resetAll();
+
+    EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.<SupervisorManager>absent());
+    replayAll();
+
+    response = supervisorResource.specGetHistory(request, "id1");
+    verifyAll();
+
+    Assert.assertEquals(503, response.getStatus());
+  }
+
+  @Test
+  public void testSpecGetHistoryWithAuthFailure() throws Exception
+  {
+    List<VersionedSupervisorSpec> versions1 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id1", null, Arrays.asList("datasource1")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, Arrays.asList("datasource3")),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id1", null, Arrays.asList("datasource1")),
+            "v2"
+        )
+    );
+    List<VersionedSupervisorSpec> versions2 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, Arrays.asList("datasource2")),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id2", null, Arrays.asList("datasource2")),
+            "v2"
+        )
+    );
+    List<VersionedSupervisorSpec> versions3 = ImmutableList.of(
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id3", null, Arrays.asList("datasource3")),
+            "v1"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, null),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id3", null, Arrays.asList("datasource2")),
+            "v2"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, null),
+            "tombstone"
+        ),
+        new VersionedSupervisorSpec(
+            new TestSupervisorSpec("id3", null, Arrays.asList("datasource3")),
+            "v2"
+        ),
+        new VersionedSupervisorSpec(
+            new NoopSupervisorSpec(null, Arrays.asList("datasource3")),
+            "tombstone"
+        )
+    );
+    Map<String, List<VersionedSupervisorSpec>> history = Maps.newHashMap();
+    history.put("id1", versions1);
+    history.put("id2", versions2);
+    history.put("id3", versions3);
+
+    EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(4);
+    EasyMock.expect(supervisorManager.getSupervisorHistory()).andReturn(history).times(4);
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
+        new AuthenticationResult("notdruid", "druid", null, null)
+    ).atLeastOnce();
+    request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
+    EasyMock.expectLastCall().anyTimes();
     replayAll();
 
-    Response response = supervisorResource.specGetHistory("id1");
+    Response response = supervisorResource.specGetHistory(request, "id1");
 
     Assert.assertEquals(200, response.getStatus());
-    Assert.assertEquals(versions, response.getEntity());
+    Assert.assertEquals(versions1, response.getEntity());
 
-    response = supervisorResource.specGetHistory("id3");
+    response = supervisorResource.specGetHistory(request, "id2");
 
+    // user is not authorized to access datasource2
     Assert.assertEquals(404, response.getStatus());
 
+    response = supervisorResource.specGetHistory(request, "id3");
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(
+        ImmutableList.of(
+            new VersionedSupervisorSpec(
+                new TestSupervisorSpec("id3", null, Arrays.asList("datasource3")),
+                "v1"
+            ),
+            new VersionedSupervisorSpec(
+                new NoopSupervisorSpec(null, null),
+                "tombstone"
+            ),
+            new VersionedSupervisorSpec(
+                new NoopSupervisorSpec(null, null),
+                "tombstone"
+            ),
+            new VersionedSupervisorSpec(
+                new TestSupervisorSpec("id3", null, Arrays.asList("datasource3")),
+                "v2"
+            ),
+            new VersionedSupervisorSpec(
+                new NoopSupervisorSpec(null, Arrays.asList("datasource3")),
+                "tombstone"
+            )
+        ),
+        response.getEntity()
+    );
+
+    response = supervisorResource.specGetHistory(request, "id4");
+    Assert.assertEquals(404, response.getStatus());
+
+
     resetAll();
 
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.<SupervisorManager>absent());
     replayAll();
 
-    response = supervisorResource.specGetHistory("id1");
+    response = supervisorResource.specGetHistory(request, "id1");
     verifyAll();
 
     Assert.assertEquals(503, response.getStatus());
@@ -371,15 +746,34 @@ public void testReset() throws Exception
     verifyAll();
   }
 
+  @Test
+  public void testNoopSupervisorSpecSerde() throws Exception
+  {
+    ObjectMapper mapper = new ObjectMapper();
+    String oldSpec = "{\"type\":\"NoopSupervisorSpec\",\"id\":null,\"dataSources\":null}";
+    NoopSupervisorSpec expectedSpec = new NoopSupervisorSpec(null, null);
+    NoopSupervisorSpec deserializedSpec = mapper.readValue(oldSpec, NoopSupervisorSpec.class);
+    Assert.assertEquals(expectedSpec, deserializedSpec);
+
+    NoopSupervisorSpec spec1 = new NoopSupervisorSpec("abcd", Lists.newArrayList("defg"));
+    NoopSupervisorSpec spec2 = mapper.readValue(
+        mapper.writeValueAsBytes(spec1),
+        NoopSupervisorSpec.class
+    );
+    Assert.assertEquals(spec1, spec2);
+  }
+
   private static class TestSupervisorSpec implements SupervisorSpec
   {
     private final String id;
     private final Supervisor supervisor;
+    private final List<String> datasources;
 
-    public TestSupervisorSpec(String id, Supervisor supervisor)
+    public TestSupervisorSpec(String id, Supervisor supervisor, List<String> datasources)
     {
       this.id = id;
       this.supervisor = supervisor;
+      this.datasources = datasources;
     }
 
     @Override
@@ -397,7 +791,38 @@ public Supervisor createSupervisor()
     @Override
     public List<String> getDataSources()
     {
-      return null;
+      return datasources;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      TestSupervisorSpec that = (TestSupervisorSpec) o;
+
+      if (getId() != null ? !getId().equals(that.getId()) : that.getId() != null) {
+        return false;
+      }
+      if (supervisor != null ? !supervisor.equals(that.supervisor) : that.supervisor != null) {
+        return false;
+      }
+      return datasources != null ? datasources.equals(that.datasources) : that.datasources == null;
+
+    }
+
+    @Override
+    public int hashCode()
+    {
+      int result = getId() != null ? getId().hashCode() : 0;
+      result = 31 * result + (supervisor != null ? supervisor.hashCode() : 0);
+      result = 31 * result + (datasources != null ? datasources.hashCode() : 0);
+      return result;
     }
   }
 }
diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentPusher.java b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentPusher.java
index 923a8029960..b9b27f48d07 100644
--- a/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentPusher.java
+++ b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentPusher.java
@@ -25,7 +25,6 @@
 import io.druid.timeline.DataSegment;
 
 import java.io.File;
-import java.io.IOException;
 import java.net.URI;
 import java.util.Map;
 import java.util.Set;
@@ -48,7 +47,7 @@ public String getPathForHadoop()
   }
 
   @Override
-  public DataSegment push(File file, DataSegment segment) throws IOException
+  public DataSegment push(File file, DataSegment segment, boolean useUniquePath)
   {
     pushedSegments.add(segment);
     return segment;
diff --git a/integration-tests/docker/broker.conf b/integration-tests/docker/broker.conf
index e7e744d5db8..d791e82b70f 100644
--- a/integration-tests/docker/broker.conf
+++ b/integration-tests/docker/broker.conf
@@ -34,6 +34,8 @@ command=java
   -Ddruid.escalator.authorizerName=basic
   -Ddruid.auth.authorizers="[\"basic\"]"
   -Ddruid.auth.authorizer.basic.type=basic
+  -Ddruid.sql.enable=true
+  -Ddruid.sql.avatica.enable=true
   -cp /shared/docker/lib/*
   io.druid.cli.Main server broker
 redirect_stderr=true
diff --git a/integration-tests/docker/router.conf b/integration-tests/docker/router.conf
index 3222c18db36..06af3aab265 100644
--- a/integration-tests/docker/router.conf
+++ b/integration-tests/docker/router.conf
@@ -23,6 +23,8 @@ command=java
   -Ddruid.escalator.authorizerName=basic
   -Ddruid.auth.authorizers="[\"basic\"]"
   -Ddruid.auth.authorizer.basic.type=basic
+  -Ddruid.sql.enable=true
+  -Ddruid.sql.avatica.enable=true
   -cp /shared/docker/lib/*
   io.druid.cli.Main server router
 redirect_stderr=true
diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml
index 0c17f2fa203..7b90aa63ac6 100644
--- a/integration-tests/pom.xml
+++ b/integration-tests/pom.xml
@@ -26,7 +26,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
     </parent>
 
     <properties>
@@ -98,6 +98,16 @@
         </dependency>
 
         <!-- Tests -->
+        <dependency>
+            <groupId>org.apache.calcite.avatica</groupId>
+            <artifactId>avatica</artifactId>
+            <version>1.10.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.calcite.avatica</groupId>
+            <artifactId>avatica-server</artifactId>
+            <version>1.10.0</version>
+        </dependency>
         <dependency>
             <groupId>org.testng</groupId>
             <artifactId>testng</artifactId>
diff --git a/integration-tests/src/main/java/io/druid/testing/clients/ClientInfoResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/ClientInfoResourceTestClient.java
index 9c66444d211..65bccb904f5 100644
--- a/integration-tests/src/main/java/io/druid/testing/clients/ClientInfoResourceTestClient.java
+++ b/integration-tests/src/main/java/io/druid/testing/clients/ClientInfoResourceTestClient.java
@@ -24,10 +24,10 @@
 import com.google.common.base.Charsets;
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.StatusResponseHandler;
-import com.metamx.http.client.response.StatusResponseHolder;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.StatusResponseHandler;
+import io.druid.java.util.http.client.response.StatusResponseHolder;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.StringUtils;
 import io.druid.testing.IntegrationTestingConfig;
diff --git a/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java
index 9ce84665343..367094cfdaf 100644
--- a/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java
+++ b/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java
@@ -24,10 +24,10 @@
 import com.google.common.base.Charsets;
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.StatusResponseHandler;
-import com.metamx.http.client.response.StatusResponseHolder;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.StatusResponseHandler;
+import io.druid.java.util.http.client.response.StatusResponseHolder;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.RE;
 import io.druid.java.util.common.StringUtils;
diff --git a/integration-tests/src/main/java/io/druid/testing/clients/EventReceiverFirehoseTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/EventReceiverFirehoseTestClient.java
index d41729d592e..83ebf5fe013 100644
--- a/integration-tests/src/main/java/io/druid/testing/clients/EventReceiverFirehoseTestClient.java
+++ b/integration-tests/src/main/java/io/druid/testing/clients/EventReceiverFirehoseTestClient.java
@@ -24,10 +24,10 @@
 import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
 import com.google.common.base.Charsets;
 import com.google.common.base.Throwables;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.StatusResponseHandler;
-import com.metamx.http.client.response.StatusResponseHolder;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.StatusResponseHandler;
+import io.druid.java.util.http.client.response.StatusResponseHolder;
 import io.druid.java.util.common.jackson.JacksonUtils;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.StringUtils;
diff --git a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java
index f47d97e4f53..38c6e3d174f 100644
--- a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java
+++ b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java
@@ -25,10 +25,10 @@
 import com.google.common.base.Predicates;
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.StatusResponseHandler;
-import com.metamx.http.client.response.StatusResponseHolder;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.StatusResponseHandler;
+import io.druid.java.util.http.client.response.StatusResponseHolder;
 import io.druid.indexer.TaskState;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.RetryUtils;
diff --git a/integration-tests/src/main/java/io/druid/testing/clients/QueryResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/QueryResourceTestClient.java
index 8060871ba5d..b6a2c6bcce3 100644
--- a/integration-tests/src/main/java/io/druid/testing/clients/QueryResourceTestClient.java
+++ b/integration-tests/src/main/java/io/druid/testing/clients/QueryResourceTestClient.java
@@ -25,10 +25,10 @@
 import com.google.common.base.Charsets;
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.StatusResponseHandler;
-import com.metamx.http.client.response.StatusResponseHolder;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.StatusResponseHandler;
+import io.druid.java.util.http.client.response.StatusResponseHolder;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.StringUtils;
 import io.druid.query.Query;
diff --git a/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java
index b2cb8b00c13..9383a032a39 100644
--- a/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java
+++ b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java
@@ -24,13 +24,13 @@
 import com.google.inject.Binder;
 import com.google.inject.Module;
 import com.google.inject.Provides;
-import com.metamx.common.lifecycle.Lifecycle;
-import com.metamx.emitter.core.LoggingEmitter;
-import com.metamx.emitter.core.LoggingEmitterConfig;
-import com.metamx.emitter.service.ServiceEmitter;
-import com.metamx.http.client.CredentialedHttpClient;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.auth.BasicCredentials;
+import io.druid.java.util.common.lifecycle.Lifecycle;
+import io.druid.java.util.emitter.core.LoggingEmitter;
+import io.druid.java.util.emitter.core.LoggingEmitterConfig;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.http.client.CredentialedHttpClient;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.auth.BasicCredentials;
 import io.druid.curator.CuratorConfig;
 import io.druid.guice.JsonConfigProvider;
 import io.druid.guice.ManageLifecycle;
diff --git a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java
index bece170369d..ed809088e00 100644
--- a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java
+++ b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java
@@ -23,10 +23,10 @@
 import com.google.common.base.Throwables;
 import com.google.inject.Injector;
 import com.google.inject.Key;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.response.StatusResponseHandler;
-import com.metamx.http.client.response.StatusResponseHolder;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.StatusResponseHandler;
+import io.druid.java.util.http.client.response.StatusResponseHolder;
 import io.druid.java.util.common.StringUtils;
 import io.druid.java.util.common.lifecycle.Lifecycle;
 import io.druid.java.util.common.logger.Logger;
diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java
index 530250c14a6..54d31540b20 100644
--- a/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java
+++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java
@@ -21,7 +21,7 @@
 
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
-import com.metamx.http.client.HttpClient;
+import io.druid.java.util.http.client.HttpClient;
 import io.druid.curator.discovery.ServerDiscoveryFactory;
 import io.druid.curator.discovery.ServerDiscoverySelector;
 import io.druid.java.util.common.DateTimes;
diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java
index 645f66dddae..392a3935874 100644
--- a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java
+++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java
@@ -22,7 +22,7 @@
 import com.beust.jcommander.internal.Lists;
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
-import com.metamx.http.client.HttpClient;
+import io.druid.java.util.http.client.HttpClient;
 import io.druid.curator.discovery.ServerDiscoveryFactory;
 import io.druid.curator.discovery.ServerDiscoverySelector;
 import io.druid.java.util.common.DateTimes;
diff --git a/integration-tests/src/test/java/io/druid/tests/security/ITBasicAuthConfigurationTest.java b/integration-tests/src/test/java/io/druid/tests/security/ITBasicAuthConfigurationTest.java
index d03ce64b346..572b565fbb9 100644
--- a/integration-tests/src/test/java/io/druid/tests/security/ITBasicAuthConfigurationTest.java
+++ b/integration-tests/src/test/java/io/druid/tests/security/ITBasicAuthConfigurationTest.java
@@ -24,12 +24,12 @@
 import com.google.common.base.Charsets;
 import com.google.common.base.Throwables;
 import com.google.inject.Inject;
-import com.metamx.http.client.CredentialedHttpClient;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.Request;
-import com.metamx.http.client.auth.BasicCredentials;
-import com.metamx.http.client.response.StatusResponseHandler;
-import com.metamx.http.client.response.StatusResponseHolder;
+import io.druid.java.util.http.client.CredentialedHttpClient;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.auth.BasicCredentials;
+import io.druid.java.util.http.client.response.StatusResponseHandler;
+import io.druid.java.util.http.client.response.StatusResponseHolder;
 import io.druid.guice.annotations.Client;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.StringUtils;
@@ -39,8 +39,10 @@
 import io.druid.server.security.Resource;
 import io.druid.server.security.ResourceAction;
 import io.druid.server.security.ResourceType;
+import io.druid.sql.avatica.DruidAvaticaHandler;
 import io.druid.testing.IntegrationTestingConfig;
 import io.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.calcite.avatica.AvaticaSqlException;
 import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.testng.Assert;
@@ -49,9 +51,14 @@
 
 import javax.ws.rs.core.MediaType;
 import java.net.URL;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 
 @Guice(moduleFactory = DruidTestModuleFactory.class)
 public class ITBasicAuthConfigurationTest
@@ -197,8 +204,80 @@ public void testAuthConfiguration() throws Exception
 
     LOG.info("Checking access for user druid99.");
     checkNodeAccess(newUser99Client);
+
+    String brokerUrl = "jdbc:avatica:remote:url=" + config.getBrokerUrl() + DruidAvaticaHandler.AVATICA_PATH;
+    String routerUrl = "jdbc:avatica:remote:url=" + config.getRouterUrl() + DruidAvaticaHandler.AVATICA_PATH;
+
+    LOG.info("Checking Avatica query on broker.");
+    testAvaticaQuery(brokerUrl);
+
+    LOG.info("Checking Avatica query on router.");
+    testAvaticaQuery(routerUrl);
+
+    LOG.info("Testing Avatica query on broker with incorrect credentials.");
+    testAvaticaAuthFailure(brokerUrl);
+    
+    LOG.info("Testing Avatica query on router with incorrect credentials.");
+    testAvaticaAuthFailure(routerUrl);
+
+    LOG.info("Checking OPTIONS requests on services...");
+    testOptionsRequests(adminClient);
+  }
+
+  private void testOptionsRequests(HttpClient httpClient)
+  {
+    makeRequest(httpClient, HttpMethod.OPTIONS, config.getCoordinatorUrl() + "/status", null);
+    makeRequest(httpClient, HttpMethod.OPTIONS, config.getIndexerUrl() + "/status", null);
+    makeRequest(httpClient, HttpMethod.OPTIONS, config.getBrokerUrl() + "/status", null);
+    makeRequest(httpClient, HttpMethod.OPTIONS, config.getHistoricalUrl() + "/status", null);
+    makeRequest(httpClient, HttpMethod.OPTIONS, config.getRouterUrl() + "/status", null);
   }
 
+  private void testAvaticaQuery(String url)
+  {
+    LOG.info("URL: " + url);
+    try {
+      Properties connectionProperties = new Properties();
+      connectionProperties.put("user", "admin");
+      connectionProperties.put("password", "priest");
+      Connection connection = DriverManager.getConnection(url, connectionProperties);
+      Statement statement = connection.createStatement();
+      statement.setMaxRows(450);
+      String query = "SELECT * FROM INFORMATION_SCHEMA.COLUMNS";
+      ResultSet resultSet = statement.executeQuery(query);
+      Assert.assertTrue(resultSet.next());
+      statement.close();
+      connection.close();
+    }
+    catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private void testAvaticaAuthFailure(String url) throws Exception
+  {
+    LOG.info("URL: " + url);
+    try {
+      Properties connectionProperties = new Properties();
+      connectionProperties.put("user", "admin");
+      connectionProperties.put("password", "wrongpassword");
+      Connection connection = DriverManager.getConnection(url, connectionProperties);
+      Statement statement = connection.createStatement();
+      statement.setMaxRows(450);
+      String query = "SELECT * FROM INFORMATION_SCHEMA.COLUMNS";
+      statement.executeQuery(query);
+    }
+    catch (AvaticaSqlException ase) {
+      Assert.assertEquals(
+          ase.getErrorMessage(),
+          "Error while executing SQL \"SELECT * FROM INFORMATION_SCHEMA.COLUMNS\": Remote driver error: ForbiddenException: Authentication failed."
+      );
+      return;
+    }
+    Assert.fail("Test failed, did not get AvaticaSqlException.");
+  }
+
+
   private void checkNodeAccess(HttpClient httpClient)
   {
     makeRequest(httpClient, HttpMethod.GET, config.getCoordinatorUrl() + "/status", null);
diff --git a/integration-tests/src/test/resources/indexer/union_select_query.json b/integration-tests/src/test/resources/indexer/union_select_query.json
index 76e4d0be63e..f35e682c313 100644
--- a/integration-tests/src/test/resources/indexer/union_select_query.json
+++ b/integration-tests/src/test/resources/indexer/union_select_query.json
@@ -75,7 +75,7 @@
                         }
                     },
                     {
-                        "segmentId": "wikipedia_index_test0_2013-08-31T0com.metamx.common.ISE: one or more twitter  queries failed0:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:43.993Z",
+                        "segmentId": "wikipedia_index_test0_2013-08-31T0io.druid.java.util.common.ISE: one or more twitter  queries failed0:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:43.993Z",
                         "offset": 0,
                         "event": {
                             "timestamp": "2013-08-31T03:32:45.000Z",
diff --git a/java-util/pom.xml b/java-util/pom.xml
index 84f093753e4..0337eaa1629 100644
--- a/java-util/pom.xml
+++ b/java-util/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>io.druid</groupId>
         <artifactId>druid</artifactId>
-        <version>0.12.0-SNAPSHOT</version>
+        <version>0.12.1-rc2</version>
     </parent>
 
     <artifactId>java-util</artifactId>
@@ -38,6 +38,12 @@
         </license>
     </licenses>
 
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <sigar.base.version>1.6.5</sigar.base.version>
+        <sigar.version>${sigar.base.version}.132</sigar.version>
+    </properties>
+
     <dependencies>
         <dependency>
             <groupId>org.slf4j</groupId>
@@ -85,6 +91,12 @@
                 </exclusion>
             </exclusions>
         </dependency>
+        <dependency>
+            <groupId>net.thisptr</groupId>
+            <artifactId>jackson-jq</artifactId>
+        </dependency>
+
+        <!-- Extra dependencies for emitter -->
         <dependency>
             <groupId>com.google.code.findbugs</groupId>
             <artifactId>jsr305</artifactId>
@@ -93,6 +105,37 @@
             <groupId>javax.validation</groupId>
             <artifactId>validation-api</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.asynchttpclient</groupId>
+            <artifactId>async-http-client</artifactId>
+        </dependency>
+
+        <!-- Extra dependencies for server-metrics -->
+        <dependency>
+            <groupId>org.hyperic</groupId>
+            <artifactId>sigar</artifactId>
+            <version>${sigar.version}</version>
+            <optional>true</optional>
+        </dependency>
+        <dependency>
+            <groupId>org.hyperic</groupId>
+            <artifactId>sigar-dist</artifactId>
+            <version>${sigar.version}</version>
+            <type>zip</type>
+            <!-- "Provided" because this dependency is used only during the build itself: some files from this
+            dependency are copied as resources. See maven-dependency-plugin configuration and <resources> below. -->
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.gridkit.lab</groupId>
+            <artifactId>jvm-attach-api</artifactId>
+        </dependency>
+
+        <!-- Extra dependency for http-client -->
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty</artifactId>
+        </dependency>
 
         <!-- Tests -->
         <dependency>
@@ -111,10 +154,46 @@
             <scope>test</scope>
             <optional>true</optional>
         </dependency>
-      <dependency>
-        <groupId>net.thisptr</groupId>
-        <artifactId>jackson-jq</artifactId>
-      </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-api</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-slf4j-impl</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-1.2-api</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-jul</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>it.unimi.dsi</groupId>
+            <artifactId>fastutil</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>
@@ -141,7 +220,56 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>copy-sigar-lib-to-resources</id>
+                        <phase>generate-resources</phase>
+                        <goals>
+                            <goal>unpack-dependencies</goal>
+                        </goals>
+                        <configuration>
+                            <includeGroupIds>org.hyperic</includeGroupIds>
+                            <includeArtifactIds>sigar-dist</includeArtifactIds>
+                            <includes>**/sigar-bin/lib/*</includes>
+                            <excludes>**/sigar-bin/lib/*jar</excludes>
+                            <outputDirectory>
+                                ${project.build.directory}
+                            </outputDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <systemProperties>
+                        <property>
+                            <name>java.library.path</name>
+                            <value>${project.build.directory}/hyperic-sigar-${sigar.base.version}/sigar-bin/lib/</value>
+                        </property>
+                    </systemProperties>
+                </configuration>
+            </plugin>
         </plugins>
+
+        <resources>
+            <resource>
+                <directory>
+                    ${project.build.directory}/hyperic-sigar-${sigar.base.version}/sigar-bin/lib
+                </directory>
+            </resource>
+        </resources>
     </build>
 
-</project>
\ No newline at end of file
+    <repositories>
+        <repository>
+            <id>sigar</id>
+            <url>https://repository.jboss.org/nexus/content/repositories/thirdparty-uploads/</url>
+        </repository>
+    </repositories>
+
+</project>
diff --git a/java-util/src/main/java/io/druid/concurrent/ConcurrentAwaitableCounter.java b/java-util/src/main/java/io/druid/concurrent/ConcurrentAwaitableCounter.java
new file mode 100644
index 00000000000..e9ffa61ff72
--- /dev/null
+++ b/java-util/src/main/java/io/druid/concurrent/ConcurrentAwaitableCounter.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.concurrent;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.AbstractQueuedLongSynchronizer;
+
+/**
+ * This synchronization object allows to {@link #increment} a counter without blocking, potentially from multiple
+ * threads (although in some use cases there is just one incrementer thread), and block in other thread(s), awaiting
+ * when the count reaches the provided value: see {@link #awaitCount}, or the specified number of events since the
+ * call: see {@link #awaitNextIncrements}.
+ *
+ * This counter wraps around {@link Long#MAX_VALUE} and starts from 0 again, so "next" count should be generally
+ * obtained by calling {@link #nextCount nextCount(currentCount)} rather than {@code currentCount + 1}.
+ *
+ * Memory consistency effects: actions in threads prior to calling {@link #increment} while the count was less than the
+ * awaited value happen-before actions following count awaiting methods such as {@link #awaitCount}.
+ */
+public final class ConcurrentAwaitableCounter
+{
+  private static final long MAX_COUNT = Long.MAX_VALUE;
+
+  /**
+   * This method should be called to obtain the next total increment count to be passed to {@link #awaitCount} methods,
+   * instead of just adding 1 to the previous count, because the count must wrap around {@link Long#MAX_VALUE} and start
+   * from 0 again.
+   */
+  public static long nextCount(long prevCount)
+  {
+    return (prevCount + 1) & MAX_COUNT;
+  }
+
+  private static class Sync extends AbstractQueuedLongSynchronizer
+  {
+    @Override
+    protected long tryAcquireShared(long countWhenWaitStarted)
+    {
+      long currentCount = getState();
+      return compareCounts(currentCount, countWhenWaitStarted) > 0 ? 1 : -1;
+    }
+
+    @Override
+    protected boolean tryReleaseShared(long increment)
+    {
+      long count;
+      long nextCount;
+      do {
+        count = getState();
+        nextCount = (count + increment) & MAX_COUNT;
+      } while (!compareAndSetState(count, nextCount));
+      return true;
+    }
+
+    long getCount()
+    {
+      return getState();
+    }
+  }
+
+  private final Sync sync = new Sync();
+
+  /**
+   * Increment the count. This method could be safely called from concurrent threads.
+   */
+  public void increment()
+  {
+    sync.releaseShared(1);
+  }
+
+  /**
+   * Await until the {@link #increment} is called on this counter object the specified number of times from the creation
+   * of this counter object.
+   */
+  public void awaitCount(long totalCount) throws InterruptedException
+  {
+    checkTotalCount(totalCount);
+    long currentCount = sync.getCount();
+    while (compareCounts(totalCount, currentCount) > 0) {
+      sync.acquireSharedInterruptibly(currentCount);
+      currentCount = sync.getCount();
+    }
+  }
+
+  private static void checkTotalCount(long totalCount)
+  {
+    if (totalCount < 0) {
+      throw new AssertionError(
+          "Total count must always be >= 0, even in the face of overflow. "
+          + "The next count should always be obtained by calling ConcurrentAwaitableCounter.nextCount(prevCount), "
+          + "not just +1"
+      );
+    }
+  }
+
+  /**
+   * Await until the {@link #increment} is called on this counter object the specified number of times from the creation
+   * of this counter object, for not longer than the specified period of time. If by this time the target increment
+   * count is not reached, {@link TimeoutException} is thrown.
+   */
+  public void awaitCount(long totalCount, long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
+  {
+    checkTotalCount(totalCount);
+    long nanos = unit.toNanos(timeout);
+    long currentCount = sync.getCount();
+    while (compareCounts(totalCount, currentCount) > 0) {
+      if (!sync.tryAcquireSharedNanos(currentCount, nanos)) {
+        throw new TimeoutException();
+      }
+      currentCount = sync.getCount();
+    }
+  }
+
+  private static int compareCounts(long count1, long count2)
+  {
+    long diff = (count1 - count2) & MAX_COUNT;
+    if (diff == 0) {
+      return 0;
+    }
+    return diff < MAX_COUNT / 2 ? 1 : -1;
+  }
+
+  /**
+   * Somewhat loosely defined wait for "next N increments", because the starting point is not defined from the Java
+   * Memory Model perspective.
+   */
+  public void awaitNextIncrements(long nextIncrements) throws InterruptedException
+  {
+    if (nextIncrements <= 0) {
+      throw new IllegalArgumentException("nextIncrements is not positive: " + nextIncrements);
+    }
+    if (nextIncrements > MAX_COUNT / 4) {
+      throw new UnsupportedOperationException("Couldn't wait for so many increments: " + nextIncrements);
+    }
+    awaitCount((sync.getCount() + nextIncrements) & MAX_COUNT);
+  }
+
+  /**
+   * The difference between this method and {@link #awaitCount(long, long, TimeUnit)} with argument 1 is that {@code
+   * awaitFirstIncrement()} returns boolean designating whether the count was await (while waiting for no longer than
+   * for the specified period of time), while {@code awaitCount()} throws {@link TimeoutException} if the count was not
+   * awaited.
+   */
+  public boolean awaitFirstIncrement(long timeout, TimeUnit unit) throws InterruptedException
+  {
+    return sync.tryAcquireSharedNanos(0, unit.toNanos(timeout));
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/common/CompressionUtils.java b/java-util/src/main/java/io/druid/java/util/common/CompressionUtils.java
index 08614ff8d75..fd7bc1e6ec5 100644
--- a/java-util/src/main/java/io/druid/java/util/common/CompressionUtils.java
+++ b/java-util/src/main/java/io/druid/java/util/common/CompressionUtils.java
@@ -48,36 +48,59 @@
 {
   private static final Logger log = new Logger(CompressionUtils.class);
   private static final int DEFAULT_RETRY_COUNT = 3;
-
-  public static final String GZ_SUFFIX = ".gz";
-  public static final String ZIP_SUFFIX = ".zip";
+  private static final String GZ_SUFFIX = ".gz";
+  private static final String ZIP_SUFFIX = ".zip";
 
   /**
    * Zip the contents of directory into the file indicated by outputZipFile. Sub directories are skipped
    *
    * @param directory     The directory whose contents should be added to the zip in the output stream.
    * @param outputZipFile The output file to write the zipped data to
+   * @param fsync         True if the output file should be fsynced to disk
    *
    * @return The number of bytes (uncompressed) read from the input directory.
    *
    * @throws IOException
    */
-  public static long zip(File directory, File outputZipFile) throws IOException
+  public static long zip(File directory, File outputZipFile, boolean fsync) throws IOException
   {
     if (!isZip(outputZipFile.getName())) {
       log.warn("No .zip suffix[%s], putting files from [%s] into it anyway.", outputZipFile, directory);
     }
 
     try (final FileOutputStream out = new FileOutputStream(outputZipFile)) {
-      return zip(directory, out);
+      long bytes = zip(directory, out);
+
+      // For explanation of why fsyncing here is a good practice:
+      // https://github.com/druid-io/druid/pull/5187#pullrequestreview-85188984
+      if (fsync) {
+        out.getChannel().force(true);
+      }
+
+      return bytes;
     }
   }
 
+  /**
+   * Zip the contents of directory into the file indicated by outputZipFile. Sub directories are skipped
+   *
+   * @param directory     The directory whose contents should be added to the zip in the output stream.
+   * @param outputZipFile The output file to write the zipped data to
+   *
+   * @return The number of bytes (uncompressed) read from the input directory.
+   *
+   * @throws IOException
+   */
+  public static long zip(File directory, File outputZipFile) throws IOException
+  {
+    return zip(directory, outputZipFile, false);
+  }
+
   /**
    * Zips the contents of the input directory to the output stream. Sub directories are skipped
    *
    * @param directory The directory whose contents should be added to the zip in the output stream.
-   * @param out       The output stream to write the zip data to. It is closed in the process
+   * @param out       The output stream to write the zip data to. Caller is responsible for closing this stream.
    *
    * @return The number of bytes (uncompressed) read from the input directory.
    *
@@ -88,23 +111,23 @@ public static long zip(File directory, OutputStream out) throws IOException
     if (!directory.isDirectory()) {
       throw new IOE("directory[%s] is not a directory", directory);
     }
-    final File[] files = directory.listFiles();
+
+    final ZipOutputStream zipOut = new ZipOutputStream(out);
 
     long totalSize = 0;
-    try (final ZipOutputStream zipOut = new ZipOutputStream(out)) {
-      for (File file : files) {
-        log.info("Adding file[%s] with size[%,d].  Total size so far[%,d]", file, file.length(), totalSize);
-        if (file.length() >= Integer.MAX_VALUE) {
-          zipOut.finish();
-          throw new IOE("file[%s] too large [%,d]", file, file.length());
-        }
-        zipOut.putNextEntry(new ZipEntry(file.getName()));
-        totalSize += Files.asByteSource(file).copyTo(zipOut);
+    for (File file : directory.listFiles()) {
+      log.info("Adding file[%s] with size[%,d].  Total size so far[%,d]", file, file.length(), totalSize);
+      if (file.length() >= Integer.MAX_VALUE) {
+        zipOut.finish();
+        throw new IOE("file[%s] too large [%,d]", file, file.length());
       }
-      zipOut.closeEntry();
-      // Workarround for http://hg.openjdk.java.net/jdk8/jdk8/jdk/rev/759aa847dcaf
-      zipOut.flush();
+      zipOut.putNextEntry(new ZipEntry(file.getName()));
+      totalSize += Files.asByteSource(file).copyTo(zipOut);
     }
+    zipOut.closeEntry();
+    // Workaround for http://hg.openjdk.java.net/jdk8/jdk8/jdk/rev/759aa847dcaf
+    zipOut.flush();
+    zipOut.finish();
 
     return totalSize;
   }
diff --git a/java-util/src/main/java/io/druid/java/util/common/FileUtils.java b/java-util/src/main/java/io/druid/java/util/common/FileUtils.java
index 72be3a57e18..0a5d0806388 100644
--- a/java-util/src/main/java/io/druid/java/util/common/FileUtils.java
+++ b/java-util/src/main/java/io/druid/java/util/common/FileUtils.java
@@ -27,11 +27,16 @@
 
 import java.io.File;
 import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FilterOutputStream;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.nio.MappedByteBuffer;
 import java.nio.channels.FileChannel;
+import java.nio.file.StandardCopyOption;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.UUID;
 
 public class FileUtils
 {
@@ -46,6 +51,7 @@ public boolean apply(Throwable input)
       return input instanceof Exception;
     }
   };
+
   /**
    * Copy input byte source to outFile. If outFile exists, it is attempted to be deleted.
    *
@@ -150,10 +156,11 @@ public void addFile(File file)
    * }}</pre>
    *
    * @param file the file to map
+   *
    * @return a {@link MappedByteBufferHandler}, wrapping a read-only buffer reflecting {@code file}
-   * @throws FileNotFoundException if the {@code file} does not exist
-   * @throws IOException if an I/O error occurs
    *
+   * @throws FileNotFoundException if the {@code file} does not exist
+   * @throws IOException           if an I/O error occurs
    * @see FileChannel#map(FileChannel.MapMode, long, long)
    */
   public static MappedByteBufferHandler map(File file) throws IOException
@@ -161,4 +168,64 @@ public static MappedByteBufferHandler map(File file) throws IOException
     MappedByteBuffer mappedByteBuffer = Files.map(file);
     return new MappedByteBufferHandler(mappedByteBuffer);
   }
+
+  /**
+   * Write to a file atomically, by first writing to a temporary file in the same directory and then moving it to
+   * the target location. This function attempts to clean up its temporary files when possible, but they may stick
+   * around (for example, if the JVM crashes partway through executing the function). In any case, the target file
+   * should be unharmed.
+   *
+   * The OutputStream passed to the consumer is uncloseable; calling close on it will do nothing. This is to ensure
+   * that the stream stays open so we can fsync it here before closing. Hopefully, this doesn't cause any problems
+   * for callers.
+   *
+   * This method is not just thread-safe, but is also safe to use from multiple processes on the same machine.
+   */
+  public static void writeAtomically(final File file, OutputStreamConsumer f) throws IOException
+  {
+    writeAtomically(file, file.getParentFile(), f);
+  }
+
+  private static void writeAtomically(final File file, final File tmpDir, OutputStreamConsumer f) throws IOException
+  {
+    final File tmpFile = new File(tmpDir, StringUtils.format(".%s.%s", file.getName(), UUID.randomUUID()));
+
+    try {
+      try (final FileOutputStream out = new FileOutputStream(tmpFile)) {
+        // Pass f an uncloseable stream so we can fsync before closing.
+        f.accept(uncloseable(out));
+
+        // fsync to avoid write-then-rename-then-crash causing empty files on some filesystems.
+        out.getChannel().force(true);
+      }
+
+      // No exception thrown; do the move.
+      java.nio.file.Files.move(
+          tmpFile.toPath(),
+          file.toPath(),
+          StandardCopyOption.ATOMIC_MOVE,
+          StandardCopyOption.REPLACE_EXISTING
+      );
+    }
+    finally {
+      tmpFile.delete();
+    }
+  }
+
+  private static OutputStream uncloseable(final OutputStream out) throws IOException
+  {
+    return new FilterOutputStream(out)
+    {
+      @Override
+      public void close() throws IOException
+      {
+        // Do nothing.
+      }
+    };
+  }
+
+  public interface OutputStreamConsumer
+  {
+    void accept(OutputStream outputStream) throws IOException;
+  }
 }
diff --git a/java-util/src/main/java/io/druid/java/util/common/StreamUtils.java b/java-util/src/main/java/io/druid/java/util/common/StreamUtils.java
index 079d6c9c810..23ff9d621ae 100644
--- a/java-util/src/main/java/io/druid/java/util/common/StreamUtils.java
+++ b/java-util/src/main/java/io/druid/java/util/common/StreamUtils.java
@@ -25,6 +25,9 @@
 import com.google.common.io.ByteSource;
 import com.google.common.io.ByteStreams;
 
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -34,6 +37,31 @@
  */
 public class StreamUtils
 {
+  /**
+   * Copy from an input stream to a file (and buffer it) and close the input stream.
+   * <p>
+   * It is highly recommended to use FileUtils.retryCopy whenever possible, and not use a raw `InputStream`
+   *
+   * @param is   The input stream to copy bytes from. `is` is closed regardless of the copy result.
+   * @param file The file to copy bytes to. Any parent directories are automatically created.
+   *
+   * @return The count of bytes written to the file
+   *
+   * @throws IOException
+   */
+  public static long copyToFileAndClose(InputStream is, File file) throws IOException
+  {
+    file.getParentFile().mkdirs();
+    try (OutputStream os = new BufferedOutputStream(new FileOutputStream(file))) {
+      final long result = ByteStreams.copy(is, os);
+      // Workarround for http://hg.openjdk.java.net/jdk8/jdk8/jdk/rev/759aa847dcaf
+      os.flush();
+      return result;
+    }
+    finally {
+      is.close();
+    }
+  }
 
   /**
    * Copy from `is` to `os` and close the streams regardless of the result.
diff --git a/java-util/src/main/java/io/druid/java/util/common/granularity/Granularities.java b/java-util/src/main/java/io/druid/java/util/common/granularity/Granularities.java
index 599110550f9..de8b6f688d2 100644
--- a/java-util/src/main/java/io/druid/java/util/common/granularity/Granularities.java
+++ b/java-util/src/main/java/io/druid/java/util/common/granularity/Granularities.java
@@ -41,4 +41,8 @@
   public static final Granularity ALL = GranularityType.ALL.getDefaultGranularity();
   public static final Granularity NONE = GranularityType.NONE.getDefaultGranularity();
 
+  public static Granularity nullToAll(Granularity granularity)
+  {
+    return granularity == null ? Granularities.ALL : granularity;
+  }
 }
diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java
index 20fab62adc2..77d711b8a98 100644
--- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java
+++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java
@@ -34,7 +34,7 @@
  * methods get called and other resources get cleaned up whenever processing is complete. Without this inversion
  * it is very easy to unintentionally leak resources when iterating over something that is backed by a resource.
  * <p>
- * Sequences also expose {#see com.metamx.common.guava.Yielder} Yielder objects which allow you to implement a
+ * Sequences also expose {#see io.druid.java.util.common.guava.Yielder} Yielder objects which allow you to implement a
  * continuation over the Sequence. Yielder do not offer the same guarantees of automatic resource management
  * as the accumulate method, but they are Closeable and will do the proper cleanup when close() is called on them.
  */
diff --git a/java-util/src/main/java/io/druid/java/util/common/lifecycle/Lifecycle.java b/java-util/src/main/java/io/druid/java/util/common/lifecycle/Lifecycle.java
index 05999be5456..eb8771adc48 100644
--- a/java-util/src/main/java/io/druid/java/util/common/lifecycle/Lifecycle.java
+++ b/java-util/src/main/java/io/druid/java/util/common/lifecycle/Lifecycle.java
@@ -403,8 +403,7 @@ public void start() throws Exception
         for (Annotation annotation : method.getAnnotations()) {
           if (annotation.annotationType()
                         .getCanonicalName()
-                        .equals("io.druid.java.util.common.lifecycle.LifecycleStart") ||
-              annotation.annotationType().getCanonicalName().equals("com.metamx.common.lifecycle.LifecycleStart")) {
+                        .equals("io.druid.java.util.common.lifecycle.LifecycleStart")) {
             doStart = true;
             break;
           }
@@ -424,8 +423,7 @@ public void stop()
         for (Annotation annotation : method.getAnnotations()) {
           if (annotation.annotationType()
                         .getCanonicalName()
-                        .equals("io.druid.java.util.common.lifecycle.LifecycleStop") ||
-              annotation.annotationType().getCanonicalName().equals("com.metamx.common.lifecycle.LifecycleStop")) {
+                        .equals("io.druid.java.util.common.lifecycle.LifecycleStop")) {
             doStop = true;
             break;
           }
diff --git a/java-util/src/main/java/io/druid/java/util/common/logger/Logger.java b/java-util/src/main/java/io/druid/java/util/common/logger/Logger.java
index 656de5f97c0..0d8189a0a08 100644
--- a/java-util/src/main/java/io/druid/java/util/common/logger/Logger.java
+++ b/java-util/src/main/java/io/druid/java/util/common/logger/Logger.java
@@ -38,6 +38,12 @@ public Logger(Class clazz)
     log = LoggerFactory.getLogger(clazz);
   }
 
+  @Override
+  public String toString()
+  {
+    return StringUtils.format("Logger{name=[%s], class[%s]}", log.getName(), log.getClass());
+  }
+
   public void trace(String message, Object... formatArgs)
   {
     if (log.isTraceEnabled()) {
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/EmittingLogger.java b/java-util/src/main/java/io/druid/java/util/emitter/EmittingLogger.java
new file mode 100644
index 00000000000..55c679d4991
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/EmittingLogger.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicates;
+import com.google.common.collect.Maps;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.emitter.service.AlertBuilder;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+
+/**
+ */
+public class EmittingLogger extends Logger
+{
+  private static volatile ServiceEmitter emitter = null;
+
+  private final String className;
+
+  public static void registerEmitter(ServiceEmitter emitter)
+  {
+    Preconditions.checkNotNull(emitter);
+    EmittingLogger.emitter = emitter;
+  }
+
+  public EmittingLogger(String className)
+  {
+    super(className);
+
+    this.className = className;
+  }
+
+  public EmittingLogger(Class clazz)
+  {
+    super(clazz);
+
+    this.className = clazz.getName();
+  }
+
+  public AlertBuilder makeAlert(String message, Object... objects)
+  {
+    return makeAlert(null, message, objects);
+  }
+
+  public AlertBuilder makeAlert(Throwable t, String message, Object... objects)
+  {
+    if (emitter == null) {
+      final String errorMessage = StringUtils.format(
+          "Emitter not initialized!  Cannot alert.  Please make sure to call %s.registerEmitter()", this.getClass()
+      );
+
+      error(errorMessage);
+      throw new ISE(errorMessage);
+    }
+
+    final AlertBuilder retVal = new EmittingAlertBuilder(t, StringUtils.format(message, objects), emitter)
+        .addData("class", className);
+
+    if (t != null) {
+      final StringWriter trace = new StringWriter();
+      final PrintWriter pw = new PrintWriter(trace);
+      t.printStackTrace(pw);
+      retVal.addData("exceptionType", t.getClass());
+      retVal.addData("exceptionMessage", t.getMessage());
+      retVal.addData("exceptionStackTrace", trace.toString());
+    }
+
+    return retVal;
+  }
+
+  public class EmittingAlertBuilder extends AlertBuilder
+  {
+    private final Throwable t;
+
+    private volatile boolean emitted = false;
+
+    private EmittingAlertBuilder(Throwable t, String description, ServiceEmitter emitter)
+    {
+      super(description, emitter);
+      this.t = t;
+    }
+
+    @Override
+    public void emit()
+    {
+      logIt("%s: %s");
+
+      emitted = true;
+
+      super.emit();
+    }
+
+    @Override
+    protected void finalize() throws Throwable
+    {
+      if (!emitted) {
+        logIt("Alert not emitted, emitting. %s: %s");
+        super.emit();
+      }
+    }
+
+    private void logIt(String format)
+    {
+      if (t == null) {
+        error(format, description, dataMap);
+      } else {
+        // Filter out the stack trace from the message, because it should be in the logline already if it's wanted.
+        error(
+            t,
+            format,
+            description,
+            Maps.filterKeys(dataMap, Predicates.not(Predicates.equalTo("exceptionStackTrace")))
+        );
+      }
+    }
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/BaseHttpEmittingConfig.java b/java-util/src/main/java/io/druid/java/util/emitter/core/BaseHttpEmittingConfig.java
new file mode 100644
index 00000000000..9838bfb1903
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/BaseHttpEmittingConfig.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.druid.java.util.common.Pair;
+
+import javax.validation.constraints.Min;
+
+public class BaseHttpEmittingConfig
+{
+  public static final long DEFAULT_FLUSH_MILLIS = 60 * 1000;
+  public static final int DEFAULT_FLUSH_COUNTS = 500;
+
+  /** ensure the event buffers don't use more than 10% of memory by default */
+  public static final int DEFAULT_MAX_BATCH_SIZE;
+  public static final int DEFAULT_BATCH_QUEUE_SIZE_LIMIT;
+  static {
+    Pair<Integer, Integer> batchConfigPair = getDefaultBatchSizeAndLimit(Runtime.getRuntime().maxMemory());
+    DEFAULT_MAX_BATCH_SIZE = batchConfigPair.lhs;
+    DEFAULT_BATCH_QUEUE_SIZE_LIMIT = batchConfigPair.rhs;
+  }
+
+  /**
+   * Do not time out in case flushTimeOut is not set
+   */
+  public static final long DEFAULT_FLUSH_TIME_OUT = Long.MAX_VALUE;
+  public static final String DEFAULT_BASIC_AUTHENTICATION = null;
+  public static final BatchingStrategy DEFAULT_BATCHING_STRATEGY = BatchingStrategy.ARRAY;
+  public static final ContentEncoding DEFAULT_CONTENT_ENCODING = null;
+  public static final float DEFAULT_HTTP_TIMEOUT_ALLOWANCE_FACTOR = 2.0f;
+  /**
+   * The default value effective doesn't set the min timeout
+   */
+  public static final int DEFAULT_MIN_HTTP_TIMEOUT_MILLIS = 0;
+
+  public static Pair<Integer, Integer> getDefaultBatchSizeAndLimit(long maxMemory)
+  {
+    long memoryLimit = maxMemory / 10;
+    long batchSize = 5 * 1024 * 1024;
+    long queueLimit = 50;
+
+    if (batchSize * queueLimit > memoryLimit) {
+      queueLimit = memoryLimit / batchSize;
+    }
+
+    // make room for at least two queue items
+    if (queueLimit < 2) {
+      queueLimit = 2;
+      batchSize = memoryLimit / queueLimit;
+    }
+
+    return new Pair<>((int) batchSize, (int) queueLimit);
+  }
+
+  @Min(1)
+  @JsonProperty
+  long flushMillis = DEFAULT_FLUSH_MILLIS;
+
+  @Min(0)
+  @JsonProperty
+  int flushCount = DEFAULT_FLUSH_COUNTS;
+
+  @Min(0)
+  @JsonProperty
+  long flushTimeOut = DEFAULT_FLUSH_TIME_OUT;
+
+  @JsonProperty
+  String basicAuthentication = DEFAULT_BASIC_AUTHENTICATION;
+
+  @JsonProperty
+  BatchingStrategy batchingStrategy = DEFAULT_BATCHING_STRATEGY;
+
+  @Min(0)
+  @JsonProperty
+  int maxBatchSize = DEFAULT_MAX_BATCH_SIZE;
+
+  @JsonProperty
+  ContentEncoding contentEncoding = DEFAULT_CONTENT_ENCODING;
+
+  @Min(0)
+  @JsonProperty
+  int batchQueueSizeLimit = DEFAULT_BATCH_QUEUE_SIZE_LIMIT;
+
+  @Min(1)
+  @JsonProperty
+  float httpTimeoutAllowanceFactor = DEFAULT_HTTP_TIMEOUT_ALLOWANCE_FACTOR;
+
+  @Min(0)
+  @JsonProperty
+  int minHttpTimeoutMillis = DEFAULT_MIN_HTTP_TIMEOUT_MILLIS;
+
+  public long getFlushMillis()
+  {
+    return flushMillis;
+  }
+
+  public int getFlushCount()
+  {
+    return flushCount;
+  }
+
+  public long getFlushTimeOut()
+  {
+    return flushTimeOut;
+  }
+
+  public String getBasicAuthentication()
+  {
+    return basicAuthentication;
+  }
+
+  public BatchingStrategy getBatchingStrategy()
+  {
+    return batchingStrategy;
+  }
+
+  public int getMaxBatchSize()
+  {
+    return maxBatchSize;
+  }
+
+  public ContentEncoding getContentEncoding()
+  {
+    return contentEncoding;
+  }
+
+  public int getBatchQueueSizeLimit()
+  {
+    return batchQueueSizeLimit;
+  }
+
+  public float getHttpTimeoutAllowanceFactor()
+  {
+    return httpTimeoutAllowanceFactor;
+  }
+
+  public int getMinHttpTimeoutMillis()
+  {
+    return minHttpTimeoutMillis;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "BaseHttpEmittingConfig{" + toStringBase() + '}';
+  }
+
+  protected String toStringBase()
+  {
+    return
+        "flushMillis=" + flushMillis +
+        ", flushCount=" + flushCount +
+        ", flushTimeOut=" + flushTimeOut +
+        ", basicAuthentication='" + basicAuthentication + '\'' +
+        ", batchingStrategy=" + batchingStrategy +
+        ", maxBatchSize=" + maxBatchSize +
+        ", contentEncoding=" + contentEncoding +
+        ", batchQueueSizeLimit=" + batchQueueSizeLimit +
+        ", httpTimeoutAllowanceFactor=" + httpTimeoutAllowanceFactor +
+        ", minHttpTimeoutMillis=" + minHttpTimeoutMillis;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/Batch.java b/java-util/src/main/java/io/druid/java/util/emitter/core/Batch.java
new file mode 100644
index 00000000000..cb0bc88042b
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/Batch.java
@@ -0,0 +1,359 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import com.google.common.base.Preconditions;
+import io.druid.java.util.common.logger.Logger;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.AbstractQueuedLongSynchronizer;
+
+/**
+ * Buffer for batched data + synchronization state.
+ * <p>
+ * The state structure ({@link AbstractQueuedLongSynchronizer#state}):
+ * Bits 0-30 - bufferWatermark
+ * Bit 31 - always 0
+ * Bits 32-62 - "parties" (the number of concurrent writers)
+ * Bit 63 - sealed flag
+ * <p>
+ * Writer threads (callers of {@link HttpPostEmitter#emit(Event)}) are eligible to come, increment bufferWatermark and
+ * write data into the buffer, as long as sealed flag is false.
+ * <p>
+ * {@link HttpPostEmitter#emittingThread} is eligible to emit the buffer, when sealed flag=true and parties=0 (all
+ * writes are completed). See {@link #isEmittingAllowed(long)}.
+ * <p>
+ * In this class, "lock" means "increment number of parties by 1", i. e. lock the emitter thread from emitting this
+ * batch. "Unlock" means "decrement number of parties by 1".
+ */
+class Batch extends AbstractQueuedLongSynchronizer
+{
+  private static final Logger log = new Logger(Batch.class);
+
+  private static final long PARTY = 1L << 32;
+  private static final long SEAL_BIT = 1L << 63;
+
+  private static int bufferWatermark(long state)
+  {
+    return (int) state;
+  }
+
+  private static int parties(long state)
+  {
+    return ((int) (state >>> 32)) & Integer.MAX_VALUE;
+  }
+
+  private static boolean isSealed(long state)
+  {
+    // The highest bit is 1.
+    return state < 0;
+  }
+
+  private static boolean isEmittingAllowed(long state)
+  {
+    return isSealed(state) && parties(state) == 0;
+  }
+
+  /**
+   * Tags (arg values) to transmit request from {@link #releaseShared(long)} to {@link #tryReleaseShared(long)}.
+   */
+  private static final long UNLOCK_TAG = 0;
+  private static final long UNLOCK_AND_SEAL_TAG = 1;
+  private static final long SEAL_TAG = 2;
+
+  /**
+   * The emitter this batch belongs to.
+   */
+  private final HttpPostEmitter emitter;
+
+  /**
+   * The data buffer of the batch.
+   */
+  final byte[] buffer;
+
+  /**
+   * Ordering number of this batch, as they filled & emitted in {@link HttpPostEmitter} serially, starting from 0.
+   * It's a boxed Long rather than primitive long, because we want to minimize the number of allocations done in
+   * {@link HttpPostEmitter#onSealExclusive} and so the probability of {@link OutOfMemoryError}.
+   * @see HttpPostEmitter#onSealExclusive
+   * @see HttpPostEmitter#concurrentBatch
+   */
+  final Long batchNumber;
+
+  /**
+   * The number of events in this batch, needed for event count-based batch emitting.
+   */
+  final AtomicInteger eventCount = new AtomicInteger(0);
+
+  /**
+   * The time when the first event was written into this batch, needed for timeout-based batch emitting.
+   */
+  private long firstEventTimestamp = -1;
+
+  Batch(HttpPostEmitter emitter, byte[] buffer, long batchNumber)
+  {
+    this.emitter = emitter;
+    this.buffer = buffer;
+    this.batchNumber = batchNumber;
+  }
+
+  int getSealedBufferWatermark()
+  {
+    long state = getState();
+    Preconditions.checkState(isSealed(state));
+    return bufferWatermark(state);
+  }
+
+  /**
+   * Tries to add (write) event to the batch, returns true, if successful. If fails, no subsequent attempts to add event
+   * to this batch will succeed, the next batch should be taken.
+   */
+  boolean tryAddEvent(byte[] event)
+  {
+    while (true) {
+      long state = getState();
+      if (isSealed(state)) {
+        return false;
+      }
+      int bufferWatermark = bufferWatermark(state);
+      if (bufferWatermark == 0) {
+        if (tryAddFirstEvent(event)) {
+          return true;
+        }
+      } else if (newBufferWatermark(bufferWatermark, event) <= emitter.maxBufferWatermark) {
+        if (tryAddNonFirstEvent(state, event)) {
+          return true;
+        }
+      } else {
+        seal();
+        return false;
+      }
+    }
+  }
+
+  private boolean tryAddFirstEvent(byte[] event)
+  {
+    if (!tryReserveFirstEventSizeAndLock(event)) {
+      return false;
+    }
+    try {
+      int bufferOffset = emitter.batchingStrategy.writeBatchStart(buffer);
+      writeEvent(event, bufferOffset);
+      eventCount.incrementAndGet();
+      firstEventTimestamp = System.currentTimeMillis();
+      return true;
+    }
+    finally {
+      unlock();
+    }
+  }
+
+  private boolean tryReserveFirstEventSizeAndLock(byte[] event)
+  {
+    return compareAndSetState(0, emitter.batchingStrategy.batchStartLength() + event.length + PARTY);
+  }
+
+  private int newBufferWatermark(int bufferWatermark, byte[] eventBytes)
+  {
+    return bufferWatermark + emitter.batchingStrategy.separatorLength() + eventBytes.length;
+  }
+
+  private boolean tryAddNonFirstEvent(long state, byte[] event)
+  {
+    int bufferOffset = tryReserveEventSizeAndLock(state, emitter.batchingStrategy.separatorLength() + event.length);
+    if (bufferOffset < 0) {
+      return false;
+    }
+    try {
+      bufferOffset = emitter.batchingStrategy.writeMessageSeparator(buffer, bufferOffset);
+      writeEvent(event, bufferOffset);
+      return true;
+    }
+    finally {
+      unlockAndSealIfNeeded();
+    }
+  }
+
+  /**
+   * Returns the buffer offset at which the caller has reserved the ability to write `size` bytes exclusively,
+   * or negative number, if the reservation attempt failed.
+   */
+  private int tryReserveEventSizeAndLock(long state, int size)
+  {
+    Preconditions.checkArgument(size > 0);
+    int bufferWatermark = bufferWatermark(state);
+    while (true) {
+      if (compareAndSetState(state, state + size + PARTY)) {
+        return bufferWatermark;
+      }
+      state = getState();
+      if (isSealed(state)) {
+        return -1;
+      }
+      bufferWatermark = bufferWatermark(state);
+      int newBufferWatermark = bufferWatermark + size;
+      Preconditions.checkState(newBufferWatermark > 0);
+      if (newBufferWatermark > emitter.maxBufferWatermark) {
+        return -1;
+      }
+    }
+  }
+
+  private void unlockAndSealIfNeeded()
+  {
+    if (eventCount.incrementAndGet() >= emitter.config.getFlushCount()) {
+      unlockAndSeal();
+    } else {
+      long timeSinceFirstEvent = System.currentTimeMillis() - firstEventTimestamp;
+      if (firstEventTimestamp > 0 && timeSinceFirstEvent > emitter.config.getFlushMillis()) {
+        unlockAndSeal();
+      } else {
+        unlock();
+      }
+    }
+  }
+
+  void sealIfFlushNeeded()
+  {
+    long timeSinceFirstEvent = System.currentTimeMillis() - firstEventTimestamp;
+    if (firstEventTimestamp > 0 && timeSinceFirstEvent > emitter.config.getFlushMillis()) {
+      seal();
+    }
+  }
+
+  private void writeEvent(byte[] event, int bufferOffset)
+  {
+    System.arraycopy(event, 0, buffer, bufferOffset, event.length);
+  }
+
+
+  private void unlock()
+  {
+    releaseShared(UNLOCK_TAG);
+  }
+
+  private void unlockAndSeal()
+  {
+    releaseShared(UNLOCK_AND_SEAL_TAG);
+  }
+
+  void seal()
+  {
+    releaseShared(SEAL_TAG);
+  }
+
+  @Override
+  protected boolean tryReleaseShared(long tag)
+  {
+    if (tag == UNLOCK_TAG) {
+      while (true) {
+        long state = getState();
+        int parties = parties(state);
+        if (parties == 0) {
+          throw new IllegalMonitorStateException();
+        }
+        long newState = state - PARTY;
+        if (compareAndSetState(state, newState)) {
+          return isEmittingAllowed(newState);
+        }
+      }
+    } else if (tag == UNLOCK_AND_SEAL_TAG) {
+      while (true) {
+        long state = getState();
+        int parties = parties(state);
+        if (parties == 0) {
+          throw new IllegalMonitorStateException();
+        }
+        long newState = (state - PARTY) | SEAL_BIT;
+        if (compareAndSetState(state, newState)) {
+          // Ensures only one thread calls emitter.onSealExclusive() for each batch.
+          if (!isSealed(state)) {
+            log.debug("Unlocked and sealed batch [%d]", batchNumber);
+            debugLogState("old state", state);
+            debugLogState("new state", newState);
+            emitter.onSealExclusive(
+                this,
+                firstEventTimestamp > 0 ? System.currentTimeMillis() - firstEventTimestamp : -1
+            );
+          }
+          return isEmittingAllowed(newState);
+        }
+      }
+    } else if (tag == SEAL_TAG) {
+      while (true) {
+        long state = getState();
+        if (isSealed(state)) {
+          // Returning false, despite acquisition could be possible now, because this thread actually didn't update the
+          // state, i. e. didn't "release" in AbstractQueuedLongSynchronizer's terms.
+          return false;
+        }
+        long newState = state | SEAL_BIT;
+        if (compareAndSetState(state, newState)) {
+          log.debug("Sealed batch [%d]", batchNumber);
+          debugLogState("old state", state);
+          debugLogState("new state", newState);
+          emitter.onSealExclusive(
+              this,
+              firstEventTimestamp > 0 ? System.currentTimeMillis() - firstEventTimestamp : -1
+          );
+          return isEmittingAllowed(newState);
+        }
+      }
+    } else {
+      throw new IllegalStateException("Unknown tag: " + tag);
+    }
+  }
+
+  void awaitEmittingAllowed()
+  {
+    acquireShared(1);
+  }
+
+  @Override
+  protected long tryAcquireShared(long ignored)
+  {
+    return isEmittingAllowed(getState()) ? 1 : -1;
+  }
+
+  @Override
+  public String toString()
+  {
+    long state = getState();
+    return "Batch{" +
+           "batchNumber=" + batchNumber +
+           ", bufferWatermark=" + bufferWatermark(state) +
+           ", parties=" + parties(state) +
+           ", isSealed=" + isSealed(state) +
+           "}";
+  }
+
+  private static void debugLogState(String name, long state)
+  {
+    if (log.isDebugEnabled()) {
+      log.debug(
+          "%s[bufferWatermark=%d, parties=%d, isSealed=%s]",
+          name,
+          bufferWatermark(state),
+          parties(state),
+          isSealed(state)
+      );
+    }
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/BatchingStrategy.java b/java-util/src/main/java/io/druid/java/util/emitter/core/BatchingStrategy.java
new file mode 100644
index 00000000000..b94f894b42c
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/BatchingStrategy.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+public enum BatchingStrategy
+{
+  ARRAY {
+
+    @Override
+    public int batchStartLength()
+    {
+      return 1;
+    }
+
+    @Override
+    public int separatorLength()
+    {
+      return 1;
+    }
+
+    @Override
+    public int batchEndLength()
+    {
+      return 2;
+    }
+
+    @Override
+    public int writeBatchStart(byte[] buffer)
+    {
+      buffer[0] = '[';
+      return batchStartLength();
+    }
+
+    @Override
+    public int writeMessageSeparator(byte[] buffer, int bufferOffset)
+    {
+      buffer[bufferOffset] = ',';
+      return bufferOffset + separatorLength();
+    }
+
+    @Override
+    public int writeBatchEnd(byte[] buffer, int bufferOffset)
+    {
+      buffer[bufferOffset] = ']';
+      buffer[bufferOffset + 1] = '\n';
+      return bufferOffset + batchEndLength();
+    }
+  },
+  NEWLINES {
+
+    @Override
+    public int batchStartLength()
+    {
+      return 0;
+    }
+
+    @Override
+    public int separatorLength()
+    {
+      return 1;
+    }
+
+    @Override
+    public int batchEndLength()
+    {
+      return 1;
+    }
+
+    @Override
+    public int writeBatchStart(byte[] buffer)
+    {
+      // Write nothing
+      return batchStartLength();
+    }
+
+    @Override
+    public int writeMessageSeparator(byte[] buffer, int bufferOffset)
+    {
+      buffer[bufferOffset] = '\n';
+      return bufferOffset + separatorLength();
+    }
+
+    @Override
+    public int writeBatchEnd(byte[] buffer, int bufferOffset)
+    {
+      return writeMessageSeparator(buffer, bufferOffset);
+    }
+  },
+  ONLY_EVENTS {
+    @Override
+    public int batchStartLength()
+    {
+      return 0;
+    }
+
+    @Override
+    public int separatorLength()
+    {
+      return 0;
+    }
+
+    @Override
+    public int batchEndLength()
+    {
+      return 0;
+    }
+
+    @Override
+    public int writeBatchStart(byte[] buffer)
+    {
+      return 0;
+    }
+
+    @Override
+    public int writeMessageSeparator(byte[] buffer, int bufferOffset)
+    {
+      return bufferOffset;
+    }
+
+    @Override
+    public int writeBatchEnd(byte[] buffer, int bufferOffset)
+    {
+      return bufferOffset;
+    }
+  };
+
+  public abstract int batchStartLength();
+
+  public abstract int separatorLength();
+
+  public abstract int batchEndLength();
+
+  public abstract int writeBatchStart(byte[] buffer);
+
+  public abstract int writeMessageSeparator(byte[] buffer, int bufferOffset);
+
+  public abstract int writeBatchEnd(byte[] buffer, int bufferOffset);
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/ComposingEmitter.java b/java-util/src/main/java/io/druid/java/util/emitter/core/ComposingEmitter.java
new file mode 100644
index 00000000000..b2d5b9fc8af
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/ComposingEmitter.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import com.google.common.base.Preconditions;
+import io.druid.java.util.common.lifecycle.LifecycleStart;
+import io.druid.java.util.common.lifecycle.LifecycleStop;
+import io.druid.java.util.common.logger.Logger;
+
+import java.io.IOException;
+import java.util.List;
+
+public class ComposingEmitter implements Emitter
+{
+  private static Logger log = new Logger(ComposingEmitter.class);
+
+  private final List<Emitter> emitters;
+
+  public ComposingEmitter(List<Emitter> emitters)
+  {
+    this.emitters = Preconditions.checkNotNull(emitters, "null emitters");
+  }
+
+  @Override
+  @LifecycleStart
+  public void start()
+  {
+    log.info("Starting Composing Emitter.");
+
+    for (Emitter e : emitters) {
+      log.info("Starting emitter %s.", e.getClass().getName());
+      e.start();
+    }
+  }
+
+  @Override
+  public void emit(Event event)
+  {
+    for (Emitter e : emitters) {
+      e.emit(event);
+    }
+  }
+
+  @Override
+  public void flush() throws IOException
+  {
+    boolean fail = false;
+    log.info("Flushing Composing Emitter.");
+
+    for (Emitter e : emitters) {
+      try {
+        log.info("Flushing emitter %s.", e.getClass().getName());
+        e.flush();
+      }
+      catch (IOException ex) {
+        log.error(ex, "Failed to flush emitter [%s]", e.getClass().getName());
+        fail = true;
+      }
+    }
+
+    if (fail) {
+      throw new IOException("failed to flush one or more emitters");
+    }
+  }
+
+  @Override
+  @LifecycleStop
+  public void close() throws IOException
+  {
+    boolean fail = false;
+    log.info("Closing Composing Emitter.");
+
+    for (Emitter e : emitters) {
+      try {
+        log.info("Closing emitter %s.", e.getClass().getName());
+        e.close();
+      }
+      catch (IOException ex) {
+        log.error(ex, "Failed to close emitter [%s]", e.getClass().getName());
+        fail = true;
+      }
+    }
+
+    if (fail) {
+      throw new IOException("failed to close one or more emitters");
+    }
+  }
+
+  @Override
+  public String toString()
+  {
+    return "ComposingEmitter{" +
+           "emitters=" + emitters +
+           '}';
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/ConcurrentTimeCounter.java b/java-util/src/main/java/io/druid/java/util/emitter/core/ConcurrentTimeCounter.java
new file mode 100644
index 00000000000..4b836789f7a
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/ConcurrentTimeCounter.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import com.google.common.primitives.UnsignedInts;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A class to accumulate simple stats of some time points. All methods are safe to use from multiple threads.
+ */
+public class ConcurrentTimeCounter
+{
+  /** Lower 32 bits for sum of {@link #add}ed times, higher 32 bits for the count */
+  private final AtomicLong timeSumAndCount = new AtomicLong(0L);
+  /** Lower 32 bits for the max {@link #add}ed time, 63th bit for indication if any value is added. */
+  private final AtomicLong max = new AtomicLong(-1);
+  /** Similar to {@link #max} */
+  private final AtomicLong min = new AtomicLong(-1);
+
+  public void add(int time)
+  {
+    long x = (1L << 32) | time;
+    timeSumAndCount.addAndGet(x);
+    updateMax(time);
+    updateMin(time);
+  }
+
+  private void updateMax(int time)
+  {
+    long max;
+    do {
+      max = this.max.get();
+      if (max >= 0 && ((int) max) >= time) {
+        return;
+      }
+    } while (!this.max.compareAndSet(max, UnsignedInts.toLong(time)));
+  }
+
+  private void updateMin(int time)
+  {
+    long min;
+    do {
+      min = this.min.get();
+      if (min >= 0 && ((int) min) <= time) {
+        return;
+      }
+    } while (!this.min.compareAndSet(min, UnsignedInts.toLong(time)));
+  }
+
+  public long getTimeSumAndCountAndReset()
+  {
+    return timeSumAndCount.getAndSet(0L);
+  }
+
+  public int getAndResetMaxTime()
+  {
+    long max = this.max.getAndSet(-1);
+    // If max < 0, means no times added yet, then return 0
+    return max >= 0 ? (int) max : 0;
+  }
+
+  public int getAndResetMinTime()
+  {
+    long min = this.min.getAndSet(-1);
+    // If min < 0, means no times added yet, then return 0
+    return min >= 0 ? (int) min : 0;
+  }
+
+  public static int timeSum(long timeSumAndCount)
+  {
+    return (int) timeSumAndCount;
+  }
+
+  public static int count(long timeSumAndCount)
+  {
+    return (int) (timeSumAndCount >> 32);
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/ContentEncoding.java b/java-util/src/main/java/io/druid/java/util/emitter/core/ContentEncoding.java
new file mode 100644
index 00000000000..ad45c24de2b
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/ContentEncoding.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+public enum ContentEncoding
+{
+  GZIP
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/Emitter.java b/java-util/src/main/java/io/druid/java/util/emitter/core/Emitter.java
new file mode 100644
index 00000000000..a3fdf88b92e
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/Emitter.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import java.io.Closeable;
+import java.io.Flushable;
+import java.io.IOException;
+
+/**
+ */
+public interface Emitter extends Closeable, Flushable
+{
+  void start();
+  void emit(Event event);
+
+  @Override
+  void flush() throws IOException;
+
+  @Override
+  void close() throws IOException;
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/Emitters.java b/java-util/src/main/java/io/druid/java/util/emitter/core/Emitters.java
new file mode 100644
index 00000000000..537233c9c4f
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/Emitters.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Maps;
+import io.druid.java.util.common.IAE;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.lifecycle.Lifecycle;
+import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.emitter.factory.EmitterFactory;
+import org.asynchttpclient.AsyncHttpClient;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+public class Emitters
+{
+  private static final Logger log = new Logger(Emitters.class);
+
+  private static final String LOG_EMITTER_PROP = "io.druid.java.util.emitter.logging";
+  private static final String HTTP_EMITTER_PROP = "io.druid.java.util.emitter.http";
+  private static final String CUSTOM_EMITTER_TYPE_PROP = "io.druid.java.util.emitter.type";
+
+  public static Emitter create(Properties props, AsyncHttpClient httpClient, Lifecycle lifecycle)
+  {
+    return create(props, httpClient, new ObjectMapper(), lifecycle);
+  }
+
+  public static Emitter create(
+      Properties props,
+      AsyncHttpClient httpClient,
+      ObjectMapper jsonMapper,
+      Lifecycle lifecycle
+  )
+  {
+    Map<String, Object> jsonified = Maps.newHashMap();
+    if (props.getProperty(LOG_EMITTER_PROP) != null) {
+      jsonified = makeLoggingMap(props);
+      jsonified.put("type", "logging");
+    } else if (props.getProperty(HTTP_EMITTER_PROP) != null) {
+      jsonified = makeHttpMap(props);
+      jsonified.put("type", "http");
+    } else if (props.getProperty(CUSTOM_EMITTER_TYPE_PROP) != null) {
+      jsonified = makeCustomFactoryMap(props);
+    } else {
+      throw new ISE(
+          "Unknown type of emitter. Please set [%s], [%s] or provide registered subtype of io.druid.java.util.emitter.core.factory.EmitterFactory via [%s]",
+          LOG_EMITTER_PROP,
+          HTTP_EMITTER_PROP,
+          CUSTOM_EMITTER_TYPE_PROP
+      );
+    }
+    return jsonMapper.convertValue(jsonified, EmitterFactory.class).makeEmitter(jsonMapper, httpClient, lifecycle);
+  }
+
+  // Package-visible for unit tests
+
+  static Map<String, Object> makeHttpMap(Properties props)
+  {
+    Map<String, Object> httpMap = Maps.newHashMap();
+
+    final String urlProperty = "io.druid.java.util.emitter.http.url";
+
+    final String baseUrl = props.getProperty(urlProperty);
+    if (baseUrl == null) {
+      throw new IAE("Property[%s] must be set", urlProperty);
+    }
+
+    httpMap.put("recipientBaseUrl", baseUrl);
+    httpMap.put("flushMillis", Long.parseLong(props.getProperty("io.druid.java.util.emitter.flushMillis", "60000")));
+    httpMap.put("flushCount", Integer.parseInt(props.getProperty("io.druid.java.util.emitter.flushCount", "300")));
+    /**
+     * The defaultValue for "io.druid.java.util.emitter.http.flushTimeOut" must be same as {@link HttpEmitterConfig.DEFAULT_FLUSH_TIME_OUT}
+     * */
+    httpMap.put(
+        "flushTimeOut",
+        Long.parseLong(props.getProperty(
+            "io.druid.java.util.emitter.http.flushTimeOut",
+            String.valueOf(Long.MAX_VALUE)
+        ))
+    );
+    if (props.containsKey("io.druid.java.util.emitter.http.basicAuthentication")) {
+      httpMap.put("basicAuthentication", props.getProperty("io.druid.java.util.emitter.http.basicAuthentication"));
+    }
+    if (props.containsKey("io.druid.java.util.emitter.http.batchingStrategy")) {
+      httpMap.put(
+          "batchingStrategy",
+          StringUtils.toUpperCase(props.getProperty("io.druid.java.util.emitter.http.batchingStrategy"))
+      );
+    }
+    if (props.containsKey("io.druid.java.util.emitter.http.maxBatchSize")) {
+      httpMap.put("maxBatchSize", Integer.parseInt(props.getProperty("io.druid.java.util.emitter.http.maxBatchSize")));
+    }
+    if (props.containsKey("io.druid.java.util.emitter.http.batchQueueSizeLimit")) {
+      httpMap.put(
+          "batchQueueSizeLimit",
+          Integer.parseInt(props.getProperty("io.druid.java.util.emitter.http.batchQueueSizeLimit"))
+      );
+    }
+    if (props.containsKey("io.druid.java.util.emitter.http.httpTimeoutAllowanceFactor")) {
+      httpMap.put(
+          "httpTimeoutAllowanceFactor",
+          Float.parseFloat(props.getProperty("io.druid.java.util.emitter.http.httpTimeoutAllowanceFactor"))
+      );
+    }
+    if (props.containsKey("io.druid.java.util.emitter.http.minHttpTimeoutMillis")) {
+      httpMap.put(
+          "minHttpTimeoutMillis",
+          Float.parseFloat(props.getProperty("io.druid.java.util.emitter.http.minHttpTimeoutMillis"))
+      );
+    }
+    return httpMap;
+  }
+
+  // Package-visible for unit tests
+  static Map<String, Object> makeLoggingMap(Properties props)
+  {
+    Map<String, Object> loggingMap = Maps.newHashMap();
+
+    loggingMap.put(
+        "loggerClass", props.getProperty("io.druid.java.util.emitter.logging.class", LoggingEmitter.class.getName())
+    );
+    loggingMap.put(
+        "logLevel", props.getProperty("io.druid.java.util.emitter.logging.level", "debug")
+    );
+    return loggingMap;
+  }
+
+  static Map<String, Object> makeCustomFactoryMap(Properties props)
+  {
+    Map<String, Object> factoryMap = Maps.newHashMap();
+    String prefix = "io.druid.java.util.emitter.";
+
+    for (Map.Entry<Object, Object> entry : props.entrySet()) {
+      String key = entry.getKey().toString();
+      if (key.startsWith(prefix)) {
+        String combinedKey = key.substring(prefix.length());
+        Map<String, Object> currentLevelJson = factoryMap;
+        String currentKey = null;
+        String[] keyPath = combinedKey.split("\\.");
+
+        for (int i = 0; i < keyPath.length - 1; i++) {
+          String keyPart = keyPath[i];
+          Object nextLevelJson = currentLevelJson.get(keyPart);
+          if (nextLevelJson == null) {
+            nextLevelJson = new HashMap<String, Object>();
+            currentLevelJson.put(keyPart, nextLevelJson);
+          }
+          currentLevelJson = (Map<String, Object>) nextLevelJson;
+        }
+
+        currentLevelJson.put(keyPath[keyPath.length - 1], entry.getValue());
+      }
+    }
+    return factoryMap;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/Event.java b/java-util/src/main/java/io/druid/java/util/emitter/core/Event.java
new file mode 100644
index 00000000000..b9588ab40a0
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/Event.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import org.joda.time.DateTime;
+
+import java.util.Map;
+
+/**
+ */
+public interface Event
+{
+  Map<String, Object> toMap();
+
+  String getFeed();
+
+  DateTime getCreatedTime();
+
+  boolean isSafeToBuffer();
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/FeedUriExtractor.java b/java-util/src/main/java/io/druid/java/util/emitter/core/FeedUriExtractor.java
new file mode 100644
index 00000000000..dbac2aa7408
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/FeedUriExtractor.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import io.druid.java.util.common.StringUtils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+
+public class FeedUriExtractor implements UriExtractor
+{
+  private String uriPattern;
+
+  public FeedUriExtractor(String uriPattern)
+  {
+    this.uriPattern = uriPattern;
+  }
+
+  @Override
+  public URI apply(Event event) throws URISyntaxException
+  {
+    return new URI(StringUtils.format(uriPattern, event.getFeed()));
+  }
+
+  @Override
+  public String toString()
+  {
+    return "FeedUriExtractor{" +
+           "uriPattern='" + uriPattern + '\'' +
+           '}';
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/HttpEmitterConfig.java b/java-util/src/main/java/io/druid/java/util/emitter/core/HttpEmitterConfig.java
new file mode 100644
index 00000000000..1db1b840ff7
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/HttpEmitterConfig.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import javax.validation.constraints.NotNull;
+
+/**
+ */
+public class HttpEmitterConfig extends BaseHttpEmittingConfig
+{
+  @NotNull
+  @JsonProperty
+  String recipientBaseUrl = null;
+
+  /**
+   * For JSON deserialization only. In other cases use {@link Builder}
+   */
+  public HttpEmitterConfig() {}
+
+  public HttpEmitterConfig(BaseHttpEmittingConfig base, String recipientBaseUrl)
+  {
+    this.flushMillis = base.flushMillis;
+    this.flushCount = base.flushCount;
+    this.flushTimeOut = base.flushTimeOut;
+    this.recipientBaseUrl = recipientBaseUrl;
+    this.basicAuthentication = base.basicAuthentication;
+    this.batchingStrategy = base.batchingStrategy;
+    this.maxBatchSize = base.maxBatchSize;
+    this.contentEncoding = base.contentEncoding;
+    this.batchQueueSizeLimit = base.batchQueueSizeLimit;
+    this.httpTimeoutAllowanceFactor = base.httpTimeoutAllowanceFactor;
+    this.minHttpTimeoutMillis = base.minHttpTimeoutMillis;
+  }
+
+  public String getRecipientBaseUrl()
+  {
+    return recipientBaseUrl;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "HttpEmitterConfig{" +
+           toStringBase() +
+           ", recipientBaseUrl=\'" + recipientBaseUrl + '\'' +
+           '}';
+  }
+
+  public static class Builder extends HttpEmitterConfig
+  {
+    public Builder(String recipientBaseUrl)
+    {
+      this.recipientBaseUrl = recipientBaseUrl;
+    }
+
+    public Builder setFlushMillis(long flushMillis)
+    {
+      this.flushMillis = flushMillis;
+      return this;
+    }
+
+    public Builder setFlushCount(int flushCount)
+    {
+      this.flushCount = flushCount;
+      return this;
+    }
+
+    public Builder setFlushTimeOut(long flushTimeOut)
+    {
+      this.flushTimeOut = flushTimeOut;
+      return this;
+    }
+
+    public Builder setBasicAuthentication(String basicAuthentication)
+    {
+      this.basicAuthentication = basicAuthentication;
+      return this;
+    }
+
+    public Builder setBatchingStrategy(BatchingStrategy batchingStrategy)
+    {
+      this.batchingStrategy = batchingStrategy;
+      return this;
+    }
+
+    public Builder setMaxBatchSize(int maxBatchSize)
+    {
+      this.maxBatchSize = maxBatchSize;
+      return this;
+    }
+
+    public Builder setContentEncoding(ContentEncoding contentEncoding)
+    {
+      this.contentEncoding = contentEncoding;
+      return this;
+    }
+
+    public Builder setBatchQueueSizeLimit(int batchQueueSizeLimit)
+    {
+      this.batchQueueSizeLimit = batchQueueSizeLimit;
+      return this;
+    }
+
+    public Builder setHttpTimeoutAllowanceFactor(float httpTimeoutAllowanceFactor)
+    {
+      this.httpTimeoutAllowanceFactor = httpTimeoutAllowanceFactor;
+      return this;
+    }
+
+    public Builder setMinHttpTimeoutMillis(int minHttpTimeoutMillis)
+    {
+      this.minHttpTimeoutMillis = minHttpTimeoutMillis;
+      return this;
+    }
+
+    public HttpEmitterConfig build()
+    {
+      return new HttpEmitterConfig(this, recipientBaseUrl);
+    }
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/HttpPostEmitter.java b/java-util/src/main/java/io/druid/java/util/emitter/core/HttpPostEmitter.java
new file mode 100644
index 00000000000..c8ea0ab6cc6
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/HttpPostEmitter.java
@@ -0,0 +1,937 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.base.Throwables;
+import com.google.common.primitives.Ints;
+import io.druid.concurrent.ConcurrentAwaitableCounter;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.RetryUtils;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.lifecycle.LifecycleStart;
+import io.druid.java.util.common.lifecycle.LifecycleStop;
+import io.druid.java.util.common.logger.Logger;
+import io.netty.handler.codec.http.HttpHeaders;
+import org.asynchttpclient.AsyncHttpClient;
+import org.asynchttpclient.ListenableFuture;
+import org.asynchttpclient.RequestBuilder;
+import org.asynchttpclient.Response;
+
+import javax.annotation.Nullable;
+import java.io.Closeable;
+import java.io.Flushable;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayDeque;
+import java.util.Base64;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.LockSupport;
+import java.util.zip.GZIPOutputStream;
+
+public class HttpPostEmitter implements Flushable, Closeable, Emitter
+{
+  private static final int MAX_EVENT_SIZE = 1023 * 1024; // Set max size slightly less than 1M to allow for metadata
+
+  private static final int MAX_SEND_RETRIES = 3;
+
+  /**
+   * Threshold of the size of {@link #buffersToEmit} when switch from using {@link
+   * BaseHttpEmittingConfig#getHttpTimeoutAllowanceFactor()} to {@link #EQUILIBRIUM_ALLOWANCE_FACTOR}
+   */
+  private static final int EMIT_QUEUE_THRESHOLD_1 = 5;
+
+  /**
+   * Threshold of the size of {@link #buffersToEmit} when switch from using {@link #EQUILIBRIUM_ALLOWANCE_FACTOR}
+   * to {@link #TIGHT_ALLOWANCE_FACTOR}.
+   */
+  private static final int EMIT_QUEUE_THRESHOLD_2 = 10;
+
+  /**
+   * 0.9 is to give room for unexpected latency or time out not being respected rigorously.
+   */
+  private static final double EQUILIBRIUM_ALLOWANCE_FACTOR = 0.9;
+
+  private static final double TIGHT_ALLOWANCE_FACTOR = 0.5;
+
+  /**
+   * Used in {@link EmittingThread#emitLargeEvents()} to ensure fair emitting of both large events and batched events.
+   */
+  private static final byte[] LARGE_EVENTS_STOP = new byte[]{};
+
+  private static final Logger log = new Logger(HttpPostEmitter.class);
+  private static final AtomicInteger instanceCounter = new AtomicInteger();
+
+  final BatchingStrategy batchingStrategy;
+  final HttpEmitterConfig config;
+  private final int bufferSize;
+  final int maxBufferWatermark;
+  private final int largeEventThreshold;
+  private final AsyncHttpClient client;
+  private final ObjectMapper jsonMapper;
+  private final String url;
+
+  private final ConcurrentLinkedQueue<byte[]> buffersToReuse = new ConcurrentLinkedQueue<>();
+  /**
+   * "Approximate" because not exactly atomically synchronized with {@link #buffersToReuse} updates. {@link
+   * ConcurrentLinkedQueue#size()} is not used, because it's O(n).
+   */
+  private final AtomicInteger approximateBuffersToReuseCount = new AtomicInteger();
+
+  /**
+   * concurrentBatch.get() == null means the service is closed. concurrentBatch.get() is the instance of Integer,
+   * it means that some thread has failed with a serious error during {@link #onSealExclusive} (with the batch number
+   * corresponding to the Integer object) and {@link #tryRecoverCurrentBatch} needs to be called. Otherwise (i. e.
+   * normally), an instance of {@link Batch} is stored in this atomic reference.
+   */
+  private final AtomicReference<Object> concurrentBatch = new AtomicReference<>();
+
+  private final ConcurrentLinkedDeque<Batch> buffersToEmit = new ConcurrentLinkedDeque<>();
+  /**
+   * See {@link #approximateBuffersToReuseCount}
+   */
+  private final AtomicInteger approximateBuffersToEmitCount = new AtomicInteger();
+  /**
+   * See {@link #approximateBuffersToReuseCount}
+   */
+  private final AtomicLong approximateEventsToEmitCount = new AtomicLong();
+
+  private final ConcurrentLinkedQueue<byte[]> largeEventsToEmit = new ConcurrentLinkedQueue<>();
+  /**
+   * See {@link #approximateBuffersToReuseCount}
+   */
+  private final AtomicInteger approximateLargeEventsToEmitCount = new AtomicInteger();
+
+  private final ConcurrentAwaitableCounter emittedBatchCounter = new ConcurrentAwaitableCounter();
+  private final EmittingThread emittingThread;
+  private final AtomicLong totalEmittedEvents = new AtomicLong();
+  private final AtomicInteger allocatedBuffers = new AtomicInteger();
+  private final AtomicInteger droppedBuffers = new AtomicInteger();
+
+  private volatile long lastFillTimeMillis;
+  private final ConcurrentTimeCounter batchFillingTimeCounter = new ConcurrentTimeCounter();
+
+  private final Object startLock = new Object();
+  private final CountDownLatch startLatch = new CountDownLatch(1);
+  private boolean running = false;
+
+  public HttpPostEmitter(HttpEmitterConfig config, AsyncHttpClient client)
+  {
+    this(config, client, new ObjectMapper());
+  }
+
+  public HttpPostEmitter(HttpEmitterConfig config, AsyncHttpClient client, ObjectMapper jsonMapper)
+  {
+    batchingStrategy = config.getBatchingStrategy();
+    final int batchOverhead = batchingStrategy.batchStartLength() + batchingStrategy.batchEndLength();
+    Preconditions.checkArgument(
+        config.getMaxBatchSize() >= MAX_EVENT_SIZE + batchOverhead,
+        StringUtils.format(
+            "maxBatchSize must be greater than MAX_EVENT_SIZE[%,d] + overhead[%,d].",
+            MAX_EVENT_SIZE,
+            batchOverhead
+        )
+    );
+    this.config = config;
+    this.bufferSize = config.getMaxBatchSize();
+    this.maxBufferWatermark = bufferSize - batchingStrategy.batchEndLength();
+    // Chosen so that if event size < largeEventThreshold, at least 2 events could fit the standard buffer.
+    this.largeEventThreshold = (bufferSize - batchOverhead - batchingStrategy.separatorLength()) / 2;
+    this.client = client;
+    this.jsonMapper = jsonMapper;
+    try {
+      this.url = new URL(config.getRecipientBaseUrl()).toString();
+    }
+    catch (MalformedURLException e) {
+      throw new ISE(e, "Bad URL: %s", config.getRecipientBaseUrl());
+    }
+    emittingThread = new EmittingThread(config);
+    long firstBatchNumber = 1;
+    concurrentBatch.set(new Batch(this, acquireBuffer(), firstBatchNumber));
+    // lastFillTimeMillis must not be 0, minHttpTimeoutMillis could be.
+    lastFillTimeMillis = Math.max(config.minHttpTimeoutMillis, 1);
+  }
+
+  @Override
+  @LifecycleStart
+  public void start()
+  {
+    synchronized (startLock) {
+      if (!running) {
+        if (startLatch.getCount() == 0) {
+          throw new IllegalStateException("Already started.");
+        }
+        running = true;
+        startLatch.countDown();
+        emittingThread.start();
+      }
+    }
+  }
+
+  private void awaitStarted()
+  {
+    try {
+      if (!startLatch.await(1, TimeUnit.SECONDS)) {
+        throw new RejectedExecutionException("Service is not started.");
+      }
+      if (isTerminated()) {
+        throw new RejectedExecutionException("Service is closed.");
+      }
+    }
+    catch (InterruptedException e) {
+      log.debug("Interrupted waiting for start");
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    }
+  }
+
+  private boolean isTerminated()
+  {
+    return concurrentBatch.get() == null;
+  }
+
+  @Override
+  public void emit(Event event)
+  {
+    emitAndReturnBatch(event);
+  }
+
+  @VisibleForTesting
+  @Nullable
+  Batch emitAndReturnBatch(Event event)
+  {
+    awaitStarted();
+
+    final byte[] eventBytes = eventToBytes(event);
+
+    if (eventBytes.length > MAX_EVENT_SIZE) {
+      log.error(
+          "Event too large to emit (%,d > %,d): %s ...",
+          eventBytes.length,
+          MAX_EVENT_SIZE,
+          StringUtils.fromUtf8(ByteBuffer.wrap(eventBytes), 1024)
+      );
+      return null;
+    }
+
+    if (eventBytes.length > largeEventThreshold) {
+      writeLargeEvent(eventBytes);
+      return null;
+    }
+
+    while (true) {
+      Object batchObj = concurrentBatch.get();
+      if (batchObj instanceof Integer) {
+        tryRecoverCurrentBatch((Integer) batchObj);
+        continue;
+      }
+      if (batchObj == null) {
+        throw new RejectedExecutionException("Service is closed.");
+      }
+      Batch batch = (Batch) batchObj;
+      if (batch.tryAddEvent(eventBytes)) {
+        return batch;
+      } else {
+        log.debug("Failed to emit an event in batch [%s]", batch);
+      }
+      // Spin loop, until the thread calling onSealExclusive() updates the concurrentBatch. This update becomes visible
+      // eventually, because concurrentBatch.get() is a volatile read.
+    }
+  }
+
+  private byte[] eventToBytes(Event event)
+  {
+    try {
+      return jsonMapper.writeValueAsBytes(event);
+    }
+    catch (IOException e) {
+      throw Throwables.propagate(e);
+    }
+  }
+
+  private void writeLargeEvent(byte[] eventBytes)
+  {
+    // It's better to drop the oldest, not latest event, but dropping the oldest is not easy to implement, because
+    // LARGE_EVENTS_STOP could be added into the queue concurrently. So just not adding the latest event.
+    // >, not >=, because largeEventsToEmit could contain LARGE_EVENTS_STOP
+    if (approximateBuffersToEmitCount.get() > config.getBatchQueueSizeLimit()) {
+      log.error(
+          "largeEventsToEmit queue size reached the limit [%d], dropping the latest large event",
+          config.getBatchQueueSizeLimit()
+      );
+    } else {
+      largeEventsToEmit.add(eventBytes);
+      approximateBuffersToEmitCount.incrementAndGet();
+      approximateLargeEventsToEmitCount.incrementAndGet();
+      approximateEventsToEmitCount.incrementAndGet();
+    }
+    wakeUpEmittingThread();
+  }
+
+  /**
+   * Called from {@link Batch} only once for each Batch in existence.
+   */
+  void onSealExclusive(Batch batch, long elapsedTimeMillis)
+  {
+    try {
+      doOnSealExclusive(batch, elapsedTimeMillis);
+    }
+    catch (Throwable t) {
+      try {
+        if (!concurrentBatch.compareAndSet(batch, batch.batchNumber)) {
+          log.error("Unexpected failure to set currentBatch to the failed Batch.batchNumber");
+        }
+        log.error(t, "Serious error during onSealExclusive(), set currentBatch to the failed Batch.batchNumber");
+      }
+      catch (Throwable t2) {
+        t.addSuppressed(t2);
+      }
+      throw t;
+    }
+  }
+
+  private void doOnSealExclusive(Batch batch, long elapsedTimeMillis)
+  {
+    batchFillingTimeCounter.add((int) Math.max(elapsedTimeMillis, 0));
+    if (elapsedTimeMillis > 0) {
+      // If elapsedTimeMillis is 0 or negative, it's likely because System.currentTimeMillis() is not monotonic, so not
+      // accounting this time for determining batch sending timeout.
+      lastFillTimeMillis = elapsedTimeMillis;
+    }
+    addBatchToEmitQueue(batch);
+    wakeUpEmittingThread();
+    if (!isTerminated()) {
+      long nextBatchNumber = ConcurrentAwaitableCounter.nextCount(batch.batchNumber);
+      byte[] newBuffer = acquireBuffer();
+      if (!concurrentBatch.compareAndSet(batch, new Batch(this, newBuffer, nextBatchNumber))) {
+        buffersToReuse.add(newBuffer);
+        // If compareAndSet failed, the service should be closed concurrently, i. e. we expect isTerminated() = true.
+        // If we don't see this, there should be some bug in HttpPostEmitter.
+        Preconditions.checkState(isTerminated());
+      }
+    }
+  }
+
+  private void tryRecoverCurrentBatch(Integer failedBatchNumber)
+  {
+    log.info("Trying to recover currentBatch");
+    long nextBatchNumber = ConcurrentAwaitableCounter.nextCount(failedBatchNumber);
+    byte[] newBuffer = acquireBuffer();
+    if (concurrentBatch.compareAndSet(failedBatchNumber, new Batch(this, newBuffer, nextBatchNumber))) {
+      log.info("Successfully recovered currentBatch");
+    } else {
+      // It's normal, a concurrent thread could succeed to recover first.
+      buffersToReuse.add(newBuffer);
+    }
+  }
+
+  private void addBatchToEmitQueue(Batch batch)
+  {
+    limitBuffersToEmitSize();
+    buffersToEmit.addLast(batch);
+    approximateBuffersToEmitCount.incrementAndGet();
+    approximateEventsToEmitCount.addAndGet(batch.eventCount.get());
+  }
+
+  private void limitBuffersToEmitSize()
+  {
+    if (approximateBuffersToEmitCount.get() >= config.getBatchQueueSizeLimit()) {
+      Batch droppedBatch = buffersToEmit.pollFirst();
+      if (droppedBatch != null) {
+        batchFinalized();
+        approximateBuffersToEmitCount.decrementAndGet();
+        approximateEventsToEmitCount.addAndGet(-droppedBatch.eventCount.get());
+        droppedBuffers.incrementAndGet();
+        log.error(
+            "buffersToEmit queue size reached the limit [%d], dropping the oldest buffer to emit",
+            config.getBatchQueueSizeLimit()
+        );
+      }
+    }
+  }
+
+  private void batchFinalized()
+  {
+    // Notify HttpPostEmitter.flush(), that the batch is emitted, or failed, or dropped.
+    emittedBatchCounter.increment();
+  }
+
+  private Batch pollBatchFromEmitQueue()
+  {
+    Batch result = buffersToEmit.pollFirst();
+    if (result == null) {
+      return null;
+    }
+    approximateBuffersToEmitCount.decrementAndGet();
+    approximateEventsToEmitCount.addAndGet(-result.eventCount.get());
+    return result;
+  }
+
+  private void wakeUpEmittingThread()
+  {
+    LockSupport.unpark(emittingThread);
+  }
+
+  @Override
+  public void flush() throws IOException
+  {
+    awaitStarted();
+    Object batchObj = concurrentBatch.get();
+    if (batchObj instanceof Batch) {
+      flush((Batch) batchObj);
+    }
+  }
+
+  private void flush(Batch batch) throws IOException
+  {
+    if (batch == null) {
+      return;
+    }
+    batch.seal();
+    try {
+      // This check doesn't always awaits for this exact batch to be emitted, because another batch could be dropped
+      // from the queue ahead of this one, in limitBuffersToEmitSize(). But there is no better way currently to wait for
+      // the exact batch, and it's not that important.
+      emittedBatchCounter.awaitCount(batch.batchNumber, config.getFlushTimeOut(), TimeUnit.MILLISECONDS);
+    }
+    catch (TimeoutException e) {
+      String message = StringUtils.format("Timed out after [%d] millis during flushing", config.getFlushTimeOut());
+      throw new IOException(message, e);
+    }
+    catch (InterruptedException e) {
+      log.debug("Thread Interrupted");
+      Thread.currentThread().interrupt();
+      throw new IOException("Thread Interrupted while flushing", e);
+    }
+  }
+
+  @Override
+  @LifecycleStop
+  public void close() throws IOException
+  {
+    synchronized (startLock) {
+      if (running) {
+        running = false;
+        Object lastBatch = concurrentBatch.getAndSet(null);
+        if (lastBatch instanceof Batch) {
+          flush((Batch) lastBatch);
+        }
+        emittingThread.shuttingDown = true;
+        // EmittingThread is interrupted after the last batch is flushed.
+        emittingThread.interrupt();
+      }
+    }
+  }
+
+  @Override
+  public String toString()
+  {
+    return "HttpPostEmitter{" +
+           "config=" + config +
+           '}';
+  }
+
+  private class EmittingThread extends Thread
+  {
+    private final ArrayDeque<FailedBuffer> failedBuffers = new ArrayDeque<>();
+    /**
+     * "Approximate", because not exactly synchronized with {@link #failedBuffers} updates. Not using size() on
+     * {@link #failedBuffers}, because access to it is not synchronized, while approximateFailedBuffersCount is queried
+     * not within EmittingThread.
+     */
+    private final AtomicInteger approximateFailedBuffersCount = new AtomicInteger();
+
+    private final ConcurrentTimeCounter successfulSendingTimeCounter = new ConcurrentTimeCounter();
+    private final ConcurrentTimeCounter failedSendingTimeCounter = new ConcurrentTimeCounter();
+
+    /**
+     * Cache the exception. Need an exception because {@link RetryUtils} operates only via exceptions.
+     */
+    private final TimeoutException timeoutLessThanMinimumException;
+
+    private boolean shuttingDown = false;
+    private ZeroCopyByteArrayOutputStream gzipBaos;
+
+    EmittingThread(HttpEmitterConfig config)
+    {
+      super("HttpPostEmitter-" + instanceCounter.incrementAndGet());
+      setDaemon(true);
+      timeoutLessThanMinimumException = new TimeoutException(
+          "Timeout less than minimum [" + config.getMinHttpTimeoutMillis() + "] ms."
+      );
+      // To not showing and writing nonsense and misleading stack trace in logs.
+      timeoutLessThanMinimumException.setStackTrace(new StackTraceElement[]{});
+    }
+
+    @Override
+    public void run()
+    {
+      while (true) {
+        boolean needsToShutdown = needsToShutdown();
+        try {
+          emitLargeEvents();
+          emitBatches();
+          tryEmitOneFailedBuffer();
+
+          if (needsToShutdown) {
+            tryEmitAndDrainAllFailedBuffers();
+            // Make GC life easier
+            drainBuffersToReuse();
+            return;
+          }
+        }
+        catch (Throwable t) {
+          log.error(t, "Uncaught exception in EmittingThread.run()");
+        }
+        if (failedBuffers.isEmpty()) {
+          // Waiting for 1/2 of config.getFlushMillis() in order to flush events not more than 50% later than specified.
+          // If nanos=0 parkNanos() doesn't wait at all, then we don't want.
+          long waitNanos = Math.max(TimeUnit.MILLISECONDS.toNanos(config.getFlushMillis()) / 2, 1);
+          LockSupport.parkNanos(HttpPostEmitter.this, waitNanos);
+        }
+      }
+    }
+
+    private boolean needsToShutdown()
+    {
+      boolean needsToShutdown = Thread.interrupted() || shuttingDown;
+      if (needsToShutdown) {
+        Object lastBatch = concurrentBatch.getAndSet(null);
+        if (lastBatch instanceof Batch) {
+          ((Batch) lastBatch).seal();
+        }
+      } else {
+        Object batch = concurrentBatch.get();
+        if (batch instanceof Batch) {
+          ((Batch) batch).sealIfFlushNeeded();
+        } else {
+          // batch == null means that HttpPostEmitter is terminated. Batch object could also be Integer, if some
+          // thread just failed with a serious error in onSealExclusive(), in this case we don't want to shutdown
+          // the emitter thread.
+          needsToShutdown = batch == null;
+        }
+      }
+      return needsToShutdown;
+    }
+
+    private void emitBatches()
+    {
+      for (Batch batch; (batch = pollBatchFromEmitQueue()) != null; ) {
+        emit(batch);
+      }
+    }
+
+    private void emit(final Batch batch)
+    {
+      // Awaits until all concurrent event writers finish copy their event bytes to the buffer. This call provides
+      // memory visibility guarantees.
+      batch.awaitEmittingAllowed();
+      try {
+        final int bufferWatermark = batch.getSealedBufferWatermark();
+        if (bufferWatermark == 0) { // sealed while empty
+          return;
+        }
+        int eventCount = batch.eventCount.get();
+        log.debug(
+            "Sending batch #%d to url[%s], event count[%d], bytes[%d]",
+            batch.batchNumber,
+            url,
+            eventCount,
+            bufferWatermark
+        );
+        int bufferEndOffset = batchingStrategy.writeBatchEnd(batch.buffer, bufferWatermark);
+
+        if (sendWithRetries(batch.buffer, bufferEndOffset, eventCount, true)) {
+          buffersToReuse.add(batch.buffer);
+          approximateBuffersToReuseCount.incrementAndGet();
+        } else {
+          limitFailedBuffersSize();
+          failedBuffers.addLast(new FailedBuffer(batch.buffer, bufferEndOffset, eventCount));
+          approximateFailedBuffersCount.incrementAndGet();
+        }
+      }
+      finally {
+        batchFinalized();
+      }
+    }
+
+    private void limitFailedBuffersSize()
+    {
+      if (failedBuffers.size() >= config.getBatchQueueSizeLimit()) {
+        failedBuffers.removeFirst();
+        approximateFailedBuffersCount.decrementAndGet();
+        droppedBuffers.incrementAndGet();
+        log.error(
+            "failedBuffers queue size reached the limit [%d], dropping the oldest failed buffer",
+            config.getBatchQueueSizeLimit()
+        );
+      }
+    }
+
+    @SuppressWarnings("ArrayEquality")
+    private void emitLargeEvents()
+    {
+      if (largeEventsToEmit.isEmpty()) {
+        return;
+      }
+      // Don't try to emit large events until exhaustion, to avoid starvation of "normal" batches, if large event
+      // posting rate is too high, though it should never happen in practice.
+      largeEventsToEmit.add(LARGE_EVENTS_STOP);
+      for (byte[] largeEvent; (largeEvent = largeEventsToEmit.poll()) != LARGE_EVENTS_STOP; ) {
+        emitLargeEvent(largeEvent);
+        approximateBuffersToEmitCount.decrementAndGet();
+        approximateLargeEventsToEmitCount.decrementAndGet();
+        approximateEventsToEmitCount.decrementAndGet();
+      }
+    }
+
+    private void emitLargeEvent(byte[] eventBytes)
+    {
+      byte[] buffer = acquireBuffer();
+      int bufferOffset = batchingStrategy.writeBatchStart(buffer);
+      System.arraycopy(eventBytes, 0, buffer, bufferOffset, eventBytes.length);
+      bufferOffset += eventBytes.length;
+      bufferOffset = batchingStrategy.writeBatchEnd(buffer, bufferOffset);
+      if (sendWithRetries(buffer, bufferOffset, 1, true)) {
+        buffersToReuse.add(buffer);
+        approximateBuffersToReuseCount.incrementAndGet();
+      } else {
+        limitFailedBuffersSize();
+        failedBuffers.addLast(new FailedBuffer(buffer, bufferOffset, 1));
+        approximateFailedBuffersCount.incrementAndGet();
+      }
+    }
+
+    private void tryEmitOneFailedBuffer()
+    {
+      FailedBuffer failedBuffer = failedBuffers.peekFirst();
+      if (failedBuffer != null) {
+        if (sendWithRetries(failedBuffer.buffer, failedBuffer.length, failedBuffer.eventCount, false)) {
+          // Remove from the queue of failed buffer.
+          failedBuffers.pollFirst();
+          approximateFailedBuffersCount.decrementAndGet();
+          // Don't add the failed buffer back to the buffersToReuse queue here, because in a situation when we were not
+          // able to emit events for a while we don't have a way to discard buffers that were used to accumulate events
+          // during that period, if they are added back to buffersToReuse. For instance it may result in having 100
+          // buffers in rotation even if we need just 2.
+        }
+      }
+    }
+
+    private void tryEmitAndDrainAllFailedBuffers()
+    {
+      for (FailedBuffer failedBuffer; (failedBuffer = failedBuffers.pollFirst()) != null; ) {
+        sendWithRetries(failedBuffer.buffer, failedBuffer.length, failedBuffer.eventCount, false);
+        approximateFailedBuffersCount.decrementAndGet();
+      }
+    }
+
+    /**
+     * Returns true if sent successfully.
+     */
+    private boolean sendWithRetries(final byte[] buffer, final int length, final int eventCount, boolean withTimeout)
+    {
+      long deadLineMillis = System.currentTimeMillis() + sendRequestTimeoutMillis(lastFillTimeMillis);
+      try {
+        RetryUtils.retry(
+            new Callable<Void>()
+            {
+              @Override
+              public Void call() throws Exception
+              {
+                send(buffer, length);
+                return null;
+              }
+            },
+            new Predicate<Throwable>()
+            {
+              @Override
+              public boolean apply(Throwable e)
+              {
+                if (withTimeout && deadLineMillis - System.currentTimeMillis() <= 0) { // overflow-aware
+                  return false;
+                }
+                if (e == timeoutLessThanMinimumException) {
+                  return false; // Doesn't make sense to retry, because the result will be the same.
+                }
+                return !(e instanceof InterruptedException);
+              }
+            },
+            MAX_SEND_RETRIES
+        );
+        totalEmittedEvents.addAndGet(eventCount);
+        return true;
+      }
+      catch (InterruptedException e) {
+        return false;
+      }
+      catch (Exception e) {
+        if (e == timeoutLessThanMinimumException) {
+          log.debug(e, "Failed to send events to url[%s] with timeout less than minimum", config.getRecipientBaseUrl());
+        } else {
+          log.error(e, "Failed to send events to url[%s]", config.getRecipientBaseUrl());
+        }
+        return false;
+      }
+    }
+
+    private void send(byte[] buffer, int length) throws Exception
+    {
+      long lastFillTimeMillis = HttpPostEmitter.this.lastFillTimeMillis;
+      final long timeoutMillis = sendRequestTimeoutMillis(lastFillTimeMillis);
+      if (timeoutMillis < config.getMinHttpTimeoutMillis()) {
+        throw timeoutLessThanMinimumException;
+      }
+      long sendingStartMs = System.currentTimeMillis();
+
+      final RequestBuilder request = new RequestBuilder("POST");
+      request.setUrl(url);
+      byte[] payload;
+      int payloadLength;
+      ContentEncoding contentEncoding = config.getContentEncoding();
+      if (contentEncoding != null) {
+        switch (contentEncoding) {
+          case GZIP:
+            try (GZIPOutputStream gzipOutputStream = acquireGzipOutputStream(length)) {
+              gzipOutputStream.write(buffer, 0, length);
+            }
+            payload = gzipBaos.getBuffer();
+            payloadLength = gzipBaos.size();
+            request.setHeader(HttpHeaders.Names.CONTENT_ENCODING, HttpHeaders.Values.GZIP);
+            break;
+          default:
+            throw new ISE("Unsupported content encoding [%s]", contentEncoding.name());
+        }
+      } else {
+        payload = buffer;
+        payloadLength = length;
+      }
+
+
+      request.setHeader(HttpHeaders.Names.CONTENT_TYPE, "application/json");
+      request.setHeader(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(payloadLength));
+      request.setBody(ByteBuffer.wrap(payload, 0, payloadLength));
+
+      if (config.getBasicAuthentication() != null) {
+        final String[] parts = config.getBasicAuthentication().split(":", 2);
+        final String user = parts[0];
+        final String password = parts.length > 1 ? parts[1] : "";
+        String encoded = Base64.getEncoder().encodeToString((user + ':' + password).getBytes(StandardCharsets.UTF_8));
+        request.setHeader(HttpHeaders.Names.AUTHORIZATION, "Basic " + encoded);
+      }
+
+      request.setRequestTimeout(Ints.saturatedCast(timeoutMillis));
+
+      ListenableFuture<Response> future = client.executeRequest(request);
+      Response response;
+      try {
+        // Don't use Future.get(timeout), because we want to avoid sending the same data twice, in case the send
+        // succeeds finally, but after the timeout.
+        response = future.get();
+      }
+      catch (ExecutionException e) {
+        accountFailedSending(sendingStartMs);
+        if (e.getCause() instanceof TimeoutException) {
+          log.error(
+              "Timing out emitter batch send, last batch fill time [%,d] ms, timeout [%,d] ms",
+              lastFillTimeMillis,
+              timeoutMillis
+          );
+        }
+        throw e;
+      }
+
+      if (response.getStatusCode() == 413) {
+        accountFailedSending(sendingStartMs);
+        throw new ISE(
+            "Received HTTP status 413 from [%s]. Batch size of [%d] may be too large, "
+            + "try adjusting maxBatchSizeBatch property",
+            config.getRecipientBaseUrl(),
+            config.getMaxBatchSize()
+        );
+      }
+
+      if (response.getStatusCode() / 100 != 2) {
+        accountFailedSending(sendingStartMs);
+        throw new ISE(
+            "Emissions of events not successful[%d: %s], with message[%s].",
+            response.getStatusCode(),
+            response.getStatusText(),
+            response.getResponseBody(StandardCharsets.UTF_8).trim()
+        );
+      }
+
+      accountSuccessfulSending(sendingStartMs);
+    }
+
+    private long sendRequestTimeoutMillis(long lastFillTimeMillis)
+    {
+      int emitQueueSize = approximateBuffersToEmitCount.get();
+      if (emitQueueSize < EMIT_QUEUE_THRESHOLD_1) {
+        return (long) (lastFillTimeMillis * config.httpTimeoutAllowanceFactor);
+      }
+      if (emitQueueSize < EMIT_QUEUE_THRESHOLD_2) {
+        // The idea is to not let buffersToEmit queue to grow faster than we can emit buffers.
+        return (long) (lastFillTimeMillis * EQUILIBRIUM_ALLOWANCE_FACTOR);
+      }
+      // If buffersToEmit still grows, try to restrict even more
+      return (long) (lastFillTimeMillis * TIGHT_ALLOWANCE_FACTOR);
+    }
+
+    private void accountSuccessfulSending(long sendingStartMs)
+    {
+      successfulSendingTimeCounter.add((int) Math.max(System.currentTimeMillis() - sendingStartMs, 0));
+    }
+
+    private void accountFailedSending(long sendingStartMs)
+    {
+      failedSendingTimeCounter.add((int) Math.max(System.currentTimeMillis() - sendingStartMs, 0));
+    }
+
+    GZIPOutputStream acquireGzipOutputStream(int length) throws IOException
+    {
+      if (gzipBaos == null) {
+        gzipBaos = new ZeroCopyByteArrayOutputStream(length);
+      } else {
+        gzipBaos.reset();
+      }
+      return new GZIPOutputStream(gzipBaos, true);
+    }
+  }
+
+  private static class FailedBuffer
+  {
+    final byte[] buffer;
+    final int length;
+    final int eventCount;
+
+    private FailedBuffer(byte[] buffer, int length, int eventCount)
+    {
+      this.buffer = buffer;
+      this.length = length;
+      this.eventCount = eventCount;
+    }
+  }
+
+  private byte[] acquireBuffer()
+  {
+    byte[] buffer = buffersToReuse.poll();
+    if (buffer == null) {
+      buffer = new byte[bufferSize];
+      allocatedBuffers.incrementAndGet();
+    } else {
+      approximateBuffersToReuseCount.decrementAndGet();
+    }
+    return buffer;
+  }
+
+  private void drainBuffersToReuse()
+  {
+    while (buffersToReuse.poll() != null) {
+      approximateBuffersToReuseCount.decrementAndGet();
+    }
+  }
+
+  /**
+   * This and the following methods are public for external monitoring purposes.
+   */
+  public int getTotalAllocatedBuffers()
+  {
+    return allocatedBuffers.get();
+  }
+
+  public int getBuffersToEmit()
+  {
+    return approximateBuffersToEmitCount.get();
+  }
+
+  public int getBuffersToReuse()
+  {
+    return approximateBuffersToReuseCount.get();
+  }
+
+  public int getFailedBuffers()
+  {
+    return emittingThread.approximateFailedBuffersCount.get();
+  }
+
+  public int getDroppedBuffers()
+  {
+    return droppedBuffers.get();
+  }
+
+  public long getTotalEmittedEvents()
+  {
+    return totalEmittedEvents.get();
+  }
+
+  public long getEventsToEmit()
+  {
+    return approximateEventsToEmitCount.get();
+  }
+
+  public long getLargeEventsToEmit()
+  {
+    return approximateLargeEventsToEmitCount.get();
+  }
+
+  public ConcurrentTimeCounter getBatchFillingTimeCounter()
+  {
+    return batchFillingTimeCounter;
+  }
+
+  public ConcurrentTimeCounter getSuccessfulSendingTimeCounter()
+  {
+    return emittingThread.successfulSendingTimeCounter;
+  }
+
+  public ConcurrentTimeCounter getFailedSendingTimeCounter()
+  {
+    return emittingThread.successfulSendingTimeCounter;
+  }
+
+  @VisibleForTesting
+  void waitForEmission(int batchNumber) throws Exception
+  {
+    emittedBatchCounter.awaitCount(batchNumber, 10, TimeUnit.SECONDS);
+  }
+
+  @VisibleForTesting
+  void joinEmitterThread() throws InterruptedException
+  {
+    emittingThread.join();
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/LoggingEmitter.java b/java-util/src/main/java/io/druid/java/util/emitter/core/LoggingEmitter.java
new file mode 100644
index 00000000000..5bf21d88e0e
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/LoggingEmitter.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+/**
+ */
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.lifecycle.LifecycleStart;
+import io.druid.java.util.common.lifecycle.LifecycleStop;
+import io.druid.java.util.common.logger.Logger;
+
+import java.io.IOException;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ */
+public class LoggingEmitter implements Emitter
+{
+  private final Logger log;
+  private final Level level;
+  private final ObjectMapper jsonMapper;
+
+  private final AtomicBoolean started = new AtomicBoolean(false);
+
+  public LoggingEmitter(LoggingEmitterConfig config, ObjectMapper jsonMapper)
+  {
+    this(new Logger(config.getLoggerClass()), Level.toLevel(config.getLogLevel()), jsonMapper);
+  }
+
+  public LoggingEmitter(Logger log, Level level, ObjectMapper jsonMapper)
+  {
+    this.log = log;
+    this.level = level;
+    this.jsonMapper = jsonMapper;
+  }
+
+  @Override
+  @LifecycleStart
+  public void start()
+  {
+    final boolean alreadyStarted = started.getAndSet(true);
+    if (!alreadyStarted) {
+      final String message = "Start: started [%s]";
+      switch (level) {
+        case TRACE:
+          if (log.isTraceEnabled()) {
+            log.trace(message, started.get());
+          }
+          break;
+        case DEBUG:
+          if (log.isDebugEnabled()) {
+            log.debug(message, started.get());
+          }
+          break;
+        case INFO:
+          if (log.isInfoEnabled()) {
+            log.info(message, started.get());
+          }
+          break;
+        case WARN:
+          log.warn(message, started.get());
+          break;
+        case ERROR:
+          log.error(message, started.get());
+          break;
+      }
+    }
+  }
+
+  @Override
+  public void emit(Event event)
+  {
+    synchronized (started) {
+      if (!started.get()) {
+        throw new RejectedExecutionException("Service not started.");
+      }
+    }
+    try {
+      final String message = "Event [%s]";
+      switch (level) {
+        case TRACE:
+          if (log.isTraceEnabled()) {
+            log.trace(message, jsonMapper.writeValueAsString(event));
+          }
+          break;
+        case DEBUG:
+          if (log.isDebugEnabled()) {
+            log.debug(message, jsonMapper.writeValueAsString(event));
+          }
+          break;
+        case INFO:
+          if (log.isInfoEnabled()) {
+            log.info(message, jsonMapper.writeValueAsString(event));
+          }
+          break;
+        case WARN:
+          log.warn(message, jsonMapper.writeValueAsString(event));
+          break;
+        case ERROR:
+          log.error(message, jsonMapper.writeValueAsString(event));
+          break;
+      }
+    }
+    catch (Exception e) {
+      log.warn(e, "Failed to generate json");
+    }
+  }
+
+  @Override
+  public void flush() throws IOException
+  {
+
+  }
+
+  @Override
+  @LifecycleStop
+  public void close() throws IOException
+  {
+    final boolean wasStarted = started.getAndSet(false);
+    if (wasStarted) {
+      final String message = "Close: started [%s]";
+      switch (level) {
+        case TRACE:
+          if (log.isTraceEnabled()) {
+            log.trace(message, started.get());
+          }
+          break;
+        case DEBUG:
+          if (log.isDebugEnabled()) {
+            log.debug(message, started.get());
+          }
+          break;
+        case INFO:
+          if (log.isInfoEnabled()) {
+            log.info(message, started.get());
+          }
+          break;
+        case WARN:
+          log.warn(message, started.get());
+          break;
+        case ERROR:
+          log.error(message, started.get());
+          break;
+      }
+    }
+  }
+
+  @Override
+  public String toString()
+  {
+    return "LoggingEmitter{" +
+           "log=" + log +
+           ", level=" + level +
+           '}';
+  }
+
+  public enum Level
+  {
+    TRACE,
+    DEBUG,
+    INFO,
+    WARN,
+    ERROR;
+
+    public static Level toLevel(String name)
+    {
+      return Level.valueOf(StringUtils.toUpperCase(name));
+    }
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/LoggingEmitterConfig.java b/java-util/src/main/java/io/druid/java/util/emitter/core/LoggingEmitterConfig.java
new file mode 100644
index 00000000000..c4aba3de01c
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/LoggingEmitterConfig.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import javax.validation.constraints.NotNull;
+
+/**
+ */
+public class LoggingEmitterConfig
+{
+  @NotNull
+  @JsonProperty
+  private String loggerClass = LoggingEmitter.class.getName();
+
+  @NotNull
+  @JsonProperty
+  private String logLevel = "info";
+
+  public String getLoggerClass()
+  {
+    return loggerClass;
+  }
+
+  public void setLoggerClass(String loggerClass)
+  {
+    this.loggerClass = loggerClass;
+  }
+
+  public String getLogLevel()
+  {
+    return logLevel;
+  }
+
+  public void setLogLevel(String logLevel)
+  {
+    this.logLevel = logLevel;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "LoggingEmitterConfig{" +
+           "loggerClass='" + loggerClass + '\'' +
+           ", logLevel='" + logLevel + '\'' +
+           '}';
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/NoopEmitter.java b/java-util/src/main/java/io/druid/java/util/emitter/core/NoopEmitter.java
new file mode 100644
index 00000000000..13897ddd390
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/NoopEmitter.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import java.io.IOException;
+
+/**
+ */
+public class NoopEmitter implements Emitter
+{
+  @Override
+  public void start()
+  {
+    // Do nothing
+  }
+
+  @Override
+  public void emit(Event event)
+  {
+    // Do nothing
+  }
+
+  @Override
+  public void flush() throws IOException
+  {
+    // Do nothing
+  }
+
+  @Override
+  public void close() throws IOException
+  {
+    // Do nothing
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/ParametrizedUriEmitter.java b/java-util/src/main/java/io/druid/java/util/emitter/core/ParametrizedUriEmitter.java
new file mode 100644
index 00000000000..18934c4f3fb
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/ParametrizedUriEmitter.java
@@ -0,0 +1,212 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableSet;
+import io.druid.java.util.common.lifecycle.Lifecycle;
+import io.druid.java.util.common.lifecycle.LifecycleStart;
+import io.druid.java.util.common.lifecycle.LifecycleStop;
+import io.druid.java.util.common.logger.Logger;
+import org.asynchttpclient.AsyncHttpClient;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.io.Closeable;
+import java.io.Flushable;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiConsumer;
+
+public class ParametrizedUriEmitter implements Flushable, Closeable, Emitter
+{
+  private static final Logger log = new Logger(ParametrizedUriEmitter.class);
+  private static final Set<String> ONLY_FEED_PARAM = ImmutableSet.of("feed");
+
+  private static UriExtractor makeUriExtractor(ParametrizedUriEmitterConfig config)
+  {
+    final String baseUri = config.getRecipientBaseUrlPattern();
+    final ParametrizedUriExtractor parametrizedUriExtractor = new ParametrizedUriExtractor(baseUri);
+    UriExtractor uriExtractor = parametrizedUriExtractor;
+    if (ONLY_FEED_PARAM.equals(parametrizedUriExtractor.getParams())) {
+      uriExtractor = new FeedUriExtractor(baseUri.replace("{feed}", "%s"));
+    }
+    return uriExtractor;
+  }
+
+  /**
+   * Type should be ConcurrentHashMap, not {@link java.util.concurrent.ConcurrentMap}, because the latter _doesn't_
+   * guarantee that the lambda passed to {@link java.util.Map#computeIfAbsent} is executed at most once.
+   */
+  private final ConcurrentHashMap<URI, HttpPostEmitter> emitters = new ConcurrentHashMap<>();
+  private final UriExtractor uriExtractor;
+  private final Object startCloseLock = new Object();
+  @GuardedBy("startCloseLock")
+  private boolean started = false;
+  @GuardedBy("startCloseLock")
+  private boolean closed = false;
+  private final Lifecycle innerLifecycle = new Lifecycle();
+  private final AsyncHttpClient client;
+  private final ObjectMapper jsonMapper;
+  private final ParametrizedUriEmitterConfig config;
+
+  public ParametrizedUriEmitter(
+      ParametrizedUriEmitterConfig config,
+      AsyncHttpClient client,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(config, client, jsonMapper, makeUriExtractor(config));
+  }
+
+  public ParametrizedUriEmitter(
+      ParametrizedUriEmitterConfig config,
+      AsyncHttpClient client,
+      ObjectMapper jsonMapper,
+      UriExtractor uriExtractor
+  )
+  {
+    this.config = config;
+    this.client = client;
+    this.jsonMapper = jsonMapper;
+    this.uriExtractor = uriExtractor;
+  }
+
+  @Override
+  @LifecycleStart
+  public void start()
+  {
+    // Use full synchronized instead of atomic flag, because otherwise some thread may think that the emitter is already
+    // started while it's in the process of starting by another thread.
+    synchronized (startCloseLock) {
+      if (started) {
+        return;
+      }
+      started = true;
+      try {
+        innerLifecycle.start();
+      }
+      catch (RuntimeException e) {
+        throw e;
+      }
+      catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  @Override
+  public void emit(Event event)
+  {
+    try {
+      URI uri = uriExtractor.apply(event);
+      HttpPostEmitter emitter = emitters.get(uri);
+      if (emitter == null) {
+        try {
+          emitter = emitters.computeIfAbsent(uri, u -> {
+            try {
+              return innerLifecycle.addMaybeStartManagedInstance(
+                  new HttpPostEmitter(
+                      config.buildHttpEmitterConfig(u.toString()),
+                      client,
+                      jsonMapper
+                  )
+              );
+            }
+            catch (Exception e) {
+              throw Throwables.propagate(e);
+            }
+          });
+        }
+        catch (RuntimeException e) {
+          log.error(e, "Error while creating or starting an HttpPostEmitter for URI[%s]", uri);
+          return;
+        }
+      }
+      emitter.emit(event);
+    }
+    catch (URISyntaxException e) {
+      log.error(e, "Failed to extract URI for event[%s]", event.toMap());
+    }
+  }
+
+  @Override
+  @LifecycleStop
+  public void close() throws IOException
+  {
+    // Use full synchronized instead of atomic flag, because otherwise some thread may think that the emitter is already
+    // closed while it's in the process of closing by another thread.
+    synchronized (startCloseLock) {
+      if (closed) {
+        return;
+      }
+      closed = true;
+      innerLifecycle.stop();
+    }
+  }
+
+  @Override
+  public void flush() throws IOException
+  {
+    Exception thrown = null;
+    for (HttpPostEmitter httpPostEmitter : emitters.values()) {
+      try {
+        httpPostEmitter.flush();
+      }
+      catch (Exception e) {
+        // If flush was interrupted, exit the loop
+        if (Thread.currentThread().isInterrupted()) {
+          if (thrown != null) {
+            e.addSuppressed(thrown);
+          }
+          throw Throwables.propagate(e);
+        }
+        if (thrown == null) {
+          thrown = e;
+        } else {
+          if (thrown != e) {
+            thrown.addSuppressed(e);
+          }
+        }
+      }
+    }
+    if (thrown != null) {
+      throw Throwables.propagate(thrown);
+    }
+  }
+
+  public void forEachEmitter(BiConsumer<URI, HttpPostEmitter> action)
+  {
+    emitters.forEach(action);
+  }
+
+  @Override
+  public String toString()
+  {
+    return "ParametrizedUriEmitter{" +
+           "emitters=" + emitters.keySet() +
+           ", uriExtractor=" + uriExtractor +
+           ", config=" + config +
+           '}';
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java b/java-util/src/main/java/io/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java
new file mode 100644
index 00000000000..ba70ea55088
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import javax.validation.constraints.NotNull;
+
+public class ParametrizedUriEmitterConfig
+{
+  private static final BaseHttpEmittingConfig DEFAULT_HTTP_EMITTING_CONFIG = new BaseHttpEmittingConfig();
+
+  @NotNull
+  @JsonProperty
+  private String recipientBaseUrlPattern;
+
+  @JsonProperty("httpEmitting")
+  private BaseHttpEmittingConfig httpEmittingConfig = DEFAULT_HTTP_EMITTING_CONFIG;
+
+  public String getRecipientBaseUrlPattern()
+  {
+    return recipientBaseUrlPattern;
+  }
+
+  public HttpEmitterConfig buildHttpEmitterConfig(String baseUri)
+  {
+    return new HttpEmitterConfig(httpEmittingConfig, baseUri);
+  }
+
+  @Override
+  public String toString()
+  {
+    return "ParametrizedUriEmitterConfig{" +
+           "recipientBaseUrlPattern='" + recipientBaseUrlPattern + '\'' +
+           ", httpEmittingConfig=" + httpEmittingConfig +
+           '}';
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/ParametrizedUriExtractor.java b/java-util/src/main/java/io/druid/java/util/emitter/core/ParametrizedUriExtractor.java
new file mode 100644
index 00000000000..6d29bd58ebc
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/ParametrizedUriExtractor.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import io.druid.java.util.common.StringUtils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ParametrizedUriExtractor implements UriExtractor
+{
+  private String uriPattern;
+  private Set<String> params;
+
+  public ParametrizedUriExtractor(String uriPattern)
+  {
+    this.uriPattern = uriPattern;
+    Matcher keyMatcher = Pattern.compile("\\{([^\\}]+)\\}").matcher(uriPattern);
+    params = new HashSet<>();
+    while (keyMatcher.find()) {
+      params.add(keyMatcher.group(1));
+    }
+  }
+
+  public Set<String> getParams()
+  {
+    return params;
+  }
+
+  @Override
+  public URI apply(Event event) throws URISyntaxException
+  {
+    Map<String, Object> eventMap = event.toMap();
+    String processedUri = uriPattern;
+    for (String key : params) {
+      Object paramValue = eventMap.get(key);
+      if (paramValue == null) {
+        throw new IllegalArgumentException(StringUtils.format(
+            "ParametrizedUriExtractor with pattern %s requires %s to be set in event, but found %s",
+            uriPattern,
+            key,
+            eventMap
+        ));
+      }
+      processedUri = processedUri.replace(StringUtils.format("{%s}", key), paramValue.toString());
+    }
+    return new URI(processedUri);
+  }
+
+  @Override
+  public String toString()
+  {
+    return "ParametrizedUriExtractor{" +
+           "uriPattern='" + uriPattern + '\'' +
+           ", params=" + params +
+           '}';
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/UriExtractor.java b/java-util/src/main/java/io/druid/java/util/emitter/core/UriExtractor.java
new file mode 100644
index 00000000000..42e57487c5a
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/UriExtractor.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+
+public interface UriExtractor
+{
+  URI apply(Event event) throws URISyntaxException;
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/core/ZeroCopyByteArrayOutputStream.java b/java-util/src/main/java/io/druid/java/util/emitter/core/ZeroCopyByteArrayOutputStream.java
new file mode 100644
index 00000000000..15e6b86935c
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/core/ZeroCopyByteArrayOutputStream.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.core;
+
+import java.io.ByteArrayOutputStream;
+
+class ZeroCopyByteArrayOutputStream extends ByteArrayOutputStream
+{
+
+  ZeroCopyByteArrayOutputStream(int capacity)
+  {
+    super(capacity);
+  }
+
+  byte[] getBuffer()
+  {
+    return buf;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/factory/EmitterFactory.java b/java-util/src/main/java/io/druid/java/util/emitter/factory/EmitterFactory.java
new file mode 100644
index 00000000000..c45ff640247
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/factory/EmitterFactory.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.factory;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.druid.java.util.common.lifecycle.Lifecycle;
+import io.druid.java.util.emitter.core.Emitter;
+import org.asynchttpclient.AsyncHttpClient;
+
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
+@JsonSubTypes(value = {
+    @JsonSubTypes.Type(name = "http", value = HttpEmitterFactory.class),
+    @JsonSubTypes.Type(name = "logging", value = LoggingEmitterFactory.class),
+    @JsonSubTypes.Type(name = "parametrized", value = ParametrizedUriEmitterFactory.class),
+    @JsonSubTypes.Type(name = "noop", value = NoopEmitterFactory.class),
+})
+public interface EmitterFactory
+{
+  Emitter makeEmitter(ObjectMapper objectMapper, AsyncHttpClient httpClient, Lifecycle lifecycle);
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/factory/HttpEmitterFactory.java b/java-util/src/main/java/io/druid/java/util/emitter/factory/HttpEmitterFactory.java
new file mode 100644
index 00000000000..fd43bd56111
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/factory/HttpEmitterFactory.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.factory;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.druid.java.util.common.lifecycle.Lifecycle;
+import io.druid.java.util.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.HttpEmitterConfig;
+import io.druid.java.util.emitter.core.HttpPostEmitter;
+import org.asynchttpclient.AsyncHttpClient;
+
+public class HttpEmitterFactory extends HttpEmitterConfig implements EmitterFactory
+{
+
+  @Override
+  public Emitter makeEmitter(ObjectMapper objectMapper, AsyncHttpClient httpClient, Lifecycle lifecycle)
+  {
+    Emitter retVal = new HttpPostEmitter(this, httpClient, objectMapper);
+    lifecycle.addManagedInstance(retVal);
+    return retVal;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/factory/LoggingEmitterFactory.java b/java-util/src/main/java/io/druid/java/util/emitter/factory/LoggingEmitterFactory.java
new file mode 100644
index 00000000000..5e7ee9325e1
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/factory/LoggingEmitterFactory.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.factory;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.druid.java.util.common.lifecycle.Lifecycle;
+import io.druid.java.util.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.LoggingEmitter;
+import io.druid.java.util.emitter.core.LoggingEmitterConfig;
+import org.asynchttpclient.AsyncHttpClient;
+
+public class LoggingEmitterFactory extends LoggingEmitterConfig implements EmitterFactory
+{
+  public LoggingEmitterFactory() {}
+
+  @Override
+  public Emitter makeEmitter(ObjectMapper objectMapper, AsyncHttpClient httpClient, Lifecycle lifecycle)
+  {
+    return makeEmitter(objectMapper, lifecycle);
+  }
+
+  public Emitter makeEmitter(ObjectMapper objectMapper, Lifecycle lifecycle)
+  {
+    Emitter retVal = new LoggingEmitter(this, objectMapper);
+    lifecycle.addManagedInstance(retVal);
+    return retVal;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/factory/NoopEmitterFactory.java b/java-util/src/main/java/io/druid/java/util/emitter/factory/NoopEmitterFactory.java
new file mode 100644
index 00000000000..1e89c7b87f5
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/factory/NoopEmitterFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.factory;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.druid.java.util.common.lifecycle.Lifecycle;
+import io.druid.java.util.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.NoopEmitter;
+import org.asynchttpclient.AsyncHttpClient;
+
+public class NoopEmitterFactory implements EmitterFactory
+{
+  @Override
+  public Emitter makeEmitter(ObjectMapper objectMapper, AsyncHttpClient httpClient, Lifecycle lifecycle)
+  {
+    return makeEmitter(lifecycle);
+  }
+
+  public Emitter makeEmitter(Lifecycle lifecycle)
+  {
+    Emitter retVal = new NoopEmitter();
+    lifecycle.addManagedInstance(retVal);
+    return retVal;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/factory/ParametrizedUriEmitterFactory.java b/java-util/src/main/java/io/druid/java/util/emitter/factory/ParametrizedUriEmitterFactory.java
new file mode 100644
index 00000000000..0c6956be9aa
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/factory/ParametrizedUriEmitterFactory.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.factory;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.druid.java.util.common.lifecycle.Lifecycle;
+import io.druid.java.util.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.ParametrizedUriEmitter;
+import io.druid.java.util.emitter.core.ParametrizedUriEmitterConfig;
+import org.asynchttpclient.AsyncHttpClient;
+
+public class ParametrizedUriEmitterFactory extends ParametrizedUriEmitterConfig implements EmitterFactory
+{
+
+  @Override
+  public Emitter makeEmitter(ObjectMapper objectMapper, AsyncHttpClient httpClient, Lifecycle lifecycle)
+  {
+    final Emitter retVal = new ParametrizedUriEmitter(this, httpClient, objectMapper);
+    lifecycle.addManagedInstance(retVal);
+    return retVal;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/service/AlertBuilder.java b/java-util/src/main/java/io/druid/java/util/emitter/service/AlertBuilder.java
new file mode 100644
index 00000000000..705da242c28
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/service/AlertBuilder.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.service;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.common.StringUtils;
+
+import java.util.Map;
+
+/**
+*/
+public class AlertBuilder extends ServiceEventBuilder<AlertEvent>
+{
+  protected final Map<String, Object> dataMap = Maps.newLinkedHashMap();
+  protected final String description;
+  protected final ServiceEmitter emitter;
+
+  protected AlertEvent.Severity severity = AlertEvent.Severity.DEFAULT;
+
+  public static AlertBuilder create(String descriptionFormat, Object... objects)
+  {
+    return AlertBuilder.createEmittable(null, descriptionFormat, objects);
+  }
+
+  public static AlertBuilder createEmittable(ServiceEmitter emitter, String descriptionFormat, Object... objects)
+  {
+    return new AlertBuilder(StringUtils.format(descriptionFormat, objects), emitter);
+  }
+
+  protected AlertBuilder(
+      String description,
+      ServiceEmitter emitter
+  )
+  {
+    this.description = description;
+    this.emitter = emitter;
+  }
+
+  public AlertBuilder addData(String identifier, Object value)
+  {
+    dataMap.put(identifier, value);
+    return this;
+  }
+
+  public AlertBuilder addData(Map<String, Object> data)
+  {
+    dataMap.putAll(data);
+    return this;
+  }
+
+  public AlertBuilder severity(AlertEvent.Severity severity)
+  {
+    this.severity = severity;
+    return this;
+  }
+
+  @Override
+  public AlertEvent build(ImmutableMap<String, String> serviceDimensions)
+  {
+    return new AlertEvent(DateTimes.nowUtc(), serviceDimensions, severity, description, dataMap);
+  }
+
+  public void emit()
+  {
+    if (emitter == null) {
+      throw new UnsupportedOperationException("Emitter is null, cannot emit.");
+    }
+
+    emitter.emit(this);
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/service/AlertEvent.java b/java-util/src/main/java/io/druid/java/util/emitter/service/AlertEvent.java
new file mode 100644
index 00000000000..81fc432e84c
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/service/AlertEvent.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.service;
+
+import com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.collect.ImmutableMap;
+import io.druid.java.util.common.DateTimes;
+import org.joda.time.DateTime;
+
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ */
+public class AlertEvent implements ServiceEvent
+{
+  private final ImmutableMap<String, String> serviceDimensions;
+  private final Severity severity;
+  private final String description;
+  private final DateTime createdTime;
+
+  private final Map<String, Object> dataMap;
+
+  public AlertEvent(
+      DateTime createdTime,
+      ImmutableMap<String, String> serviceDimensions,
+      Severity severity,
+      String description,
+      Map<String, Object> dataMap
+  )
+  {
+    this.createdTime = createdTime;
+    this.serviceDimensions = serviceDimensions;
+    this.severity = severity;
+    this.description = description;
+    this.dataMap = dataMap;
+  }
+
+  public AlertEvent(
+      DateTime createdTime,
+      String service,
+      String host,
+      Severity severity,
+      String description,
+      Map<String, Object> dataMap
+  )
+  {
+    this(createdTime, ImmutableMap.of("service", service, "host", host), severity, description, dataMap);
+  }
+
+  public AlertEvent(
+      String service,
+      String host,
+      Severity severity,
+      String description,
+      Map<String, Object> dataMap
+  )
+  {
+    this(DateTimes.nowUtc(), service, host, severity, description, dataMap);
+  }
+
+  public AlertEvent(
+      String service,
+      String host,
+      String description,
+      Map<String, Object> dataMap
+  )
+  {
+    this(DateTimes.nowUtc(), service, host, Severity.DEFAULT, description, dataMap);
+  }
+
+  public AlertEvent(
+      String service,
+      String host,
+      String description
+  )
+  {
+    this(DateTimes.nowUtc(), service, host, Severity.DEFAULT, description, ImmutableMap.<String, Object>of());
+  }
+
+  @Override
+  public DateTime getCreatedTime()
+  {
+    return createdTime;
+  }
+
+  @Override
+  public String getFeed()
+  {
+    return "alerts";
+  }
+
+  @Override
+  public String getService()
+  {
+    return serviceDimensions.get("service");
+  }
+
+  @Override
+  public String getHost()
+  {
+    return serviceDimensions.get("host");
+  }
+
+  @Override
+  public boolean isSafeToBuffer()
+  {
+    return false;
+  }
+
+  public Severity getSeverity()
+  {
+    return severity;
+  }
+
+  public String getDescription()
+  {
+    return description;
+  }
+
+  public Map<String, Object> getDataMap()
+  {
+    return Collections.unmodifiableMap(dataMap);
+  }
+
+  @Override
+  @JsonValue
+  public Map<String, Object> toMap()
+  {
+    return ImmutableMap.<String, Object>builder()
+        .put("feed", getFeed())
+        .put("timestamp", createdTime.toString())
+        .putAll(serviceDimensions)
+        .put("severity", severity.toString())
+        .put("description", description)
+        .put("data", dataMap)
+        .build();
+  }
+
+  public enum Severity
+  {
+    ANOMALY {
+      @Override
+      public String toString()
+      {
+        return "anomaly";
+      }
+    },
+
+    COMPONENT_FAILURE {
+      @Override
+      public String toString()
+      {
+        return "component-failure";
+      }
+    },
+
+    SERVICE_FAILURE {
+      @Override
+      public String toString()
+      {
+        return "service-failure";
+      }
+    };
+
+    public static final Severity DEFAULT = COMPONENT_FAILURE;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceEmitter.java b/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceEmitter.java
new file mode 100644
index 00000000000..f18fdf09cfa
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceEmitter.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.service;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import io.druid.java.util.common.lifecycle.LifecycleStart;
+import io.druid.java.util.common.lifecycle.LifecycleStop;
+import io.druid.java.util.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.Event;
+
+import java.io.IOException;
+
+public class ServiceEmitter implements Emitter
+{
+  private final ImmutableMap<String, String> serviceDimensions;
+  private final Emitter emitter;
+
+  public ServiceEmitter(String service, String host, Emitter emitter)
+  {
+    this(service, host, emitter, ImmutableMap.<String, String>of());
+  }
+
+  public ServiceEmitter(
+      String service,
+      String host,
+      Emitter emitter,
+      ImmutableMap<String, String> otherServiceDimensions
+  )
+  {
+    this.serviceDimensions = ImmutableMap
+        .<String, String>builder()
+        .put("service", Preconditions.checkNotNull(service))
+        .put("host", Preconditions.checkNotNull(host))
+        .putAll(otherServiceDimensions)
+        .build();
+    this.emitter = emitter;
+  }
+
+  public String getService()
+  {
+    return serviceDimensions.get("service");
+  }
+
+  public String getHost()
+  {
+    return serviceDimensions.get("host");
+  }
+
+  @Override
+  @LifecycleStart
+  public void start()
+  {
+    emitter.start();
+  }
+
+  @Override
+  public void emit(Event event)
+  {
+    emitter.emit(event);
+  }
+
+  public void emit(ServiceEventBuilder builder)
+  {
+    emit(builder.build(serviceDimensions));
+  }
+
+  @Override
+  public void flush() throws IOException
+  {
+    emitter.flush();
+  }
+
+  @Override
+  @LifecycleStop
+  public void close() throws IOException
+  {
+    emitter.close();
+  }
+
+  @Override
+  public String toString()
+  {
+    return "ServiceEmitter{" +
+           "serviceDimensions=" + serviceDimensions +
+           ", emitter=" + emitter +
+           '}';
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceEvent.java b/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceEvent.java
new file mode 100644
index 00000000000..4e983cd464d
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceEvent.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.service;
+
+import io.druid.java.util.emitter.core.Event;
+
+public interface ServiceEvent extends Event
+{
+  String getService();
+
+  String getHost();
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceEventBuilder.java b/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceEventBuilder.java
new file mode 100644
index 00000000000..0fc45e57a40
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceEventBuilder.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.service;
+
+import com.google.common.collect.ImmutableMap;
+import io.druid.java.util.emitter.core.Event;
+
+public abstract class ServiceEventBuilder<X extends Event>
+{
+  public abstract X build(ImmutableMap<String, String> serviceDimensions);
+
+  public X build(String service, String host)
+  {
+    return build(ImmutableMap.of("service", service, "host", host));
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceMetricEvent.java b/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceMetricEvent.java
new file mode 100644
index 00000000000..0c4bc9b549b
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/emitter/service/ServiceMetricEvent.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.emitter.service;
+
+import com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.common.ISE;
+import org.joda.time.DateTime;
+
+import java.util.Arrays;
+import java.util.Map;
+
+/**
+ */
+public class ServiceMetricEvent implements ServiceEvent
+{
+  public static Builder builder()
+  {
+    return new Builder();
+  }
+
+  private final DateTime createdTime;
+  private final ImmutableMap<String, String> serviceDims;
+  private final Map<String, Object> userDims;
+  private final String feed;
+  private final String metric;
+  private final Number value;
+
+  private ServiceMetricEvent(
+      DateTime createdTime,
+      ImmutableMap<String, String> serviceDims,
+      Map<String, Object> userDims,
+      String feed,
+      String metric,
+      Number value
+  )
+  {
+    this.createdTime = createdTime != null ? createdTime : DateTimes.nowUtc();
+    this.serviceDims = serviceDims;
+    this.userDims = userDims;
+    this.feed = feed;
+    this.metric = metric;
+    this.value = value;
+  }
+
+  @Override
+  public DateTime getCreatedTime()
+  {
+    return createdTime;
+  }
+
+  @Override
+  public String getFeed()
+  {
+    return feed;
+  }
+
+  @Override
+  public String getService()
+  {
+    return serviceDims.get("service");
+  }
+
+  @Override
+  public String getHost()
+  {
+    return serviceDims.get("host");
+  }
+
+  public Map<String, Object> getUserDims()
+  {
+    return ImmutableMap.copyOf(userDims);
+  }
+
+  public String getMetric()
+  {
+    return metric;
+  }
+
+  public Number getValue()
+  {
+    return value;
+  }
+
+  @Override
+  public boolean isSafeToBuffer()
+  {
+    return true;
+  }
+
+  @Override
+  @JsonValue
+  public Map<String, Object> toMap()
+  {
+    return ImmutableMap.<String, Object>builder()
+                       .put("feed", getFeed())
+                       .put("timestamp", createdTime.toString())
+                       .putAll(serviceDims)
+                       .put("metric", metric)
+                       .put("value", value)
+                       .putAll(
+                           Maps.filterEntries(
+                               userDims,
+                               new Predicate<Map.Entry<String, Object>>()
+                               {
+                                 @Override
+                                 public boolean apply(Map.Entry<String, Object> input)
+                                 {
+                                   return input.getKey() != null;
+                                 }
+                               }
+                           )
+                       )
+                       .build();
+  }
+
+  public static class Builder
+  {
+    private final Map<String, Object> userDims = Maps.newTreeMap();
+    private String feed = "metrics";
+
+    public Builder setFeed(String feed)
+    {
+      this.feed = feed;
+      return this;
+    }
+
+    public Builder setDimension(String dim, String[] values)
+    {
+      userDims.put(dim, Arrays.asList(values));
+      return this;
+    }
+
+    public Builder setDimension(String dim, String value)
+    {
+      userDims.put(dim, value);
+      return this;
+    }
+
+    public Object getDimension(String dim)
+    {
+      return userDims.get(dim);
+    }
+
+    public ServiceEventBuilder<ServiceMetricEvent> build(
+        final String metric,
+        final Number value
+    )
+    {
+      return build(null, metric, value);
+    }
+
+    public ServiceEventBuilder<ServiceMetricEvent> build(
+        final DateTime createdTime,
+        final String metric,
+        final Number value
+    )
+    {
+      if (Double.isNaN(value.doubleValue())) {
+        throw new ISE("Value of NaN is not allowed!");
+      }
+      if (Double.isInfinite(value.doubleValue())) {
+        throw new ISE("Value of Infinite is not allowed!");
+      }
+
+      return new ServiceEventBuilder<ServiceMetricEvent>()
+      {
+        @Override
+        public ServiceMetricEvent build(ImmutableMap<String, String> serviceDimensions)
+        {
+          return new ServiceMetricEvent(
+              createdTime,
+              serviceDimensions,
+              userDims,
+              feed,
+              metric,
+              value
+          );
+        }
+      };
+    }
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/AbstractHttpClient.java b/java-util/src/main/java/io/druid/java/util/http/client/AbstractHttpClient.java
new file mode 100644
index 00000000000..8335aa56109
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/AbstractHttpClient.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import io.druid.java.util.http.client.response.HttpResponseHandler;
+
+public abstract class AbstractHttpClient implements HttpClient
+{
+  @Override
+  public <Intermediate, Final> ListenableFuture<Final> go(
+      final Request request,
+      final HttpResponseHandler<Intermediate, Final> handler
+  )
+  {
+    return go(request, handler, null);
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/CredentialedHttpClient.java b/java-util/src/main/java/io/druid/java/util/http/client/CredentialedHttpClient.java
new file mode 100644
index 00000000000..d5689443353
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/CredentialedHttpClient.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import io.druid.java.util.http.client.auth.Credentials;
+import io.druid.java.util.http.client.response.HttpResponseHandler;
+import org.joda.time.Duration;
+
+/**
+ */
+public class CredentialedHttpClient extends AbstractHttpClient
+{
+
+  private final Credentials creds;
+  private final HttpClient delegate;
+
+  public CredentialedHttpClient(Credentials creds, HttpClient delegate)
+  {
+    this.creds = creds;
+    this.delegate = delegate;
+  }
+
+  @Override
+  public <Intermediate, Final> ListenableFuture<Final> go(
+      Request request,
+      HttpResponseHandler<Intermediate, Final> handler,
+      Duration requestReadTimeout
+  )
+  {
+    return delegate.go(creds.addCredentials(request), handler, requestReadTimeout);
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/EnforceSslHttpClient.java b/java-util/src/main/java/io/druid/java/util/http/client/EnforceSslHttpClient.java
new file mode 100644
index 00000000000..c2e41d1c270
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/EnforceSslHttpClient.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.http.client.response.HttpResponseHandler;
+import org.joda.time.Duration;
+
+import java.net.URL;
+
+/**
+ */
+public class EnforceSslHttpClient extends AbstractHttpClient
+{
+
+  private final HttpClient delegate;
+
+  public EnforceSslHttpClient(
+      HttpClient delegate
+  )
+  {
+    this.delegate = delegate;
+  }
+
+  @Override
+  public <Intermediate, Final> ListenableFuture<Final> go(
+      Request request,
+      HttpResponseHandler<Intermediate, Final> handler,
+      Duration requestReadTimeout
+  )
+  {
+    URL url = request.getUrl();
+
+    if (!"https".equals(url.getProtocol())) {
+      throw new IllegalArgumentException(StringUtils.format("Requests must be over https, got[%s].", url));
+    }
+
+    return delegate.go(request, handler, requestReadTimeout);
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/HttpClient.java b/java-util/src/main/java/io/druid/java/util/http/client/HttpClient.java
new file mode 100644
index 00000000000..5950b863ed5
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/HttpClient.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import io.druid.java.util.http.client.response.HttpResponseHandler;
+import org.joda.time.Duration;
+
+/**
+ * Interface for Async HTTP client libraries.
+ */
+public interface HttpClient
+{
+  /**
+   * Submit a request and process the response with the given response handler.
+   * <p>
+   * Note that the Request object passed in to the HttpClient *may* be mutated by the actual client.  This is
+   * largely done by composed clients, but the contract is that mutation is possible.  It is the caller's
+   * responsibility to pass in a copy of the Request object if they want to have an object that is not mutated.
+   *
+   * @param request        Request to process, this *may* be mutated by the client
+   * @param handler        An asynchronous response handler that will be used to process results from the http call
+   * @param <Intermediate> The type of the intermediate results from the handler
+   * @param <Final>        The type of the final results that the returned ListenableFuture will contain
+   *
+   * @return A listenable future that will eventually provide an object of type Final
+   */
+  <Intermediate, Final> ListenableFuture<Final> go(
+      Request request,
+      HttpResponseHandler<Intermediate, Final> handler
+  );
+
+  /**
+   * Submit a request and process the response with the given response handler.
+   * <p>
+   * Note that the Request object passed in to the HttpClient *may* be mutated by the actual client.  This is
+   * largely done by composed clients, but the contract is that mutation is possible.  It is the caller's
+   * responsibility to pass in a copy of the Request object if they want to have an object that is not mutated.
+   *
+   * @param request        Request to process, this *may* be mutated by the client
+   * @param handler        An asynchronous response handler that will be used to process results from the http call
+   * @param readTimeout    Read timeout to use for this request. Leave null to use the default readTimeout. Set to zero
+   *                       to disable timeouts for this request.
+   * @param <Intermediate> The type of the intermediate results from the handler
+   * @param <Final>        The type of the final results that the returned ListenableFuture will contain
+   *
+   * @return A listenable future that will eventually provide an object of type Final
+   */
+  <Intermediate, Final> ListenableFuture<Final> go(
+      Request request,
+      HttpResponseHandler<Intermediate, Final> handler,
+      Duration readTimeout
+  );
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/HttpClientConfig.java b/java-util/src/main/java/io/druid/java/util/http/client/HttpClientConfig.java
new file mode 100644
index 00000000000..54e45805272
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/HttpClientConfig.java
@@ -0,0 +1,289 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client;
+
+import org.joda.time.Duration;
+import org.joda.time.Period;
+
+import javax.net.ssl.SSLContext;
+
+/**
+ */
+public class HttpClientConfig
+{
+  public enum CompressionCodec
+  {
+    IDENTITY {
+      @Override
+      public String getEncodingString()
+      {
+        return "identity";
+      }
+    },
+    GZIP {
+      @Override
+      public String getEncodingString()
+      {
+        return "gzip";
+      }
+    },
+    DEFLATE {
+      @Override
+      public String getEncodingString()
+      {
+        return "deflate";
+      }
+    };
+
+    /**
+     * Get the header-ified name of this encoding, which should go in "Accept-Encoding" and
+     * "Content-Encoding" headers. This is not just the lowercasing of the enum name, since
+     * we may one day support x- encodings like LZ4, which would likely be an enum named
+     * "LZ4" that has an encoding string like "x-lz4".
+     *
+     * @return encoding name
+     */
+    public abstract String getEncodingString();
+  }
+
+  public static final CompressionCodec DEFAULT_COMPRESSION_CODEC = CompressionCodec.GZIP;
+
+  // Default from NioClientSocketChannelFactory.DEFAULT_BOSS_COUNT, which is private:
+  private static final int DEFAULT_BOSS_COUNT = 1;
+
+  // Default from SelectorUtil.DEFAULT_IO_THREADS, which is private:
+  private static final int DEFAULT_WORKER_COUNT = Runtime.getRuntime().availableProcessors() * 2;
+
+  private static final Duration DEFAULT_UNUSED_CONNECTION_TIMEOUT_DURATION = new Period("PT4M").toStandardDuration();
+
+  public static Builder builder()
+  {
+    return new Builder();
+  }
+
+  private final int numConnections;
+  private final SSLContext sslContext;
+  private final Duration readTimeout;
+  private final Duration sslHandshakeTimeout;
+  private final int bossPoolSize;
+  private final int workerPoolSize;
+  private final CompressionCodec compressionCodec;
+  private final Duration unusedConnectionTimeoutDuration;
+
+  @Deprecated // Use the builder instead
+  public HttpClientConfig(
+      int numConnections,
+      SSLContext sslContext
+  )
+  {
+    this(
+        numConnections,
+        sslContext,
+        Duration.ZERO,
+        null,
+        DEFAULT_BOSS_COUNT,
+        DEFAULT_WORKER_COUNT,
+        DEFAULT_COMPRESSION_CODEC,
+        DEFAULT_UNUSED_CONNECTION_TIMEOUT_DURATION
+    );
+  }
+
+  @Deprecated // Use the builder instead
+  public HttpClientConfig(
+      int numConnections,
+      SSLContext sslContext,
+      Duration readTimeout
+  )
+  {
+    this(
+        numConnections,
+        sslContext,
+        readTimeout,
+        null,
+        DEFAULT_BOSS_COUNT,
+        DEFAULT_WORKER_COUNT,
+        DEFAULT_COMPRESSION_CODEC,
+        DEFAULT_UNUSED_CONNECTION_TIMEOUT_DURATION
+    );
+  }
+
+  @Deprecated // Use the builder instead
+  public HttpClientConfig(
+      int numConnections,
+      SSLContext sslContext,
+      Duration readTimeout,
+      Duration sslHandshakeTimeout
+  )
+  {
+    this(
+        numConnections,
+        sslContext,
+        readTimeout,
+        sslHandshakeTimeout,
+        DEFAULT_BOSS_COUNT,
+        DEFAULT_WORKER_COUNT,
+        DEFAULT_COMPRESSION_CODEC,
+        DEFAULT_UNUSED_CONNECTION_TIMEOUT_DURATION
+    );
+  }
+
+  private HttpClientConfig(
+      int numConnections,
+      SSLContext sslContext,
+      Duration readTimeout,
+      Duration sslHandshakeTimeout,
+      int bossPoolSize,
+      int workerPoolSize,
+      CompressionCodec compressionCodec,
+      Duration unusedConnectionTimeoutDuration
+  )
+  {
+    this.numConnections = numConnections;
+    this.sslContext = sslContext;
+    this.readTimeout = readTimeout;
+    this.sslHandshakeTimeout = sslHandshakeTimeout;
+    this.bossPoolSize = bossPoolSize;
+    this.workerPoolSize = workerPoolSize;
+    this.compressionCodec = compressionCodec;
+    this.unusedConnectionTimeoutDuration = unusedConnectionTimeoutDuration;
+  }
+
+  public int getNumConnections()
+  {
+    return numConnections;
+  }
+
+  public SSLContext getSslContext()
+  {
+    return sslContext;
+  }
+
+  public Duration getReadTimeout()
+  {
+    return readTimeout;
+  }
+
+  public Duration getSslHandshakeTimeout()
+  {
+    return sslHandshakeTimeout;
+  }
+
+  public int getBossPoolSize()
+  {
+    return bossPoolSize;
+  }
+
+  public int getWorkerPoolSize()
+  {
+    return workerPoolSize;
+  }
+
+  public CompressionCodec getCompressionCodec()
+  {
+    return compressionCodec;
+  }
+
+  public Duration getUnusedConnectionTimeoutDuration()
+  {
+    return unusedConnectionTimeoutDuration;
+  }
+
+  public static class Builder
+  {
+    private int numConnections = 1;
+    private SSLContext sslContext = null;
+    private Duration readTimeout = null;
+    private Duration sslHandshakeTimeout = null;
+    private int bossCount = DEFAULT_BOSS_COUNT;
+    private int workerCount = DEFAULT_WORKER_COUNT;
+    private CompressionCodec compressionCodec = DEFAULT_COMPRESSION_CODEC;
+    private Duration unusedConnectionTimeoutDuration = DEFAULT_UNUSED_CONNECTION_TIMEOUT_DURATION;
+
+    private Builder() {}
+
+    public Builder withNumConnections(int numConnections)
+    {
+      this.numConnections = numConnections;
+      return this;
+    }
+
+    public Builder withSslContext(SSLContext sslContext)
+    {
+      this.sslContext = sslContext;
+      return this;
+    }
+
+    public Builder withSslContext(String keyStorePath, String keyStorePassword)
+    {
+      this.sslContext = HttpClientInit.sslContextWithTrustedKeyStore(keyStorePath, keyStorePassword);
+      return this;
+    }
+
+    public Builder withReadTimeout(Duration readTimeout)
+    {
+      this.readTimeout = readTimeout;
+      return this;
+    }
+
+    public Builder withSslHandshakeTimeout(Duration sslHandshakeTimeout)
+    {
+      this.sslHandshakeTimeout = sslHandshakeTimeout;
+      return this;
+    }
+
+    public Builder withBossCount(int bossCount)
+    {
+      this.bossCount = bossCount;
+      return this;
+    }
+
+    public Builder withWorkerCount(int workerCount)
+    {
+      this.workerCount = workerCount;
+      return this;
+    }
+
+    public Builder withCompressionCodec(CompressionCodec compressionCodec)
+    {
+      this.compressionCodec = compressionCodec;
+      return this;
+    }
+
+    public Builder withUnusedConnectionTimeoutDuration(Duration unusedConnectionTimeoutDuration)
+    {
+      this.unusedConnectionTimeoutDuration = unusedConnectionTimeoutDuration;
+      return this;
+    }
+
+    public HttpClientConfig build()
+    {
+      return new HttpClientConfig(
+          numConnections,
+          sslContext,
+          readTimeout,
+          sslHandshakeTimeout,
+          bossCount,
+          workerCount,
+          compressionCodec,
+          unusedConnectionTimeoutDuration
+      );
+    }
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/HttpClientInit.java b/java-util/src/main/java/io/druid/java/util/http/client/HttpClientInit.java
new file mode 100644
index 00000000000..1141829a960
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/HttpClientInit.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client;
+
+import com.google.common.base.Throwables;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.druid.java.util.common.guava.CloseQuietly;
+import io.druid.java.util.common.lifecycle.Lifecycle;
+import io.druid.java.util.http.client.netty.HttpClientPipelineFactory;
+import io.druid.java.util.http.client.pool.ChannelResourceFactory;
+import io.druid.java.util.http.client.pool.ResourcePool;
+import io.druid.java.util.http.client.pool.ResourcePoolConfig;
+import org.jboss.netty.bootstrap.ClientBootstrap;
+import org.jboss.netty.channel.socket.nio.NioClientBossPool;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+import org.jboss.netty.channel.socket.nio.NioWorkerPool;
+import org.jboss.netty.logging.InternalLoggerFactory;
+import org.jboss.netty.logging.Slf4JLoggerFactory;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.ThreadNameDeterminer;
+import org.jboss.netty.util.Timer;
+import org.joda.time.Duration;
+
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.security.KeyManagementException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+/**
+ */
+public class HttpClientInit
+{
+  public static HttpClient createClient(HttpClientConfig config, Lifecycle lifecycle)
+  {
+    try {
+      // We need to use the full constructor in order to set a ThreadNameDeterminer. The other parameters are taken
+      // from the defaults in HashedWheelTimer's other constructors.
+      final HashedWheelTimer timer = new HashedWheelTimer(
+          new ThreadFactoryBuilder().setDaemon(true)
+                                    .setNameFormat("HttpClient-Timer-%s")
+                                    .build(),
+          ThreadNameDeterminer.CURRENT,
+          100,
+          TimeUnit.MILLISECONDS,
+          512
+      );
+      lifecycle.addMaybeStartHandler(
+          new Lifecycle.Handler()
+          {
+            @Override
+            public void start() throws Exception
+            {
+              timer.start();
+            }
+
+            @Override
+            public void stop()
+            {
+              timer.stop();
+            }
+          }
+      );
+      return lifecycle.addMaybeStartManagedInstance(
+          new NettyHttpClient(
+              new ResourcePool<>(
+                  new ChannelResourceFactory(
+                      createBootstrap(lifecycle, timer, config.getBossPoolSize(), config.getWorkerPoolSize()),
+                      config.getSslContext(),
+                      timer,
+                      config.getSslHandshakeTimeout() == null ? -1 : config.getSslHandshakeTimeout().getMillis()
+                  ),
+                  new ResourcePoolConfig(
+                      config.getNumConnections(),
+                      config.getUnusedConnectionTimeoutDuration().getMillis()
+                  )
+              ),
+              config.getReadTimeout(),
+              config.getCompressionCodec(),
+              timer
+          )
+      );
+    }
+    catch (Exception e) {
+      throw Throwables.propagate(e);
+    }
+  }
+
+  @Deprecated
+  public static HttpClient createClient(ResourcePoolConfig config, final SSLContext sslContext, Lifecycle lifecycle)
+  {
+    return createClient(
+        new HttpClientConfig(config.getMaxPerKey(), sslContext, Duration.ZERO),
+        lifecycle
+    );
+  }
+
+  @Deprecated // use createClient directly
+  public static ClientBootstrap createBootstrap(Lifecycle lifecycle, Timer timer)
+  {
+    final HttpClientConfig defaultConfig = HttpClientConfig.builder().build();
+    return createBootstrap(lifecycle, timer, defaultConfig.getBossPoolSize(), defaultConfig.getWorkerPoolSize());
+  }
+
+  @Deprecated // use createClient directly
+  public static ClientBootstrap createBootstrap(Lifecycle lifecycle)
+  {
+    final Timer timer = new HashedWheelTimer(new ThreadFactoryBuilder().setDaemon(true).build());
+    return createBootstrap(lifecycle, timer);
+  }
+
+  public static SSLContext sslContextWithTrustedKeyStore(final String keyStorePath, final String keyStorePassword)
+  {
+    FileInputStream in = null;
+    try {
+      final KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+
+      in = new FileInputStream(keyStorePath);
+      ks.load(in, keyStorePassword.toCharArray());
+      in.close();
+
+      final TrustManagerFactory tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      final SSLContext sslContext = SSLContext.getInstance("TLS");
+      sslContext.init(null, tmf.getTrustManagers(), null);
+
+      return sslContext;
+    }
+    catch (CertificateException e) {
+      throw Throwables.propagate(e);
+    }
+    catch (NoSuchAlgorithmException e) {
+      throw Throwables.propagate(e);
+    }
+    catch (KeyStoreException e) {
+      throw Throwables.propagate(e);
+    }
+    catch (KeyManagementException e) {
+      throw Throwables.propagate(e);
+    }
+    catch (FileNotFoundException e) {
+      throw Throwables.propagate(e);
+    }
+    catch (IOException e) {
+      throw Throwables.propagate(e);
+    }
+    finally {
+      CloseQuietly.close(in);
+    }
+  }
+
+  private static ClientBootstrap createBootstrap(Lifecycle lifecycle, Timer timer, int bossPoolSize, int workerPoolSize)
+  {
+    final NioClientBossPool bossPool = new NioClientBossPool(
+        Executors.newCachedThreadPool(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("HttpClient-Netty-Boss-%s")
+                .build()
+        ),
+        bossPoolSize,
+        timer,
+        ThreadNameDeterminer.CURRENT
+    );
+
+    final NioWorkerPool workerPool = new NioWorkerPool(
+        Executors.newCachedThreadPool(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("HttpClient-Netty-Worker-%s")
+                .build()
+        ),
+        workerPoolSize,
+        ThreadNameDeterminer.CURRENT
+    );
+
+    final ClientBootstrap bootstrap = new ClientBootstrap(new NioClientSocketChannelFactory(bossPool, workerPool));
+
+    bootstrap.setOption("keepAlive", true);
+    bootstrap.setPipelineFactory(new HttpClientPipelineFactory());
+
+    InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory());
+
+    try {
+      lifecycle.addMaybeStartHandler(
+          new Lifecycle.Handler()
+          {
+            @Override
+            public void start() throws Exception
+            {
+            }
+
+            @Override
+            public void stop()
+            {
+              bootstrap.releaseExternalResources();
+            }
+          }
+      );
+    }
+    catch (Exception e) {
+      throw Throwables.propagate(e);
+    }
+
+    return bootstrap;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/NettyHttpClient.java b/java-util/src/main/java/io/druid/java/util/http/client/NettyHttpClient.java
new file mode 100644
index 00000000000..0d343a06f1a
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/NettyHttpClient.java
@@ -0,0 +1,405 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.Multimap;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import io.druid.java.util.common.IAE;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.lifecycle.LifecycleStart;
+import io.druid.java.util.common.lifecycle.LifecycleStop;
+import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.http.client.pool.ResourceContainer;
+import io.druid.java.util.http.client.pool.ResourcePool;
+import io.druid.java.util.http.client.response.ClientResponse;
+import io.druid.java.util.http.client.response.HttpResponseHandler;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.jboss.netty.handler.codec.http.DefaultHttpRequest;
+import org.jboss.netty.handler.codec.http.HttpChunk;
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpVersion;
+import org.jboss.netty.handler.timeout.ReadTimeoutHandler;
+import org.jboss.netty.util.Timer;
+import org.joda.time.Duration;
+
+import java.net.URL;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ */
+public class NettyHttpClient extends AbstractHttpClient
+{
+  private static final Logger log = new Logger(NettyHttpClient.class);
+
+  private static final String READ_TIMEOUT_HANDLER_NAME = "read-timeout";
+  private static final String LAST_HANDLER_NAME = "last-handler";
+
+  private final Timer timer;
+  private final ResourcePool<String, ChannelFuture> pool;
+  private final HttpClientConfig.CompressionCodec compressionCodec;
+  private final Duration defaultReadTimeout;
+
+  public NettyHttpClient(
+      ResourcePool<String, ChannelFuture> pool
+  )
+  {
+    this(pool, null, HttpClientConfig.DEFAULT_COMPRESSION_CODEC, null);
+  }
+
+  NettyHttpClient(
+      ResourcePool<String, ChannelFuture> pool,
+      Duration defaultReadTimeout,
+      HttpClientConfig.CompressionCodec compressionCodec,
+      Timer timer
+  )
+  {
+    this.pool = Preconditions.checkNotNull(pool, "pool");
+    this.defaultReadTimeout = defaultReadTimeout;
+    this.compressionCodec = Preconditions.checkNotNull(compressionCodec);
+    this.timer = timer;
+
+    if (defaultReadTimeout != null && defaultReadTimeout.getMillis() > 0) {
+      Preconditions.checkNotNull(timer, "timer");
+    }
+  }
+
+  @LifecycleStart
+  public void start()
+  {
+  }
+
+  @LifecycleStop
+  public void stop()
+  {
+    pool.close();
+  }
+
+  public HttpClient withReadTimeout(Duration readTimeout)
+  {
+    return new NettyHttpClient(pool, readTimeout, compressionCodec, timer);
+  }
+
+  public NettyHttpClient withTimer(Timer timer)
+  {
+    return new NettyHttpClient(pool, defaultReadTimeout, compressionCodec, timer);
+  }
+
+  @Override
+  public <Intermediate, Final> ListenableFuture<Final> go(
+      final Request request,
+      final HttpResponseHandler<Intermediate, Final> handler,
+      final Duration requestReadTimeout
+  )
+  {
+    final HttpMethod method = request.getMethod();
+    final URL url = request.getUrl();
+    final Multimap<String, String> headers = request.getHeaders();
+
+    final String requestDesc = StringUtils.format("%s %s", method, url);
+    if (log.isDebugEnabled()) {
+      log.debug("[%s] starting", requestDesc);
+    }
+
+    // Block while acquiring a channel from the pool, then complete the request asynchronously.
+    final Channel channel;
+    final String hostKey = getPoolKey(url);
+    final ResourceContainer<ChannelFuture> channelResourceContainer = pool.take(hostKey);
+    final ChannelFuture channelFuture = channelResourceContainer.get().awaitUninterruptibly();
+    if (!channelFuture.isSuccess()) {
+      channelResourceContainer.returnResource(); // Some other poor sap will have to deal with it...
+      return Futures.immediateFailedFuture(
+          new ChannelException(
+              "Faulty channel in resource pool",
+              channelFuture.getCause()
+          )
+      );
+    } else {
+      channel = channelFuture.getChannel();
+    }
+
+    final String urlFile = Strings.nullToEmpty(url.getFile());
+    final HttpRequest httpRequest = new DefaultHttpRequest(
+        HttpVersion.HTTP_1_1,
+        method,
+        urlFile.isEmpty() ? "/" : urlFile
+    );
+
+    if (!headers.containsKey(HttpHeaders.Names.HOST)) {
+      httpRequest.headers().add(HttpHeaders.Names.HOST, getHost(url));
+    }
+
+    // If Accept-Encoding is set in the Request, use that. Otherwise use the default from "compressionCodec".
+    if (!headers.containsKey(HttpHeaders.Names.ACCEPT_ENCODING)) {
+      httpRequest.headers().set(HttpHeaders.Names.ACCEPT_ENCODING, compressionCodec.getEncodingString());
+    }
+
+    for (Map.Entry<String, Collection<String>> entry : headers.asMap().entrySet()) {
+      String key = entry.getKey();
+
+      for (String obj : entry.getValue()) {
+        httpRequest.headers().add(key, obj);
+      }
+    }
+
+    if (request.hasContent()) {
+      httpRequest.setContent(request.getContent());
+    }
+
+    final long readTimeout = getReadTimeout(requestReadTimeout);
+    final SettableFuture<Final> retVal = SettableFuture.create();
+
+    if (readTimeout > 0) {
+      channel.getPipeline().addLast(
+          READ_TIMEOUT_HANDLER_NAME,
+          new ReadTimeoutHandler(timer, readTimeout, TimeUnit.MILLISECONDS)
+      );
+    }
+
+    channel.getPipeline().addLast(
+        LAST_HANDLER_NAME,
+        new SimpleChannelUpstreamHandler()
+        {
+          private volatile ClientResponse<Intermediate> response = null;
+
+          @Override
+          public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception
+          {
+            if (log.isDebugEnabled()) {
+              log.debug("[%s] messageReceived: %s", requestDesc, e.getMessage());
+            }
+            try {
+              Object msg = e.getMessage();
+
+              if (msg instanceof HttpResponse) {
+                HttpResponse httpResponse = (HttpResponse) msg;
+                if (log.isDebugEnabled()) {
+                  log.debug("[%s] Got response: %s", requestDesc, httpResponse.getStatus());
+                }
+
+                response = handler.handleResponse(httpResponse);
+                if (response.isFinished()) {
+                  retVal.set((Final) response.getObj());
+                }
+
+                if (!httpResponse.isChunked()) {
+                  finishRequest();
+                }
+              } else if (msg instanceof HttpChunk) {
+                HttpChunk httpChunk = (HttpChunk) msg;
+                if (log.isDebugEnabled()) {
+                  log.debug(
+                      "[%s] Got chunk: %sB, last=%s",
+                      requestDesc,
+                      httpChunk.getContent().readableBytes(),
+                      httpChunk.isLast()
+                  );
+                }
+
+                if (httpChunk.isLast()) {
+                  finishRequest();
+                } else {
+                  response = handler.handleChunk(response, httpChunk);
+                  if (response.isFinished() && !retVal.isDone()) {
+                    retVal.set((Final) response.getObj());
+                  }
+                }
+              } else {
+                throw new IllegalStateException(StringUtils.format("Unknown message type[%s]", msg.getClass()));
+              }
+            }
+            catch (Exception ex) {
+              log.warn(ex, "[%s] Exception thrown while processing message, closing channel.", requestDesc);
+
+              if (!retVal.isDone()) {
+                retVal.set(null);
+              }
+              channel.close();
+              channelResourceContainer.returnResource();
+
+              throw ex;
+            }
+          }
+
+          private void finishRequest()
+          {
+            ClientResponse<Final> finalResponse = handler.done(response);
+            if (!finalResponse.isFinished()) {
+              throw new IllegalStateException(
+                  StringUtils.format(
+                      "[%s] Didn't get a completed ClientResponse Object from [%s]",
+                      requestDesc,
+                      handler.getClass()
+                  )
+              );
+            }
+            if (!retVal.isDone()) {
+              retVal.set(finalResponse.getObj());
+            }
+            removeHandlers();
+            channelResourceContainer.returnResource();
+          }
+
+          @Override
+          public void exceptionCaught(ChannelHandlerContext context, ExceptionEvent event) throws Exception
+          {
+            if (log.isDebugEnabled()) {
+              final Throwable cause = event.getCause();
+              if (cause == null) {
+                log.debug("[%s] Caught exception", requestDesc);
+              } else {
+                log.debug(cause, "[%s] Caught exception", requestDesc);
+              }
+            }
+
+            retVal.setException(event.getCause());
+            // response is non-null if we received initial chunk and then exception occurs
+            if (response != null) {
+              handler.exceptionCaught(response, event.getCause());
+            }
+            removeHandlers();
+            try {
+              channel.close();
+            }
+            catch (Exception e) {
+              // ignore
+            }
+            finally {
+              channelResourceContainer.returnResource();
+            }
+
+            context.sendUpstream(event);
+          }
+
+          @Override
+          public void channelDisconnected(ChannelHandlerContext context, ChannelStateEvent event) throws Exception
+          {
+            if (log.isDebugEnabled()) {
+              log.debug("[%s] Channel disconnected", requestDesc);
+            }
+            // response is non-null if we received initial chunk and then exception occurs
+            if (response != null) {
+              handler.exceptionCaught(response, new ChannelException("Channel disconnected"));
+            }
+            channel.close();
+            channelResourceContainer.returnResource();
+            if (!retVal.isDone()) {
+              log.warn("[%s] Channel disconnected before response complete", requestDesc);
+              retVal.setException(new ChannelException("Channel disconnected"));
+            }
+            context.sendUpstream(event);
+          }
+
+          private void removeHandlers()
+          {
+            if (readTimeout > 0) {
+              channel.getPipeline().remove(READ_TIMEOUT_HANDLER_NAME);
+            }
+            channel.getPipeline().remove(LAST_HANDLER_NAME);
+          }
+        }
+    );
+
+    channel.write(httpRequest).addListener(
+        new ChannelFutureListener()
+        {
+          @Override
+          public void operationComplete(ChannelFuture future) throws Exception
+          {
+            if (!future.isSuccess()) {
+              channel.close();
+              channelResourceContainer.returnResource();
+              if (!retVal.isDone()) {
+                retVal.setException(
+                    new ChannelException(
+                        StringUtils.format("[%s] Failed to write request to channel", requestDesc),
+                        future.getCause()
+                    )
+                );
+              }
+            }
+          }
+        }
+    );
+
+    return retVal;
+  }
+
+  private long getReadTimeout(Duration requestReadTimeout)
+  {
+    final long timeout;
+    if (requestReadTimeout != null) {
+      timeout = requestReadTimeout.getMillis();
+    } else if (defaultReadTimeout != null) {
+      timeout = defaultReadTimeout.getMillis();
+    } else {
+      timeout = 0;
+    }
+
+    if (timeout > 0 && timer == null) {
+      log.warn("Cannot time out requests without a timer! Disabling timeout for this request.");
+      return 0;
+    } else {
+      return timeout;
+    }
+  }
+
+  private String getHost(URL url)
+  {
+    int port = url.getPort();
+
+    if (port == -1) {
+      final String protocol = url.getProtocol();
+
+      if ("http".equalsIgnoreCase(protocol)) {
+        port = 80;
+      } else if ("https".equalsIgnoreCase(protocol)) {
+        port = 443;
+      } else {
+        throw new IAE("Cannot figure out default port for protocol[%s], please set Host header.", protocol);
+      }
+    }
+
+    return StringUtils.format("%s:%s", url.getHost(), port);
+  }
+
+  private String getPoolKey(URL url)
+  {
+    return StringUtils.format(
+        "%s://%s:%s", url.getProtocol(), url.getHost(), url.getPort() == -1 ? url.getDefaultPort() : url.getPort()
+    );
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/Request.java b/java-util/src/main/java/io/druid/java/util/http/client/Request.java
new file mode 100644
index 00000000000..f49764101a8
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/Request.java
@@ -0,0 +1,200 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
+import io.druid.java.util.common.StringUtils;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBufferFactory;
+import org.jboss.netty.buffer.HeapChannelBufferFactory;
+import org.jboss.netty.handler.codec.base64.Base64;
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ */
+public class Request
+{
+  private static final ChannelBufferFactory factory = HeapChannelBufferFactory.getInstance();
+
+  private final HttpMethod method;
+  private final URL url;
+  private final Multimap<String, String> headers = Multimaps.newListMultimap(
+      Maps.<String, Collection<String>>newHashMap(),
+      new Supplier<List<String>>()
+      {
+        @Override
+        public List<String> get()
+        {
+          return Lists.newArrayList();
+        }
+      }
+  );
+
+  private ChannelBuffer content;
+
+  public Request(
+      HttpMethod method,
+      URL url
+  )
+  {
+    this.method = method;
+    this.url = url;
+  }
+
+  public HttpMethod getMethod()
+  {
+    return method;
+  }
+
+  public URL getUrl()
+  {
+    return url;
+  }
+
+  public Multimap<String, String> getHeaders()
+  {
+    return headers;
+  }
+
+  public boolean hasContent()
+  {
+    return content != null;
+  }
+
+  public ChannelBuffer getContent()
+  {
+    return content;
+  }
+
+  public Request copy()
+  {
+    Request retVal = new Request(method, url);
+    retVal.headers.putAll(this.headers);
+    retVal.content = content == null ? null : content.copy();
+    return retVal;
+  }
+
+  public Request setHeader(String header, String value)
+  {
+    headers.replaceValues(header, Arrays.asList(value));
+    return this;
+  }
+
+  public Request setHeaderValues(String header, Iterable<String> value)
+  {
+    headers.replaceValues(header, value);
+    return this;
+  }
+
+  public Request setHeaderValues(Multimap<String, String> inHeaders)
+  {
+    for (Map.Entry<String, Collection<String>> entry : inHeaders.asMap().entrySet()) {
+      this.setHeaderValues(entry.getKey(), entry.getValue());
+    }
+    return this;
+  }
+
+  public Request addHeader(String header, String value)
+  {
+    headers.put(header, value);
+    return this;
+  }
+
+  public Request addHeaderValues(String header, Iterable<String> value)
+  {
+    headers.putAll(header, value);
+    return this;
+  }
+
+  public Request addHeaderValues(Multimap<String, String> inHeaders)
+  {
+    for (Map.Entry<String, Collection<String>> entry : inHeaders.asMap().entrySet()) {
+      this.addHeaderValues(entry.getKey(), entry.getValue());
+    }
+    return this;
+  }
+
+  public Request setContent(byte[] bytes)
+  {
+    return setContent(null, bytes);
+  }
+
+  public Request setContent(byte[] bytes, int offset, int length)
+  {
+    return setContent(null, bytes, offset, length);
+  }
+
+  public Request setContent(ChannelBuffer content)
+  {
+    return setContent(null, content);
+  }
+
+  public Request setContent(String contentType, byte[] bytes)
+  {
+    return setContent(contentType, bytes, 0, bytes.length);
+  }
+
+  public Request setContent(String contentType, byte[] bytes, int offset, int length)
+  {
+    return setContent(contentType, factory.getBuffer(bytes, offset, length));
+  }
+
+  public Request setContent(String contentType, ChannelBuffer content)
+  {
+    if (contentType != null) {
+      setHeader(HttpHeaders.Names.CONTENT_TYPE, contentType);
+    }
+
+    this.content = content;
+
+    setHeader(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(content.writerIndex()));
+
+    return this;
+  }
+
+  public Request setBasicAuthentication(String username, String password)
+  {
+    final String base64Value = base64Encode(StringUtils.format("%s:%s", username, password));
+    setHeader(HttpHeaders.Names.AUTHORIZATION, StringUtils.format("Basic %s", base64Value));
+    return this;
+  }
+
+  private String base64Encode(final String value)
+  {
+    final ChannelBufferFactory bufferFactory = HeapChannelBufferFactory.getInstance();
+
+    return Base64
+        .encode(bufferFactory.getBuffer(ByteBuffer.wrap(value.getBytes(Charsets.UTF_8))), false)
+        .toString(Charsets.UTF_8);
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/auth/BasicCredentials.java b/java-util/src/main/java/io/druid/java/util/http/client/auth/BasicCredentials.java
new file mode 100644
index 00000000000..29cc85f5f38
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/auth/BasicCredentials.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.auth;
+
+import io.druid.java.util.http.client.Request;
+
+/**
+ */
+public class BasicCredentials implements Credentials
+{
+  private final String username;
+  private final String password;
+
+  public BasicCredentials(
+      String username,
+      String password
+  )
+  {
+    this.username = username;
+    this.password = password;
+  }
+
+  @Override
+  public Request addCredentials(Request builder)
+  {
+    return builder.setBasicAuthentication(username, password);
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/auth/Credentials.java b/java-util/src/main/java/io/druid/java/util/http/client/auth/Credentials.java
new file mode 100644
index 00000000000..35e204616cf
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/auth/Credentials.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.auth;
+
+import io.druid.java.util.http.client.Request;
+
+/**
+ */
+public interface Credentials
+{
+  Request addCredentials(Request builder);
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/io/AppendableByteArrayInputStream.java b/java-util/src/main/java/io/druid/java/util/http/client/io/AppendableByteArrayInputStream.java
new file mode 100644
index 00000000000..8c3c336cd56
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/io/AppendableByteArrayInputStream.java
@@ -0,0 +1,211 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.io;
+
+import io.druid.java.util.common.logger.Logger;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.LinkedList;
+
+/**
+ */
+public class AppendableByteArrayInputStream extends InputStream
+{
+  private static final Logger log = new Logger(AppendableByteArrayInputStream.class);
+
+  private final LinkedList<byte[]> bytes = new LinkedList<byte[]>();
+  private final SingleByteReaderDoer singleByteReaderDoer = new SingleByteReaderDoer();
+
+  private volatile boolean done = false;
+  private volatile Throwable throwable;
+  private volatile int available = 0;
+
+  private byte[] curr = new byte[]{};
+  private int currIndex = 0;
+
+  public void add(byte[] bytesToAdd)
+  {
+    if (bytesToAdd.length == 0) {
+      return;
+    }
+
+    synchronized (singleByteReaderDoer) {
+      bytes.addLast(bytesToAdd);
+      available += bytesToAdd.length;
+      singleByteReaderDoer.notify();
+    }
+  }
+
+  public void done()
+  {
+    synchronized (singleByteReaderDoer) {
+      done = true;
+      singleByteReaderDoer.notify();
+    }
+  }
+
+  public void exceptionCaught(Throwable t)
+  {
+    synchronized (singleByteReaderDoer) {
+      done = true;
+      throwable = t;
+      singleByteReaderDoer.notify();
+    }
+  }
+
+  @Override
+  public int read() throws IOException
+  {
+    if (scanThroughBytesAndDoSomething(1, singleByteReaderDoer) == 0) {
+      return -1;
+    }
+    return singleByteReaderDoer.getRetVal();
+  }
+
+  @Override
+  public int read(final byte[] b, final int off, int len) throws IOException
+  {
+    if (b == null) {
+      throw new NullPointerException();
+    } else if (off < 0 || len < 0 || len > b.length - off) {
+      throw new IndexOutOfBoundsException();
+    } else if (len == 0) {
+      return 0;
+    }
+
+    final long retVal = scanThroughBytesAndDoSomething(
+        len,
+        new Doer()
+        {
+          int currOff = off;
+
+          @Override
+          public void doSomethingWithByteArray(int numRead)
+          {
+            System.arraycopy(curr, currIndex, b, currOff, numRead);
+            currOff += numRead;
+          }
+        }
+    );
+    return retVal == 0 ? -1 : (int) retVal;
+  }
+
+  @Override
+  public long skip(long n) throws IOException
+  {
+    return scanThroughBytesAndDoSomething(
+        n,
+        new Doer()
+        {
+          @Override
+          public void doSomethingWithByteArray(int numToScan)
+          {
+          }
+        }
+    );
+  }
+
+  private long scanThroughBytesAndDoSomething(long numToScan, Doer doer) throws IOException
+  {
+    long numScanned = 0;
+    long numPulled = 0;
+
+    while (numToScan > numScanned) {
+      if (currIndex >= curr.length) {
+        synchronized (singleByteReaderDoer) {
+          if (bytes.isEmpty()) {
+            if (done) {
+              break;
+            }
+            try {
+              available -= numPulled;
+              numPulled = 0;
+              singleByteReaderDoer.wait();
+            }
+            catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+              throw new IOException("Interrupted!");
+            }
+          }
+
+          if (throwable != null) {
+            throw new IOException(throwable);
+          }
+
+          if (bytes.isEmpty()) {
+            if (done) {
+              break;
+            } else {
+              log.debug("bytes was empty, but read thread was awakened without being done.  This shouldn't happen.");
+              continue;
+            }
+          }
+
+          curr = bytes.removeFirst();
+          currIndex = 0;
+        }
+      }
+
+      final long numToPullFromCurr = Math.min(curr.length - currIndex, numToScan - numScanned);
+      doer.doSomethingWithByteArray((int) numToPullFromCurr);
+      numScanned += numToPullFromCurr;
+      currIndex += numToPullFromCurr;
+      numPulled += numToPullFromCurr;
+    }
+
+    synchronized (singleByteReaderDoer) {
+      available -= numPulled;
+    }
+
+    return numScanned;
+  }
+
+  @Override
+  public int available() throws IOException
+  {
+    return available;
+  }
+
+  private interface Doer
+  {
+    void doSomethingWithByteArray(int numToScan);
+  }
+
+  private class SingleByteReaderDoer implements Doer
+  {
+    private int retVal;
+
+    public SingleByteReaderDoer()
+    {
+    }
+
+    @Override
+    public void doSomethingWithByteArray(int numToScan)
+    {
+      retVal = curr[currIndex];
+    }
+
+    public int getRetVal()
+    {
+      return retVal;
+    }
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/netty/HttpClientPipelineFactory.java b/java-util/src/main/java/io/druid/java/util/http/client/netty/HttpClientPipelineFactory.java
new file mode 100644
index 00000000000..eddda1a53a8
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/netty/HttpClientPipelineFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.netty;
+
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.DefaultChannelPipeline;
+import org.jboss.netty.handler.codec.http.HttpClientCodec;
+import org.jboss.netty.handler.codec.http.HttpContentDecompressor;
+
+/**
+ */
+public class HttpClientPipelineFactory implements ChannelPipelineFactory
+{
+  @Override
+  public ChannelPipeline getPipeline() throws Exception
+  {
+    ChannelPipeline pipeline = new DefaultChannelPipeline();
+
+    pipeline.addLast("codec", new HttpClientCodec());
+    pipeline.addLast("inflater", new HttpContentDecompressor());
+
+    return pipeline;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/pool/ChannelResourceFactory.java b/java-util/src/main/java/io/druid/java/util/http/client/pool/ChannelResourceFactory.java
new file mode 100644
index 00000000000..91dbae44f7d
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/pool/ChannelResourceFactory.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.pool;
+
+import com.google.common.base.Preconditions;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.logger.Logger;
+import org.jboss.netty.bootstrap.ClientBootstrap;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.util.Timer;
+
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLParameters;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.URL;
+
+/**
+ */
+public class ChannelResourceFactory implements ResourceFactory<String, ChannelFuture>
+{
+  private static final Logger log = new Logger(ChannelResourceFactory.class);
+
+  private static final long DEFAULT_SSL_HANDSHAKE_TIMEOUT = 10000L; /* 10 seconds */
+
+  private final ClientBootstrap bootstrap;
+  private final SSLContext sslContext;
+  private final Timer timer;
+  private final long sslHandshakeTimeout;
+
+  public ChannelResourceFactory(
+      ClientBootstrap bootstrap,
+      SSLContext sslContext,
+      Timer timer,
+      long sslHandshakeTimeout
+  )
+  {
+    this.bootstrap = Preconditions.checkNotNull(bootstrap, "bootstrap");
+    this.sslContext = sslContext;
+    this.timer = timer;
+    this.sslHandshakeTimeout = sslHandshakeTimeout >= 0 ? sslHandshakeTimeout : DEFAULT_SSL_HANDSHAKE_TIMEOUT;
+
+    if (sslContext != null) {
+      Preconditions.checkNotNull(timer, "timer is required when sslContext is present");
+    }
+  }
+
+  @Override
+  public ChannelFuture generate(final String hostname)
+  {
+    log.info("Generating: %s", hostname);
+    URL url = null;
+    try {
+      url = new URL(hostname);
+    }
+    catch (MalformedURLException e) {
+      throw new RuntimeException(e);
+    }
+
+    final String host = url.getHost();
+    final int port = url.getPort() == -1 ? url.getDefaultPort() : url.getPort();
+    final ChannelFuture retVal;
+    final ChannelFuture connectFuture = bootstrap.connect(new InetSocketAddress(host, port));
+
+    if ("https".equals(url.getProtocol())) {
+      if (sslContext == null) {
+        throw new IllegalStateException("No sslContext set, cannot do https");
+      }
+
+      final SSLEngine sslEngine = sslContext.createSSLEngine(host, port);
+      final SSLParameters sslParameters = new SSLParameters();
+      sslParameters.setEndpointIdentificationAlgorithm("HTTPS");
+      sslEngine.setSSLParameters(sslParameters);
+      sslEngine.setUseClientMode(true);
+      final SslHandler sslHandler = new SslHandler(
+          sslEngine,
+          SslHandler.getDefaultBufferPool(),
+          false,
+          timer,
+          sslHandshakeTimeout
+      );
+
+      // https://github.com/netty/netty/issues/160
+      sslHandler.setCloseOnSSLException(true);
+
+      final ChannelPipeline pipeline = connectFuture.getChannel().getPipeline();
+      pipeline.addFirst("ssl", sslHandler);
+
+      final ChannelFuture handshakeFuture = Channels.future(connectFuture.getChannel());
+      connectFuture.addListener(
+          new ChannelFutureListener()
+          {
+            @Override
+            public void operationComplete(ChannelFuture f) throws Exception
+            {
+              if (f.isSuccess()) {
+                sslHandler.handshake().addListener(
+                    new ChannelFutureListener()
+                    {
+                      @Override
+                      public void operationComplete(ChannelFuture f2) throws Exception
+                      {
+                        if (f2.isSuccess()) {
+                          handshakeFuture.setSuccess();
+                        } else {
+                          handshakeFuture.setFailure(
+                              new ChannelException(
+                                  StringUtils.format("Failed to handshake with host[%s]", hostname),
+                                  f2.getCause()
+                              )
+                          );
+                        }
+                      }
+                    }
+                );
+              } else {
+                handshakeFuture.setFailure(
+                    new ChannelException(
+                        StringUtils.format("Failed to connect to host[%s]", hostname),
+                        f.getCause()
+                    )
+                );
+              }
+            }
+          }
+      );
+
+      retVal = handshakeFuture;
+    } else {
+      retVal = connectFuture;
+    }
+
+    return retVal;
+  }
+
+  @Override
+  public boolean isGood(ChannelFuture resource)
+  {
+    Channel channel = resource.awaitUninterruptibly().getChannel();
+
+    boolean isSuccess = resource.isSuccess();
+    boolean isConnected = channel.isConnected();
+    boolean isOpen = channel.isOpen();
+
+    if (log.isTraceEnabled()) {
+      log.trace("isGood = isSucess[%s] && isConnected[%s] && isOpen[%s]", isSuccess, isConnected, isOpen);
+    }
+
+    return isSuccess && isConnected && isOpen;
+  }
+
+  @Override
+  public void close(ChannelFuture resource)
+  {
+    log.trace("Closing");
+    resource.awaitUninterruptibly().getChannel().close();
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourceContainer.java b/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourceContainer.java
new file mode 100644
index 00000000000..dcaac747f3e
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourceContainer.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.pool;
+
+/**
+ */
+public interface ResourceContainer<ResourceType>
+{
+  ResourceType get();
+  void returnResource();
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourceFactory.java b/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourceFactory.java
new file mode 100644
index 00000000000..32434530463
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourceFactory.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.pool;
+
+/**
+ */
+public interface ResourceFactory<K, V>
+{
+  V generate(K key);
+  boolean isGood(V resource);
+  void close(V resource);
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourcePool.java b/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourcePool.java
new file mode 100644
index 00000000000..a9254f96de7
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourcePool.java
@@ -0,0 +1,301 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.pool;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableSet;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.logger.Logger;
+
+import java.io.Closeable;
+import java.util.ArrayDeque;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ */
+public class ResourcePool<K, V> implements Closeable
+{
+  private static final Logger log = new Logger(ResourcePool.class);
+  private final LoadingCache<K, ImmediateCreationResourceHolder<K, V>> pool;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+
+  public ResourcePool(
+      final ResourceFactory<K, V> factory,
+      final ResourcePoolConfig config
+  )
+  {
+    this.pool = CacheBuilder.newBuilder().build(
+        new CacheLoader<K, ImmediateCreationResourceHolder<K, V>>()
+        {
+          @Override
+          public ImmediateCreationResourceHolder<K, V> load(K input) throws Exception
+          {
+            return new ImmediateCreationResourceHolder<K, V>(
+                config.getMaxPerKey(),
+                config.getUnusedConnectionTimeoutMillis(),
+                input,
+                factory
+            );
+          }
+        }
+    );
+  }
+
+  public ResourceContainer<V> take(final K key)
+  {
+    if (closed.get()) {
+      log.error(StringUtils.format("take(%s) called even though I'm closed.", key));
+      return null;
+    }
+
+    final ImmediateCreationResourceHolder<K, V> holder;
+    try {
+      holder = pool.get(key);
+    }
+    catch (ExecutionException e) {
+      throw Throwables.propagate(e);
+    }
+    final V value = holder.get();
+
+    return new ResourceContainer<V>()
+    {
+      private final AtomicBoolean returned = new AtomicBoolean(false);
+
+      @Override
+      public V get()
+      {
+        Preconditions.checkState(!returned.get(), "Resource for key[%s] has been returned, cannot get().", key);
+        return value;
+      }
+
+      @Override
+      public void returnResource()
+      {
+        if (returned.getAndSet(true)) {
+          log.warn(StringUtils.format("Resource at key[%s] was returned multiple times?", key));
+        } else {
+          holder.giveBack(value);
+        }
+      }
+
+      @Override
+      protected void finalize() throws Throwable
+      {
+        if (!returned.get()) {
+          log.warn(
+              StringUtils.format(
+                  "Resource[%s] at key[%s] was not returned before Container was finalized, potential resource leak.",
+                  value,
+                  key
+              )
+          );
+          returnResource();
+        }
+        super.finalize();
+      }
+    };
+  }
+
+  @Override
+  public void close()
+  {
+    closed.set(true);
+    final Map<K, ImmediateCreationResourceHolder<K, V>> mapView = pool.asMap();
+    for (K k : ImmutableSet.copyOf(mapView.keySet())) {
+      mapView.remove(k).close();
+    }
+  }
+
+  private static class ImmediateCreationResourceHolder<K, V>
+  {
+    private final int maxSize;
+    private final K key;
+    private final ResourceFactory<K, V> factory;
+    private final ArrayDeque<ResourceHolder<V>> resourceHolderList;
+    private int deficit = 0;
+    private boolean closed = false;
+    private final long unusedResourceTimeoutMillis;
+
+    private ImmediateCreationResourceHolder(
+        int maxSize,
+        long unusedResourceTimeoutMillis,
+        K key,
+        ResourceFactory<K, V> factory
+    )
+    {
+      this.maxSize = maxSize;
+      this.key = key;
+      this.factory = factory;
+      this.unusedResourceTimeoutMillis = unusedResourceTimeoutMillis;
+      this.resourceHolderList = new ArrayDeque<>();
+
+      for (int i = 0; i < maxSize; ++i) {
+        resourceHolderList.add(new ResourceHolder<>(
+            System.currentTimeMillis(),
+            Preconditions.checkNotNull(
+                factory.generate(key),
+                "factory.generate(key)"
+            )
+        ));
+      }
+    }
+
+    V get()
+    {
+      // resourceHolderList can't have nulls, so we'll use a null to signal that we need to create a new resource.
+      final V poolVal;
+      synchronized (this) {
+        while (!closed && resourceHolderList.size() == 0 && deficit == 0) {
+          try {
+            this.wait();
+          }
+          catch (InterruptedException e) {
+            Thread.interrupted();
+            return null;
+          }
+        }
+
+        if (closed) {
+          log.info(StringUtils.format("get() called even though I'm closed. key[%s]", key));
+          return null;
+        } else if (!resourceHolderList.isEmpty()) {
+          ResourceHolder<V> holder = resourceHolderList.removeFirst();
+          if (System.currentTimeMillis() - holder.getLastAccessedTime() > unusedResourceTimeoutMillis) {
+            factory.close(holder.getResource());
+            poolVal = factory.generate(key);
+          } else {
+            poolVal = holder.getResource();
+          }
+        } else if (deficit > 0) {
+          deficit--;
+          poolVal = null;
+        } else {
+          throw new IllegalStateException("WTF?! No objects left, and no object deficit. This is probably a bug.");
+        }
+      }
+
+      // At this point, we must either return a valid resource or increment "deficit".
+      final V retVal;
+      try {
+        if (poolVal != null && factory.isGood(poolVal)) {
+          retVal = poolVal;
+        } else {
+          if (poolVal != null) {
+            factory.close(poolVal);
+          }
+          retVal = factory.generate(key);
+        }
+      }
+      catch (Throwable e) {
+        synchronized (this) {
+          deficit++;
+          this.notifyAll();
+        }
+        throw Throwables.propagate(e);
+      }
+
+      return retVal;
+    }
+
+    void giveBack(V object)
+    {
+      Preconditions.checkNotNull(object, "object");
+
+      synchronized (this) {
+        if (closed) {
+          log.info(StringUtils.format("giveBack called after being closed. key[%s]", key));
+          factory.close(object);
+          return;
+        }
+
+        if (resourceHolderList.size() >= maxSize) {
+          if (holderListContains(object)) {
+            log.warn(
+                StringUtils.format(
+                    "Returning object[%s] at key[%s] that has already been returned!? Skipping",
+                    object,
+                    key
+                ),
+                new Exception("Exception for stacktrace")
+            );
+          } else {
+            log.warn(
+                StringUtils.format(
+                    "Returning object[%s] at key[%s] even though we already have all that we can hold[%s]!? Skipping",
+                    object,
+                    key,
+                    resourceHolderList
+                ),
+                new Exception("Exception for stacktrace")
+            );
+          }
+          return;
+        }
+
+        resourceHolderList.addLast(new ResourceHolder<>(System.currentTimeMillis(), object));
+        this.notifyAll();
+      }
+    }
+
+    private boolean holderListContains(V object)
+    {
+      return resourceHolderList.stream().anyMatch(a -> a.getResource().equals(object));
+    }
+
+    void close()
+    {
+      synchronized (this) {
+        closed = true;
+        resourceHolderList.forEach(v -> factory.close(v.getResource()));
+        resourceHolderList.clear();
+        this.notifyAll();
+      }
+    }
+  }
+
+  private static class ResourceHolder<V>
+  {
+    private long lastAccessedTime;
+    private V resource;
+
+    public ResourceHolder(long lastAccessedTime, V resource)
+    {
+      this.resource = resource;
+      this.lastAccessedTime = lastAccessedTime;
+    }
+
+    public long getLastAccessedTime()
+    {
+      return lastAccessedTime;
+    }
+
+    public V getResource()
+    {
+      return resource;
+    }
+
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourcePoolConfig.java b/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourcePoolConfig.java
new file mode 100644
index 00000000000..66217d84434
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourcePoolConfig.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.pool;
+
+/**
+ */
+public class ResourcePoolConfig
+{
+  private final int maxPerKey;
+  private final long unusedConnectionTimeoutMillis;
+
+  public ResourcePoolConfig(
+      int maxPerKey,
+      long unusedConnectionTimeoutMillis
+  )
+  {
+    this.maxPerKey = maxPerKey;
+    this.unusedConnectionTimeoutMillis = unusedConnectionTimeoutMillis;
+  }
+
+  @Deprecated
+  public ResourcePoolConfig(
+      int maxPerKey,
+      boolean cleanIdle,
+      long unusedConnectionTimeoutMillis
+  )
+  {
+    this(maxPerKey, unusedConnectionTimeoutMillis);
+
+    if (cleanIdle) {
+      throw new IllegalStateException(
+          "Cleaning up idle connections is a bad idea.  "
+          + "If your services can't handle the max number then lower the max number."
+      );
+    }
+  }
+
+  public int getMaxPerKey()
+  {
+    return maxPerKey;
+  }
+
+  public boolean isCleanIdle()
+  {
+    return false;
+  }
+
+  public long getUnusedConnectionTimeoutMillis()
+  {
+    return unusedConnectionTimeoutMillis;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourceVerifier.java b/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourceVerifier.java
new file mode 100644
index 00000000000..926b4bba0ab
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/pool/ResourceVerifier.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.pool;
+
+/**
+ */
+public interface ResourceVerifier<T>
+{
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/response/ClientResponse.java b/java-util/src/main/java/io/druid/java/util/http/client/response/ClientResponse.java
new file mode 100644
index 00000000000..e5bd2ddb16f
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/response/ClientResponse.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.response;
+
+/**
+ */
+public class ClientResponse<T>
+{
+  private final boolean finished;
+  private final T obj;
+
+  public static <T> ClientResponse<T> finished(T obj)
+  {
+    return new ClientResponse<T>(true, obj);
+  }
+
+  public static <T> ClientResponse<T> unfinished(T obj)
+  {
+    return new ClientResponse<T>(false, obj);
+  }
+
+  protected ClientResponse(
+      boolean finished,
+      T obj
+  )
+  {
+    this.finished = finished;
+    this.obj = obj;
+  }
+
+  public boolean isFinished()
+  {
+    return finished;
+  }
+
+  public T getObj()
+  {
+    return obj;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/response/FullResponseHandler.java b/java-util/src/main/java/io/druid/java/util/http/client/response/FullResponseHandler.java
new file mode 100644
index 00000000000..2b75b494143
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/response/FullResponseHandler.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.response;
+
+import org.jboss.netty.handler.codec.http.HttpChunk;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+
+import java.nio.charset.Charset;
+
+/**
+ */
+public class FullResponseHandler implements HttpResponseHandler<FullResponseHolder, FullResponseHolder>
+{
+  private final Charset charset;
+
+  public FullResponseHandler(Charset charset)
+  {
+    this.charset = charset;
+  }
+
+  @Override
+  public ClientResponse<FullResponseHolder> handleResponse(HttpResponse response)
+  {
+    return ClientResponse.unfinished(
+        new FullResponseHolder(
+            response.getStatus(),
+            response,
+            new StringBuilder(response.getContent().toString(charset))
+        )
+    );
+  }
+
+  @Override
+  public ClientResponse<FullResponseHolder> handleChunk(
+      ClientResponse<FullResponseHolder> response,
+      HttpChunk chunk
+  )
+  {
+    final StringBuilder builder = response.getObj().getBuilder();
+
+    if (builder == null) {
+      return ClientResponse.finished(null);
+    }
+
+    builder.append(chunk.getContent().toString(charset));
+    return response;
+  }
+
+  @Override
+  public ClientResponse<FullResponseHolder> done(ClientResponse<FullResponseHolder> response)
+  {
+    return ClientResponse.finished(response.getObj());
+  }
+
+  @Override
+  public void exceptionCaught(
+      ClientResponse<FullResponseHolder> clientResponse, Throwable e
+  )
+  {
+    // Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished
+  }
+
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/response/FullResponseHolder.java b/java-util/src/main/java/io/druid/java/util/http/client/response/FullResponseHolder.java
new file mode 100644
index 00000000000..b75610cdae8
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/response/FullResponseHolder.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.response;
+
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+/**
+ */
+public class FullResponseHolder
+{
+  private final HttpResponseStatus status;
+  private final HttpResponse response;
+  private final StringBuilder builder;
+
+  public FullResponseHolder(
+      HttpResponseStatus status,
+      HttpResponse response,
+      StringBuilder builder
+  )
+  {
+    this.status = status;
+    this.response = response;
+    this.builder = builder;
+  }
+
+  public HttpResponseStatus getStatus()
+  {
+    return status;
+  }
+
+  public HttpResponse getResponse()
+  {
+    return response;
+  }
+
+  public StringBuilder getBuilder()
+  {
+    return builder;
+  }
+
+  public String getContent()
+  {
+    return builder.toString();
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/response/HttpResponseHandler.java b/java-util/src/main/java/io/druid/java/util/http/client/response/HttpResponseHandler.java
new file mode 100644
index 00000000000..69eb792cb9c
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/response/HttpResponseHandler.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.response;
+
+import org.jboss.netty.handler.codec.http.HttpChunk;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+
+/**
+ * A handler for an HTTP request.
+ *
+ * The ClientResponse object passed around is used to store state between further chunks and indicate when it is safe
+ * to hand the object back to the caller.
+ *
+ * If the response is chunked, the ClientResponse object returned from handleResponse will be passed in as the
+ * first argument to handleChunk().
+ *
+ * If the ClientResponse object is marked as finished, that indicates that the object stored is safe to hand
+ * off to the caller.  This is most often done either from the done() method after all content has been processed or
+ * from the initial handleResponse method to indicate that the object is thread-safe and aware that it might be
+ * accessed before all chunks come back.
+ *
+ * Note: if you return a finished ClientResponse object from anything other than the done() method, IntermediateType
+ * must be castable to FinalType
+ */
+public interface HttpResponseHandler<IntermediateType, FinalType>
+{
+  /**
+   * Handles the initial HttpResponse object that comes back from Netty.
+   *
+   * @param response - response from Netty
+   * @return
+   */
+  ClientResponse<IntermediateType> handleResponse(HttpResponse response);
+  ClientResponse<IntermediateType> handleChunk(ClientResponse<IntermediateType> clientResponse, HttpChunk chunk);
+  ClientResponse<FinalType> done(ClientResponse<IntermediateType> clientResponse);
+  void exceptionCaught(ClientResponse<IntermediateType> clientResponse, Throwable e);
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/response/InputStreamResponseHandler.java b/java-util/src/main/java/io/druid/java/util/http/client/response/InputStreamResponseHandler.java
new file mode 100644
index 00000000000..e2792842d22
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/response/InputStreamResponseHandler.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.response;
+
+import io.druid.java.util.http.client.io.AppendableByteArrayInputStream;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.handler.codec.http.HttpChunk;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+
+import java.io.InputStream;
+
+/**
+ */
+public class InputStreamResponseHandler implements HttpResponseHandler<AppendableByteArrayInputStream, InputStream>
+{
+  @Override
+  public ClientResponse<AppendableByteArrayInputStream> handleResponse(HttpResponse response)
+  {
+    AppendableByteArrayInputStream in = new AppendableByteArrayInputStream();
+    in.add(getContentBytes(response.getContent()));
+    return ClientResponse.finished(in);
+  }
+
+  @Override
+  public ClientResponse<AppendableByteArrayInputStream> handleChunk(
+      ClientResponse<AppendableByteArrayInputStream> clientResponse, HttpChunk chunk
+  )
+  {
+    clientResponse.getObj().add(getContentBytes(chunk.getContent()));
+    return clientResponse;
+  }
+
+  @Override
+  public ClientResponse<InputStream> done(ClientResponse<AppendableByteArrayInputStream> clientResponse)
+  {
+    final AppendableByteArrayInputStream obj = clientResponse.getObj();
+    obj.done();
+    return ClientResponse.<InputStream>finished(obj);
+  }
+
+  @Override
+  public void exceptionCaught(
+      ClientResponse<AppendableByteArrayInputStream> clientResponse,
+      Throwable e
+  )
+  {
+    final AppendableByteArrayInputStream obj = clientResponse.getObj();
+    obj.exceptionCaught(e);
+  }
+
+  private byte[] getContentBytes(ChannelBuffer content)
+  {
+    byte[] contentBytes = new byte[content.readableBytes()];
+    content.readBytes(contentBytes);
+    return contentBytes;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/response/SequenceInputStreamResponseHandler.java b/java-util/src/main/java/io/druid/java/util/http/client/response/SequenceInputStreamResponseHandler.java
new file mode 100644
index 00000000000..ffe0a9a1418
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/response/SequenceInputStreamResponseHandler.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.response;
+
+import com.google.common.base.Throwables;
+import com.google.common.io.ByteSource;
+import io.druid.java.util.common.logger.Logger;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBufferInputStream;
+import org.jboss.netty.handler.codec.http.HttpChunk;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.SequenceInputStream;
+import java.util.Enumeration;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A HTTP response handler which uses sequence input streams to create a final InputStream.
+ * Any particular instance is encouraged to overwrite a method and call the super if they need extra handling of the
+ * method parameters.
+ *
+ * This implementation uses a blocking queue to feed a SequenceInputStream that is terminated whenever the handler's Done
+ * method is called or a throwable is detected.
+ *
+ * The resulting InputStream will attempt to terminate normally, but on exception in HttpResponseHandler
+ * may end with an IOException upon read()
+ */
+public class SequenceInputStreamResponseHandler implements HttpResponseHandler<InputStream, InputStream>
+{
+  private static final Logger log = new Logger(SequenceInputStreamResponseHandler.class);
+  private final AtomicLong byteCount = new AtomicLong(0);
+  private final BlockingQueue<InputStream> queue = new LinkedBlockingQueue<>();
+  private final AtomicBoolean done = new AtomicBoolean(false);
+
+  @Override
+  public ClientResponse<InputStream> handleResponse(HttpResponse response)
+  {
+    try {
+      queue.put(new ChannelBufferInputStream(response.getContent()));
+    }
+    catch (InterruptedException e) {
+      log.error(e, "Queue appending interrupted");
+      Thread.currentThread().interrupt();
+      throw Throwables.propagate(e);
+    }
+    byteCount.addAndGet(response.getContent().readableBytes());
+    return ClientResponse.<InputStream>finished(
+        new SequenceInputStream(
+            new Enumeration<InputStream>()
+            {
+              @Override
+              public boolean hasMoreElements()
+              {
+                // Done is always true until the last stream has be put in the queue.
+                // Then the stream should be spouting good InputStreams.
+                synchronized (done) {
+                  return !done.get() || !queue.isEmpty();
+                }
+              }
+
+              @Override
+              public InputStream nextElement()
+              {
+                try {
+                  return queue.take();
+                }
+                catch (InterruptedException e) {
+                  log.warn(e, "Thread interrupted while taking from queue");
+                  Thread.currentThread().interrupt();
+                  throw Throwables.propagate(e);
+                }
+              }
+            }
+        )
+    );
+  }
+
+  @Override
+  public ClientResponse<InputStream> handleChunk(
+      ClientResponse<InputStream> clientResponse, HttpChunk chunk
+  )
+  {
+    final ChannelBuffer channelBuffer = chunk.getContent();
+    final int bytes = channelBuffer.readableBytes();
+    if (bytes > 0) {
+      try {
+        queue.put(new ChannelBufferInputStream(channelBuffer));
+        // Queue.size() can be expensive in some implementations, but LinkedBlockingQueue.size is just an AtomicLong
+        log.debug("Added stream. Queue length %d", queue.size());
+      }
+      catch (InterruptedException e) {
+        log.warn(e, "Thread interrupted while adding to queue");
+        Thread.currentThread().interrupt();
+        throw Throwables.propagate(e);
+      }
+      byteCount.addAndGet(bytes);
+    } else {
+      log.debug("Skipping zero length chunk");
+    }
+    return clientResponse;
+  }
+
+  @Override
+  public ClientResponse<InputStream> done(ClientResponse<InputStream> clientResponse)
+  {
+    synchronized (done) {
+      try {
+        // An empty byte array is put at the end to give the SequenceInputStream.close() as something to close out
+        // after done is set to true, regardless of the rest of the stream's state.
+        queue.put(ByteSource.empty().openStream());
+        log.debug("Added terminal empty stream");
+      }
+      catch (InterruptedException e) {
+        log.warn(e, "Thread interrupted while adding to queue");
+        Thread.currentThread().interrupt();
+        throw Throwables.propagate(e);
+      }
+      catch (IOException e) {
+        // This should never happen
+        log.wtf(e, "The empty stream threw an IOException");
+        throw Throwables.propagate(e);
+      }
+      finally {
+        log.debug("Done after adding %d bytes of streams", byteCount.get());
+        done.set(true);
+      }
+    }
+    return ClientResponse.<InputStream>finished(clientResponse.getObj());
+  }
+
+  @Override
+  public void exceptionCaught(final ClientResponse<InputStream> clientResponse, final Throwable e)
+  {
+    // Don't wait for lock in case the lock had something to do with the error
+    synchronized (done) {
+      done.set(true);
+      // Make a best effort to put a zero length buffer into the queue in case something is waiting on the take()
+      // If nothing is waiting on take(), this will be closed out anyways.
+      final boolean accepted = queue.offer(
+          new InputStream()
+          {
+            @Override
+            public int read() throws IOException
+            {
+              throw new IOException(e);
+            }
+          }
+      );
+      if (!accepted) {
+        log.warn("Unable to place final IOException offer in queue");
+      } else {
+        log.debug("Placed IOException in queue");
+      }
+      log.debug(e, "Exception with queue length of %d and %d bytes available", queue.size(), byteCount.get());
+    }
+  }
+
+  public final long getByteCount()
+  {
+    return byteCount.get();
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/response/StatusResponseHandler.java b/java-util/src/main/java/io/druid/java/util/http/client/response/StatusResponseHandler.java
new file mode 100644
index 00000000000..c0b55e902e9
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/response/StatusResponseHandler.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.response;
+
+import org.jboss.netty.handler.codec.http.HttpChunk;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+
+import java.nio.charset.Charset;
+
+/**
+ */
+public class StatusResponseHandler implements HttpResponseHandler<StatusResponseHolder, StatusResponseHolder>
+{
+  private final Charset charset;
+
+  public StatusResponseHandler(Charset charset)
+  {
+    this.charset = charset;
+  }
+
+  @Override
+  public ClientResponse<StatusResponseHolder> handleResponse(HttpResponse response)
+  {
+    return ClientResponse.unfinished(
+        new StatusResponseHolder(
+            response.getStatus(),
+            new StringBuilder(response.getContent().toString(charset))
+        )
+    );
+  }
+
+  @Override
+  public ClientResponse<StatusResponseHolder> handleChunk(
+      ClientResponse<StatusResponseHolder> response,
+      HttpChunk chunk
+  )
+  {
+    final StringBuilder builder = response.getObj().getBuilder();
+
+    if (builder == null) {
+      return ClientResponse.finished(null);
+    }
+
+    builder.append(chunk.getContent().toString(charset));
+    return response;
+  }
+
+  @Override
+  public ClientResponse<StatusResponseHolder> done(ClientResponse<StatusResponseHolder> response)
+  {
+    return ClientResponse.finished(response.getObj());
+  }
+
+  @Override
+  public void exceptionCaught(
+      ClientResponse<StatusResponseHolder> clientResponse, Throwable e
+  )
+  {
+    // Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished
+  }
+
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/response/StatusResponseHolder.java b/java-util/src/main/java/io/druid/java/util/http/client/response/StatusResponseHolder.java
new file mode 100644
index 00000000000..58142e1a2c2
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/response/StatusResponseHolder.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.response;
+
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+/**
+ */
+public class StatusResponseHolder
+{
+  private final HttpResponseStatus status;
+  private final StringBuilder builder;
+
+  public StatusResponseHolder(
+      HttpResponseStatus status,
+      StringBuilder builder
+  )
+  {
+    this.status = status;
+    this.builder = builder;
+  }
+
+  public HttpResponseStatus getStatus()
+  {
+    return status;
+  }
+
+  public StringBuilder getBuilder()
+  {
+    return builder;
+  }
+
+  public String getContent()
+  {
+    return builder.toString();
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/http/client/response/ToStringResponseHandler.java b/java-util/src/main/java/io/druid/java/util/http/client/response/ToStringResponseHandler.java
new file mode 100644
index 00000000000..2533b6b9080
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/http/client/response/ToStringResponseHandler.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.http.client.response;
+
+import org.jboss.netty.handler.codec.http.HttpChunk;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+
+import java.nio.charset.Charset;
+
+/**
+ */
+public class ToStringResponseHandler implements HttpResponseHandler<StringBuilder, String>
+{
+  private final Charset charset;
+
+  public ToStringResponseHandler(Charset charset)
+  {
+    this.charset = charset;
+  }
+
+  @Override
+  public ClientResponse<StringBuilder> handleResponse(HttpResponse response)
+  {
+    return ClientResponse.unfinished(new StringBuilder(response.getContent().toString(charset)));
+  }
+
+  @Override
+  public ClientResponse<StringBuilder> handleChunk(
+      ClientResponse<StringBuilder> response,
+      HttpChunk chunk
+  )
+  {
+    final StringBuilder builder = response.getObj();
+    if (builder == null) {
+      return ClientResponse.finished(null);
+    }
+
+    builder.append(chunk.getContent().toString(charset));
+    return response;
+  }
+
+  @Override
+  public ClientResponse<String> done(ClientResponse<StringBuilder> response)
+  {
+    final StringBuilder builder = response.getObj();
+    if (builder == null) {
+      return ClientResponse.finished(null);
+    }
+
+    return ClientResponse.finished(builder.toString());
+  }
+
+  @Override
+  public void exceptionCaught(
+      ClientResponse<StringBuilder> clientResponse, Throwable e
+  )
+  {
+    // Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished
+  }
+
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/AbstractMonitor.java b/java-util/src/main/java/io/druid/java/util/metrics/AbstractMonitor.java
new file mode 100644
index 00000000000..e3f757d619f
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/AbstractMonitor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.metrics;
+
+
+import io.druid.java.util.emitter.service.ServiceEmitter;
+
+/**
+ */
+public abstract class AbstractMonitor implements Monitor
+{
+  private volatile boolean started = false;
+
+  @Override
+  public void start()
+  {
+    started = true;
+  }
+
+  @Override
+  public void stop()
+  {
+    started = false;
+  }
+
+  @Override
+  public boolean monitor(ServiceEmitter emitter)
+  {
+    if (started) {
+      return doMonitor(emitter);
+    }
+
+    return false;
+  }
+
+  public abstract boolean doMonitor(ServiceEmitter emitter);
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/CgroupUtil.java b/java-util/src/main/java/io/druid/java/util/metrics/CgroupUtil.java
new file mode 100644
index 00000000000..a476b4f3159
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/CgroupUtil.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.metrics;
+
+import java.util.regex.Pattern;
+
+public class CgroupUtil
+{
+  public static final String SPACE_MATCH = Pattern.quote(" ");
+  public static final String COMMA_MATCH = Pattern.quote(",");
+  public static final String COLON_MATCH = Pattern.quote(":");
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/CompoundMonitor.java b/java-util/src/main/java/io/druid/java/util/metrics/CompoundMonitor.java
new file mode 100644
index 00000000000..1e1ca4634a0
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/CompoundMonitor.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.metrics;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+
+import java.util.Arrays;
+import java.util.List;
+
+public abstract class CompoundMonitor implements Monitor
+{
+  private final List<Monitor> monitors;
+
+  public CompoundMonitor(List<Monitor> monitors)
+  {
+    this.monitors = monitors;
+  }
+
+  public CompoundMonitor(Monitor... monitors)
+  {
+    this(Arrays.asList(monitors));
+  }
+
+  @Override
+  public void start()
+  {
+    for (Monitor monitor : monitors) {
+      monitor.start();
+    }
+  }
+
+  @Override
+  public void stop()
+  {
+    for (Monitor monitor : monitors) {
+      monitor.stop();
+    }
+  }
+
+  @Override
+  public boolean monitor(final ServiceEmitter emitter)
+  {
+    return shouldReschedule(Lists.transform(monitors,
+        new Function<Monitor, Boolean>()
+        {
+          @Override
+          public Boolean apply(Monitor monitor)
+          {
+            return monitor.monitor(emitter);
+          }
+        }
+    ));
+  }
+
+  public abstract boolean shouldReschedule(List<Boolean> reschedules);
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/CpuAcctDeltaMonitor.java b/java-util/src/main/java/io/druid/java/util/metrics/CpuAcctDeltaMonitor.java
new file mode 100644
index 00000000000..e3a21f1d720
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/CpuAcctDeltaMonitor.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.metrics;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
+import io.druid.java.util.metrics.cgroups.CgroupDiscoverer;
+import io.druid.java.util.metrics.cgroups.CpuAcct;
+import io.druid.java.util.metrics.cgroups.ProcSelfCgroupDiscoverer;
+import org.joda.time.DateTime;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class CpuAcctDeltaMonitor extends FeedDefiningMonitor
+{
+  private static final Logger log = new Logger(CpuAcctDeltaMonitor.class);
+  private final AtomicReference<SnapshotHolder> priorSnapshot = new AtomicReference<>(null);
+  private final Map<String, String[]> dimensions;
+
+  private final CgroupDiscoverer cgroupDiscoverer;
+
+  public CpuAcctDeltaMonitor()
+  {
+    this(ImmutableMap.of());
+  }
+
+  public CpuAcctDeltaMonitor(final Map<String, String[]> dimensions)
+  {
+    this(dimensions, DEFAULT_METRICS_FEED);
+  }
+
+  public CpuAcctDeltaMonitor(final Map<String, String[]> dimensions, final String feed)
+  {
+    this(feed, dimensions, new ProcSelfCgroupDiscoverer());
+  }
+
+  public CpuAcctDeltaMonitor(
+      String feed,
+      Map<String, String[]> dimensions,
+      CgroupDiscoverer cgroupDiscoverer
+  )
+  {
+    super(feed);
+    Preconditions.checkNotNull(dimensions);
+    this.dimensions = ImmutableMap.copyOf(dimensions);
+    this.cgroupDiscoverer = Preconditions.checkNotNull(cgroupDiscoverer, "cgroupDiscoverer required");
+  }
+
+  @Override
+  public boolean doMonitor(ServiceEmitter emitter)
+  {
+    final CpuAcct cpuAcct = new CpuAcct(cgroupDiscoverer);
+    final CpuAcct.CpuAcctMetric snapshot = cpuAcct.snapshot();
+    final long nanoTime = System.nanoTime(); // Approx time... may be influenced by an unlucky GC
+    final DateTime dateTime = DateTimes.nowUtc();
+    final SnapshotHolder priorSnapshotHolder = this.priorSnapshot.get();
+    if (!priorSnapshot.compareAndSet(priorSnapshotHolder, new SnapshotHolder(snapshot, nanoTime))) {
+      log.debug("Pre-empted by another monitor run");
+      return false;
+    }
+    if (priorSnapshotHolder == null) {
+      log.info("Detected first run, storing result for next run");
+      return false;
+    }
+    final long elapsedNs = nanoTime - priorSnapshotHolder.timestamp;
+    if (snapshot.cpuCount() != priorSnapshotHolder.metric.cpuCount()) {
+      log.warn(
+          "Prior CPU count [%d] does not match current cpu count [%d]. Skipping metrics emission",
+          priorSnapshotHolder.metric.cpuCount(),
+          snapshot.cpuCount()
+      );
+      return false;
+    }
+    for (int i = 0; i < snapshot.cpuCount(); ++i) {
+      final ServiceMetricEvent.Builder builderUsr = builder()
+          .setDimension("cpuName", Integer.toString(i))
+          .setDimension("cpuTime", "usr");
+      final ServiceMetricEvent.Builder builderSys = builder()
+          .setDimension("cpuName", Integer.toString(i))
+          .setDimension("cpuTime", "sys");
+      MonitorUtils.addDimensionsToBuilder(builderUsr, dimensions);
+      MonitorUtils.addDimensionsToBuilder(builderSys, dimensions);
+      emitter.emit(builderUsr.build(
+          dateTime,
+          "cgroup/cpu_time_delta_ns",
+          snapshot.usrTime(i) - priorSnapshotHolder.metric.usrTime(i)
+      ));
+      emitter.emit(builderSys.build(
+          dateTime,
+          "cgroup/cpu_time_delta_ns",
+          snapshot.sysTime(i) - priorSnapshotHolder.metric.sysTime(i)
+      ));
+    }
+    if (snapshot.cpuCount() > 0) {
+      // Don't bother emitting metrics if there aren't actually any cpus (usually from error)
+      emitter.emit(builder().build(dateTime, "cgroup/cpu_time_delta_ns_elapsed", elapsedNs));
+    }
+    return true;
+  }
+
+  static class SnapshotHolder
+  {
+    private final CpuAcct.CpuAcctMetric metric;
+    private final long timestamp;
+
+    SnapshotHolder(CpuAcct.CpuAcctMetric metric, long timestamp)
+    {
+      this.metric = metric;
+      this.timestamp = timestamp;
+    }
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/FeedDefiningMonitor.java b/java-util/src/main/java/io/druid/java/util/metrics/FeedDefiningMonitor.java
new file mode 100644
index 00000000000..aa3d2863d5b
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/FeedDefiningMonitor.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.metrics;
+
+import com.google.common.base.Preconditions;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
+
+public abstract class FeedDefiningMonitor extends AbstractMonitor
+{
+  public static final String DEFAULT_METRICS_FEED = "metrics";
+  protected final String feed;
+
+  public FeedDefiningMonitor(String feed)
+  {
+    Preconditions.checkNotNull(feed);
+    this.feed = feed;
+  }
+
+  protected ServiceMetricEvent.Builder builder()
+  {
+    return ServiceMetricEvent.builder().setFeed(feed);
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/HttpPostEmitterMonitor.java b/java-util/src/main/java/io/druid/java/util/metrics/HttpPostEmitterMonitor.java
new file mode 100644
index 00000000000..590eabda17d
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/HttpPostEmitterMonitor.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.metrics;
+
+import com.google.common.collect.ImmutableMap;
+import io.druid.java.util.emitter.core.ConcurrentTimeCounter;
+import io.druid.java.util.emitter.core.HttpPostEmitter;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
+
+public class HttpPostEmitterMonitor extends FeedDefiningMonitor
+{
+  private final HttpPostEmitter httpPostEmitter;
+  private final ImmutableMap<String, String> extraDimensions;
+  private final ServiceMetricEvent.Builder builder;
+  private long lastTotalEmittedEvents = 0;
+  private int lastDroppedBuffers = 0;
+
+  public HttpPostEmitterMonitor(
+      String feed,
+      HttpPostEmitter httpPostEmitter,
+      ImmutableMap<String, String> extraDimensions
+  )
+  {
+    super(feed);
+    this.httpPostEmitter = httpPostEmitter;
+    this.extraDimensions = extraDimensions;
+    this.builder = builder();
+  }
+
+  @Override
+  public boolean doMonitor(ServiceEmitter emitter)
+  {
+    long newTotalEmittedEvents = httpPostEmitter.getTotalEmittedEvents();
+    long totalEmittedEventsDiff = newTotalEmittedEvents - lastTotalEmittedEvents;
+    emitter.emit(builder.build("emitter/events/emitted", totalEmittedEventsDiff));
+    lastTotalEmittedEvents = newTotalEmittedEvents;
+
+    int newDroppedBuffers = httpPostEmitter.getDroppedBuffers();
+    int droppedBuffersDiff = newDroppedBuffers - lastDroppedBuffers;
+    emitter.emit(builder.build("emitter/buffers/dropped", droppedBuffersDiff));
+    lastDroppedBuffers = newDroppedBuffers;
+
+    emitTimeCounterMetrics(emitter, httpPostEmitter.getBatchFillingTimeCounter(), "emitter/batchFilling/");
+    emitTimeCounterMetrics(emitter, httpPostEmitter.getSuccessfulSendingTimeCounter(), "emitter/successfulSending/");
+    emitTimeCounterMetrics(emitter, httpPostEmitter.getFailedSendingTimeCounter(), "emitter/failedSending/");
+
+    emitter.emit(builder.build("emitter/events/emitQueue", httpPostEmitter.getEventsToEmit()));
+    emitter.emit(builder.build("emitter/events/large/emitQueue", httpPostEmitter.getLargeEventsToEmit()));
+    emitter.emit(builder.build("emitter/buffers/totalAllocated", httpPostEmitter.getTotalAllocatedBuffers()));
+    emitter.emit(builder.build("emitter/buffers/emitQueue", httpPostEmitter.getBuffersToEmit()));
+    emitter.emit(builder.build("emitter/buffers/failed", httpPostEmitter.getFailedBuffers()));
+    emitter.emit(builder.build("emitter/buffers/reuseQueue", httpPostEmitter.getBuffersToReuse()));
+
+    return true;
+  }
+
+  private void emitTimeCounterMetrics(ServiceEmitter emitter, ConcurrentTimeCounter timeCounter, String metricNameBase)
+  {
+    long timeSumAndCount = timeCounter.getTimeSumAndCountAndReset();
+    emitter.emit(builder.build(metricNameBase + "timeMsSum", ConcurrentTimeCounter.timeSum(timeSumAndCount)));
+    emitter.emit(builder.build(metricNameBase + "count", ConcurrentTimeCounter.count(timeSumAndCount)));
+    emitter.emit(builder.build(metricNameBase + "maxTimeMs", timeCounter.getAndResetMaxTime()));
+    emitter.emit(builder.build(metricNameBase + "minTimeMs", timeCounter.getAndResetMinTime()));
+  }
+
+  @Override
+  protected ServiceMetricEvent.Builder builder()
+  {
+    ServiceMetricEvent.Builder builder = super.builder();
+    extraDimensions.forEach(builder::setDimension);
+    return builder;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/JvmCpuMonitor.java b/java-util/src/main/java/io/druid/java/util/metrics/JvmCpuMonitor.java
new file mode 100644
index 00000000000..fcefa3d9bfd
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/JvmCpuMonitor.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.metrics;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.hyperic.sigar.ProcCpu;
+import org.hyperic.sigar.Sigar;
+import org.hyperic.sigar.SigarException;
+
+import java.util.Map;
+
+public class JvmCpuMonitor extends FeedDefiningMonitor
+{
+  private static final Logger log = new Logger(JvmCpuMonitor.class);
+
+  private final Sigar sigar = SigarUtil.getSigar();
+  private final long currentProcessId = sigar.getPid();
+
+  private final KeyedDiff diff = new KeyedDiff();
+
+  private Map<String, String[]> dimensions;
+
+  public JvmCpuMonitor()
+  {
+    this(ImmutableMap.<String, String[]>of());
+  }
+
+  public JvmCpuMonitor(Map<String, String[]> dimensions)
+  {
+    this(dimensions, DEFAULT_METRICS_FEED);
+  }
+
+  public JvmCpuMonitor(Map<String, String[]> dimensions, String feed)
+  {
+    super(feed);
+    Preconditions.checkNotNull(dimensions);
+    this.dimensions = ImmutableMap.copyOf(dimensions);
+  }
+
+  @Override
+  public boolean doMonitor(ServiceEmitter emitter)
+  {
+    // process CPU
+    try {
+      ProcCpu procCpu = sigar.getProcCpu(currentProcessId);
+      final ServiceMetricEvent.Builder builder = builder();
+      MonitorUtils.addDimensionsToBuilder(builder, dimensions);
+      // delta for total, sys, user
+      Map<String, Long> procDiff = diff.to(
+          "proc/cpu", ImmutableMap.of(
+              "jvm/cpu/total", procCpu.getTotal(),
+              "jvm/cpu/sys", procCpu.getSys(),
+              "jvm/cpu/user", procCpu.getUser()
+          )
+      );
+      if (procDiff != null) {
+        for (Map.Entry<String, Long> entry : procDiff.entrySet()) {
+          emitter.emit(builder.build(entry.getKey(), entry.getValue()));
+        }
+      }
+      emitter.emit(builder.build("jvm/cpu/percent", procCpu.getPercent()));
+    }
+    catch (SigarException e) {
+      log.error(e, "Failed to get ProcCpu");
+    }
+    return true;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/JvmMonitor.java b/java-util/src/main/java/io/druid/java/util/metrics/JvmMonitor.java
new file mode 100644
index 00000000000..6f696be05be
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/JvmMonitor.java
@@ -0,0 +1,315 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.metrics;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.gridkit.lab.jvm.perfdata.JStatData;
+import org.gridkit.lab.jvm.perfdata.JStatData.LongCounter;
+import org.gridkit.lab.jvm.perfdata.JStatData.StringCounter;
+import org.gridkit.lab.jvm.perfdata.JStatData.TickCounter;
+
+import java.lang.management.BufferPoolMXBean;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryPoolMXBean;
+import java.lang.management.MemoryType;
+import java.lang.management.MemoryUsage;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class JvmMonitor extends FeedDefiningMonitor
+{
+  private final Map<String, String[]> dimensions;
+  private final long pid;
+
+  private final GcCounters gcCounters = new GcCounters();
+
+  public JvmMonitor()
+  {
+    this(ImmutableMap.of());
+  }
+
+  public JvmMonitor(Map<String, String[]> dimensions)
+  {
+    this(dimensions, DEFAULT_METRICS_FEED);
+  }
+
+  public JvmMonitor(Map<String, String[]> dimensions, String feed)
+  {
+    this(dimensions, feed, JvmPidDiscoverer.instance());
+  }
+
+  public JvmMonitor(Map<String, String[]> dimensions, String feed, PidDiscoverer pidDiscoverer)
+  {
+    super(feed);
+    Preconditions.checkNotNull(dimensions);
+    this.dimensions = ImmutableMap.copyOf(dimensions);
+    this.pid = Preconditions.checkNotNull(pidDiscoverer).getPid();
+  }
+
+  @Override
+  public boolean doMonitor(ServiceEmitter emitter)
+  {
+    emitJvmMemMetrics(emitter);
+    emitDirectMemMetrics(emitter);
+    emitGcMetrics(emitter);
+
+    return true;
+  }
+
+  /**
+   * These metrics are going to be replaced by new jvm/gc/mem/* metrics
+   */
+  @Deprecated
+  private void emitJvmMemMetrics(ServiceEmitter emitter)
+  {
+    // I have no idea why, but jvm/mem is slightly more than the sum of jvm/pool. Let's just include
+    // them both.
+    final Map<String, MemoryUsage> usages = ImmutableMap.of(
+        "heap", ManagementFactory.getMemoryMXBean().getHeapMemoryUsage(),
+        "nonheap", ManagementFactory.getMemoryMXBean().getNonHeapMemoryUsage()
+    );
+    for (Map.Entry<String, MemoryUsage> entry : usages.entrySet()) {
+      final String kind = entry.getKey();
+      final MemoryUsage usage = entry.getValue();
+      final ServiceMetricEvent.Builder builder = builder().setDimension("memKind", kind);
+      MonitorUtils.addDimensionsToBuilder(builder, dimensions);
+
+      emitter.emit(builder.build("jvm/mem/max", usage.getMax()));
+      emitter.emit(builder.build("jvm/mem/committed", usage.getCommitted()));
+      emitter.emit(builder.build("jvm/mem/used", usage.getUsed()));
+      emitter.emit(builder.build("jvm/mem/init", usage.getInit()));
+    }
+
+    // jvm/pool
+    for (MemoryPoolMXBean pool : ManagementFactory.getMemoryPoolMXBeans()) {
+      final String kind = pool.getType() == MemoryType.HEAP ? "heap" : "nonheap";
+      final MemoryUsage usage = pool.getUsage();
+      final ServiceMetricEvent.Builder builder = builder()
+          .setDimension("poolKind", kind)
+          .setDimension("poolName", pool.getName());
+      MonitorUtils.addDimensionsToBuilder(builder, dimensions);
+
+      emitter.emit(builder.build("jvm/pool/max", usage.getMax()));
+      emitter.emit(builder.build("jvm/pool/committed", usage.getCommitted()));
+      emitter.emit(builder.build("jvm/pool/used", usage.getUsed()));
+      emitter.emit(builder.build("jvm/pool/init", usage.getInit()));
+    }
+  }
+
+  private void emitDirectMemMetrics(ServiceEmitter emitter)
+  {
+    for (BufferPoolMXBean pool : ManagementFactory.getPlatformMXBeans(BufferPoolMXBean.class)) {
+      final ServiceMetricEvent.Builder builder = builder().setDimension("bufferpoolName", pool.getName());
+      MonitorUtils.addDimensionsToBuilder(builder, dimensions);
+
+      emitter.emit(builder.build("jvm/bufferpool/capacity", pool.getTotalCapacity()));
+      emitter.emit(builder.build("jvm/bufferpool/used", pool.getMemoryUsed()));
+      emitter.emit(builder.build("jvm/bufferpool/count", pool.getCount()));
+    }
+  }
+
+  private void emitGcMetrics(ServiceEmitter emitter)
+  {
+    gcCounters.emit(emitter, dimensions);
+  }
+
+  /**
+   * The following GC-related code is partially based on
+   * https://github.com/aragozin/jvm-tools/blob/e0e37692648951440aa1a4ea5046261cb360df70/
+   * sjk-core/src/main/java/org/gridkit/jvmtool/PerfCounterGcCpuUsageMonitor.java
+   */
+  private class GcCounters
+  {
+    private final List<GcGeneration> generations = new ArrayList<>();
+
+    GcCounters()
+    {
+      // connect to itself
+      final JStatData jStatData = JStatData.connect(pid);
+      final Map<String, JStatData.Counter<?>> jStatCounters = jStatData.getAllCounters();
+
+      generations.add(new GcGeneration(jStatCounters, 0, "young"));
+      generations.add(new GcGeneration(jStatCounters, 1, "old"));
+      // Removed in Java 8 but still actual for previous Java versions
+      if (jStatCounters.containsKey("sun.gc.generation.2.name")) {
+        generations.add(new GcGeneration(jStatCounters, 2, "perm"));
+      }
+    }
+
+    void emit(ServiceEmitter emitter, Map<String, String[]> dimensions)
+    {
+      for (GcGeneration generation : generations) {
+        generation.emit(emitter, dimensions);
+      }
+    }
+  }
+
+  private class GcGeneration
+  {
+    private final String name;
+    private final GcGenerationCollector collector;
+    private final List<GcGenerationSpace> spaces = new ArrayList<>();
+
+    GcGeneration(Map<String, JStatData.Counter<?>> jStatCounters, long genIndex, String name)
+    {
+      this.name = StringUtils.toLowerCase(name);
+
+      long spacesCount = ((JStatData.LongCounter) jStatCounters.get(
+          StringUtils.format("sun.gc.generation.%d.spaces", genIndex)
+      )).getLong();
+      for (long spaceIndex = 0; spaceIndex < spacesCount; spaceIndex++) {
+        spaces.add(new GcGenerationSpace(jStatCounters, genIndex, spaceIndex));
+      }
+
+      if (jStatCounters.containsKey(StringUtils.format("sun.gc.collector.%d.name", genIndex))) {
+        collector = new GcGenerationCollector(jStatCounters, genIndex);
+      } else {
+        collector = null;
+      }
+    }
+
+    void emit(ServiceEmitter emitter, Map<String, String[]> dimensions)
+    {
+      ImmutableMap.Builder<String, String[]> dimensionsCopyBuilder = ImmutableMap
+          .<String, String[]>builder()
+          .putAll(dimensions)
+          .put("gcGen", new String[]{name});
+
+      if (collector != null) {
+        dimensionsCopyBuilder.put("gcName", new String[]{collector.name});
+      }
+
+      Map<String, String[]> dimensionsCopy = dimensionsCopyBuilder.build();
+
+      if (collector != null) {
+        collector.emit(emitter, dimensionsCopy);
+      }
+
+      for (GcGenerationSpace space : spaces) {
+        space.emit(emitter, dimensionsCopy);
+      }
+    }
+  }
+
+  private class GcGenerationCollector
+  {
+    private final String name;
+    private final LongCounter invocationsCounter;
+    private final TickCounter cpuCounter;
+    private long lastInvocations = 0;
+    private long lastCpuNanos = 0;
+
+    GcGenerationCollector(Map<String, JStatData.Counter<?>> jStatCounters, long genIndex)
+    {
+      String collectorKeyPrefix = StringUtils.format("sun.gc.collector.%d", genIndex);
+
+      String nameKey = StringUtils.format("%s.name", collectorKeyPrefix);
+      StringCounter nameCounter = (StringCounter) jStatCounters.get(nameKey);
+      name = getReadableName(nameCounter.getString());
+
+      invocationsCounter = (LongCounter) jStatCounters.get(StringUtils.format("%s.invocations", collectorKeyPrefix));
+      cpuCounter = (TickCounter) jStatCounters.get(StringUtils.format("%s.time", collectorKeyPrefix));
+    }
+
+    void emit(ServiceEmitter emitter, Map<String, String[]> dimensions)
+    {
+      final ServiceMetricEvent.Builder builder = builder();
+      MonitorUtils.addDimensionsToBuilder(builder, dimensions);
+
+      long newInvocations = invocationsCounter.getLong();
+      emitter.emit(builder.build("jvm/gc/count", newInvocations - lastInvocations));
+      lastInvocations = newInvocations;
+
+      long newCpuNanos = cpuCounter.getNanos();
+      emitter.emit(builder.build("jvm/gc/cpu", newCpuNanos - lastCpuNanos));
+      lastCpuNanos = newCpuNanos;
+    }
+
+    private String getReadableName(String name)
+    {
+      switch (name) {
+        // Young gen
+        case "Copy":
+          return "serial";
+        case "PSScavenge":
+          return "parallel";
+        case "PCopy":
+          return "cms";
+        case "G1 incremental collections":
+          return "g1";
+
+        // Old gen
+        case "MCS":
+          return "serial";
+        case "PSParallelCompact":
+          return "parallel";
+        case "CMS":
+          return "cms";
+        case "G1 stop-the-world full collections":
+          return "g1";
+
+        default:
+          return name;
+      }
+    }
+  }
+
+  private class GcGenerationSpace
+  {
+    private final String name;
+
+    private final LongCounter maxCounter;
+    private final LongCounter capacityCounter;
+    private final LongCounter usedCounter;
+    private final LongCounter initCounter;
+
+    GcGenerationSpace(Map<String, JStatData.Counter<?>> jStatCounters, long genIndex, long spaceIndex)
+    {
+      String spaceKeyPrefix = StringUtils.format("sun.gc.generation.%d.space.%d", genIndex, spaceIndex);
+
+      String nameKey = StringUtils.format("%s.name", spaceKeyPrefix);
+      StringCounter nameCounter = (StringCounter) jStatCounters.get(nameKey);
+      name = StringUtils.toLowerCase(nameCounter.toString());
+
+      maxCounter = (LongCounter) jStatCounters.get(StringUtils.format("%s.maxCapacity", spaceKeyPrefix));
+      capacityCounter = (LongCounter) jStatCounters.get(StringUtils.format("%s.capacity", spaceKeyPrefix));
+      usedCounter = (LongCounter) jStatCounters.get(StringUtils.format("%s.used", spaceKeyPrefix));
+      initCounter = (LongCounter) jStatCounters.get(StringUtils.format("%s.initCapacity", spaceKeyPrefix));
+    }
+
+    void emit(ServiceEmitter emitter, Map<String, String[]> dimensions)
+    {
+      final ServiceMetricEvent.Builder builder = builder();
+      MonitorUtils.addDimensionsToBuilder(builder, dimensions);
+
+      builder.setDimension("gcGenSpaceName", name);
+
+      emitter.emit(builder.build("jvm/gc/mem/max", maxCounter.getLong()));
+      emitter.emit(builder.build("jvm/gc/mem/capacity", capacityCounter.getLong()));
+      emitter.emit(builder.build("jvm/gc/mem/used", usedCounter.getLong()));
+      emitter.emit(builder.build("jvm/gc/mem/init", initCounter.getLong()));
+    }
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/JvmPidDiscoverer.java b/java-util/src/main/java/io/druid/java/util/metrics/JvmPidDiscoverer.java
new file mode 100644
index 00000000000..b279f43e18a
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/JvmPidDiscoverer.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.metrics;
+
+import io.druid.java.util.common.RE;
+
+import java.lang.management.ManagementFactory;
+import java.util.regex.Pattern;
+
+/**
+ * For systems that for whatever reason cannot use Sigar (through io.druid.java.util.metrics.SigarPidDiscoverer ),
+ * this attempts to get the PID from the JVM "name".
+ */
+public class JvmPidDiscoverer implements PidDiscoverer
+{
+  private static final JvmPidDiscoverer INSTANCE = new JvmPidDiscoverer();
+
+  public static JvmPidDiscoverer instance()
+  {
+    return INSTANCE;
+  }
+
+  /**
+   * use {JvmPidDiscoverer.instance()}
+   */
+  private JvmPidDiscoverer()
+  {
+  }
+
+  /**
+   * Returns the PID as a best guess. This uses methods that are not guaranteed to actually be the PID.
+   * <p>
+   * TODO: switch to ProcessHandle.current().getPid() for java9 potentially
+   *
+   * @return the PID of the current jvm if available
+   *
+   * @throws RuntimeException if the pid cannot be determined
+   */
+  @Override
+  public long getPid()
+  {
+    return Inner.PID;
+  }
+
+  private static class Inner
+  {
+    private static final long PID;
+
+    static {
+      final String jvmName = ManagementFactory.getRuntimeMXBean().getName();
+      final String[] nameSplits = jvmName.split(Pattern.quote("@"));
+      if (nameSplits.length != 2) {
+        throw new RE("Unable to determine pid from [%s]", jvmName);
+      }
+      try {
+        PID = Long.parseLong(nameSplits[0]);
+      }
+      catch (NumberFormatException nfe) {
+        throw new RE(nfe, "Unable to determine pid from [%s]", jvmName);
+      }
+    }
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/JvmThreadsMonitor.java b/java-util/src/main/java/io/druid/java/util/metrics/JvmThreadsMonitor.java
new file mode 100644
index 00000000000..60efeb00447
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/JvmThreadsMonitor.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.metrics;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
+
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadMXBean;
+import java.util.Map;
+
+public class JvmThreadsMonitor extends FeedDefiningMonitor
+{
+  private final Map<String, String[]> dimensions;
+
+  private int lastLiveThreads = 0;
+  private long lastStartedThreads = 0;
+
+  public JvmThreadsMonitor()
+  {
+    this(ImmutableMap.<String, String[]>of());
+  }
+
+  public JvmThreadsMonitor(Map<String, String[]> dimensions)
+  {
+    this(dimensions, DEFAULT_METRICS_FEED);
+  }
+
+  public JvmThreadsMonitor(Map<String, String[]> dimensions, String feed)
+  {
+    super(feed);
+    Preconditions.checkNotNull(dimensions);
+    this.dimensions = ImmutableMap.copyOf(dimensions);
+  }
+
+  @Override
+  public boolean doMonitor(ServiceEmitter emitter)
+  {
+    ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+
+    final ServiceMetricEvent.Builder builder = builder();
+    MonitorUtils.addDimensionsToBuilder(builder, dimensions);
+
+    // Because between next two calls on ThreadMXBean new threads can be started we can observe some inconsistency
+    // in counters values and finished counter could be even negative
+    int newLiveThreads = threadBean.getThreadCount();
+    long newStartedThreads = threadBean.getTotalStartedThreadCount();
+
+    long startedThreadsDiff = newStartedThreads - lastStartedThreads;
+
+    emitter.emit(builder.build("jvm/threads/started", startedThreadsDiff));
+    emitter.emit(builder.build("jvm/threads/finished", lastLiveThreads + startedThreadsDiff - newLiveThreads));
+    emitter.emit(builder.build("jvm/threads/live", newLiveThreads));
+    emitter.emit(builder.build("jvm/threads/liveDaemon", threadBean.getDaemonThreadCount()));
+
+    emitter.emit(builder.build("jvm/threads/livePeak", threadBean.getPeakThreadCount()));
+    threadBean.resetPeakThreadCount();
+
+    lastStartedThreads = newStartedThreads;
+    lastLiveThreads = newLiveThreads;
+
+    return true;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/KeyedDiff.java b/java-util/src/main/java/io/druid/java/util/metrics/KeyedDiff.java
new file mode 100644
index 00000000000..7378071f719
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/KeyedDiff.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.java.util.metrics;
+
+import io.druid.java.util.common.logger.Logger;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class KeyedDiff
+{
+  private static final Logger log = new Logger(KeyedDiff.class);
+
+  private final Map<String, Map<String, Long>> prevs = new HashMap<String, Map<String, Long>>();
+
+  public Map<String, Long> to(String key, Map<String, Long> curr)
+  {
+    final Map<String, Long> prev = prevs.put(key, curr);
+    if (prev != null) {
+      return subtract(curr, prev);
+    } else {
+      log.debug("No previous data for key[%s]", key);
+      return null;
+    }
+  }
+
+  public static Map<String, Long> subtract(Map<String, Long> xs, Map<String, Long> ys)
+  {
+    assert xs.keySet().equals(ys.keySet());
+    final Map<String, Long> zs = new HashMap<String, Long>();
+    for (String k : xs.keySet()) {
+      zs.put(k, xs.get(k) - ys.get(k));
+    }
+    return zs;
+  }
+}
diff --git a/java-util/src/main/java/io/druid/java/util/metrics/Monitor.java b/java-util/src/main/java/io/druid/java/util/metrics/Monitor.java
new file mode 100644
index 00000000000..666477b108b
--- /dev/null
+++ b/java-util/src/main/java/io/druid/java/util/metrics/Monitor.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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

  (This diff was longer than 20,000 lines, and has been truncated...)


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@xxxxxxxxxxxxxxxx


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@xxxxxxxxxxxxxxxx
For additional commands, e-mail: dev-help@xxxxxxxxxxxxxxxx