Merge pull request #524 from metamx/fix-router2

Small fixes to router
diff --git a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java
index 60cc710..eb5bf46 100644
--- a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java
+++ b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java
@@ -24,9 +24,8 @@
 import com.google.common.collect.Lists;
 import com.metamx.common.guava.Sequence;
 import com.metamx.common.guava.Sequences;
-import io.druid.query.aggregation.FinalizeMetricManipulationFn;
-import io.druid.query.aggregation.IdentityMetricManipulationFn;
 import io.druid.query.aggregation.MetricManipulationFn;
+import io.druid.query.aggregation.MetricManipulatorFns;
 
 import javax.annotation.Nullable;
 
@@ -58,11 +57,11 @@
 
     if (shouldFinalize) {
       queryToRun = query.withOverriddenContext(ImmutableMap.<String, Object>of("finalize", false));
-      metricManipulationFn = new FinalizeMetricManipulationFn();
+      metricManipulationFn = MetricManipulatorFns.finalizing();
 
     } else {
       queryToRun = query;
-      metricManipulationFn = new IdentityMetricManipulationFn();
+      metricManipulationFn = MetricManipulatorFns.identity();
     }
     if (isBySegment) {
       finalizerFn = new Function<T, T>()
diff --git a/processing/src/main/java/io/druid/query/ReflectionLoaderThingy.java b/processing/src/main/java/io/druid/query/ReflectionLoaderThingy.java
index 7563655..f7c7d91 100644
--- a/processing/src/main/java/io/druid/query/ReflectionLoaderThingy.java
+++ b/processing/src/main/java/io/druid/query/ReflectionLoaderThingy.java
@@ -27,6 +27,7 @@
 
 /**
  */
+@Deprecated
 public class ReflectionLoaderThingy<T>
 {
   private static final Logger log = new Logger(ReflectionLoaderThingy.class);
diff --git a/processing/src/main/java/io/druid/query/ReflectionQueryToolChestWarehouse.java b/processing/src/main/java/io/druid/query/ReflectionQueryToolChestWarehouse.java
index 37ad0e9..569a0d5 100644
--- a/processing/src/main/java/io/druid/query/ReflectionQueryToolChestWarehouse.java
+++ b/processing/src/main/java/io/druid/query/ReflectionQueryToolChestWarehouse.java
@@ -21,6 +21,7 @@
 
 /**
  */
+@Deprecated
 public class ReflectionQueryToolChestWarehouse implements QueryToolChestWarehouse
 {
   ReflectionLoaderThingy<QueryToolChest> loader = ReflectionLoaderThingy.create(QueryToolChest.class);
diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java
index 809a903c..61617c4 100644
--- a/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java
+++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java
@@ -81,5 +81,4 @@
     }
     return new Pair(condensedAggs, condensedPostAggs);
   }
-
 }
diff --git a/processing/src/main/java/io/druid/query/aggregation/Aggregators.java b/processing/src/main/java/io/druid/query/aggregation/Aggregators.java
new file mode 100644
index 0000000..bbfc1bc
--- /dev/null
+++ b/processing/src/main/java/io/druid/query/aggregation/Aggregators.java
@@ -0,0 +1,105 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright (C) 2012, 2013  Metamarkets Group Inc.
+ *
+ * This program is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU General Public License
+ * as published by the Free Software Foundation; either version 2
+ * of the License, or (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
+ */
+
+package io.druid.query.aggregation;
+
+import java.nio.ByteBuffer;
+
+/**
+ */
+public class Aggregators
+{
+  public static Aggregator noopAggregator()
+  {
+    return new Aggregator()
+    {
+      @Override
+      public void aggregate()
+      {
+
+      }
+
+      @Override
+      public void reset()
+      {
+
+      }
+
+      @Override
+      public Object get()
+      {
+        return null;
+      }
+
+      @Override
+      public float getFloat()
+      {
+        return 0;
+      }
+
+      @Override
+      public String getName()
+      {
+        return null;
+      }
+
+      @Override
+      public void close()
+      {
+
+      }
+    };
+  }
+
+  public static BufferAggregator noopBufferAggregator()
+  {
+    return new BufferAggregator()
+    {
+      @Override
+      public void init(ByteBuffer buf, int position)
+      {
+
+      }
+
+      @Override
+      public void aggregate(ByteBuffer buf, int position)
+      {
+
+      }
+
+      @Override
+      public Object get(ByteBuffer buf, int position)
+      {
+        return null;
+      }
+
+      @Override
+      public float getFloat(ByteBuffer buf, int position)
+      {
+        return 0;
+      }
+
+      @Override
+      public void close()
+      {
+
+      }
+    };
+  }
+}
diff --git a/processing/src/main/java/io/druid/query/aggregation/FinalizeMetricManipulationFn.java b/processing/src/main/java/io/druid/query/aggregation/FinalizeMetricManipulationFn.java
deleted file mode 100644
index e532421..0000000
--- a/processing/src/main/java/io/druid/query/aggregation/FinalizeMetricManipulationFn.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Druid - a distributed column store.
- * Copyright (C) 2012, 2013  Metamarkets Group Inc.
- *
- * This program is free software; you can redistribute it and/or
- * modify it under the terms of the GNU General Public License
- * as published by the Free Software Foundation; either version 2
- * of the License, or (at your option) any later version.
- *
- * This program is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
- * GNU General Public License for more details.
- *
- * You should have received a copy of the GNU General Public License
- * along with this program; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
- */
-
-package io.druid.query.aggregation;
-
-/**
- */
-public class FinalizeMetricManipulationFn implements MetricManipulationFn
-{
-  @Override
-  public Object manipulate(AggregatorFactory factory, Object object)
-  {
-    return factory.finalizeComputation(object);
-  }
-}
diff --git a/processing/src/main/java/io/druid/query/aggregation/IdentityMetricManipulationFn.java b/processing/src/main/java/io/druid/query/aggregation/IdentityMetricManipulationFn.java
deleted file mode 100644
index 6b99838..0000000
--- a/processing/src/main/java/io/druid/query/aggregation/IdentityMetricManipulationFn.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Druid - a distributed column store.
- * Copyright (C) 2012, 2013  Metamarkets Group Inc.
- *
- * This program is free software; you can redistribute it and/or
- * modify it under the terms of the GNU General Public License
- * as published by the Free Software Foundation; either version 2
- * of the License, or (at your option) any later version.
- *
- * This program is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
- * GNU General Public License for more details.
- *
- * You should have received a copy of the GNU General Public License
- * along with this program; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
- */
-
-package io.druid.query.aggregation;
-
-/**
- */
-public class IdentityMetricManipulationFn implements MetricManipulationFn
-{
-  @Override
-  public Object manipulate(AggregatorFactory factory, Object object)
-  {
-    return object;
-  }
-}
diff --git a/processing/src/main/java/io/druid/query/aggregation/MetricManipulatorFns.java b/processing/src/main/java/io/druid/query/aggregation/MetricManipulatorFns.java
new file mode 100644
index 0000000..4872aac
--- /dev/null
+++ b/processing/src/main/java/io/druid/query/aggregation/MetricManipulatorFns.java
@@ -0,0 +1,42 @@
+package io.druid.query.aggregation;
+
+/**
+ */
+public class MetricManipulatorFns
+{
+  public static MetricManipulationFn identity()
+  {
+    return new MetricManipulationFn()
+    {
+      @Override
+      public Object manipulate(AggregatorFactory factory, Object object)
+      {
+        return object;
+      }
+    };
+  }
+
+  public static MetricManipulationFn finalizing()
+  {
+    return new MetricManipulationFn()
+    {
+      @Override
+      public Object manipulate(AggregatorFactory factory, Object object)
+      {
+        return factory.finalizeComputation(object);
+      }
+    };
+  }
+
+  public static MetricManipulationFn deserializing()
+  {
+    return new MetricManipulationFn()
+    {
+      @Override
+      public Object manipulate(AggregatorFactory factory, Object object)
+      {
+        return factory.deserialize(object);
+      }
+    };
+  }
+}
diff --git a/processing/src/main/java/io/druid/query/aggregation/NoopAggregator.java b/processing/src/main/java/io/druid/query/aggregation/NoopAggregator.java
deleted file mode 100644
index 7e2fc7e..0000000
--- a/processing/src/main/java/io/druid/query/aggregation/NoopAggregator.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Druid - a distributed column store.
- * Copyright (C) 2012, 2013  Metamarkets Group Inc.
- *
- * This program is free software; you can redistribute it and/or
- * modify it under the terms of the GNU General Public License
- * as published by the Free Software Foundation; either version 2
- * of the License, or (at your option) any later version.
- *
- * This program is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
- * GNU General Public License for more details.
- *
- * You should have received a copy of the GNU General Public License
- * along with this program; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
- */
-
-package io.druid.query.aggregation;
-
-/**
- */
-public class NoopAggregator implements Aggregator
-{
-  private final String name;
-
-  public NoopAggregator(
-      String name
-  )
-  {
-    this.name = name;
-  }
-
-  @Override
-  public void aggregate()
-  {
-  }
-
-  @Override
-  public void reset()
-  {
-  }
-
-  @Override
-  public Object get()
-  {
-    return null;
-  }
-
-  @Override
-  public float getFloat()
-  {
-    return 0;
-  }
-
-  @Override
-  public String getName()
-  {
-    return name;
-  }
-
-  @Override
-  public void close()
-  {
-    // no resources to cleanup
-  }
-}
diff --git a/processing/src/main/java/io/druid/query/aggregation/NoopBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/NoopBufferAggregator.java
deleted file mode 100644
index 7ff2d7c..0000000
--- a/processing/src/main/java/io/druid/query/aggregation/NoopBufferAggregator.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Druid - a distributed column store.
- * Copyright (C) 2012, 2013  Metamarkets Group Inc.
- *
- * This program is free software; you can redistribute it and/or
- * modify it under the terms of the GNU General Public License
- * as published by the Free Software Foundation; either version 2
- * of the License, or (at your option) any later version.
- *
- * This program is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
- * GNU General Public License for more details.
- *
- * You should have received a copy of the GNU General Public License
- * along with this program; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
- */
-
-package io.druid.query.aggregation;
-
-import java.nio.ByteBuffer;
-
-/**
- */
-public class NoopBufferAggregator implements BufferAggregator
-{
-  @Override
-  public void init(ByteBuffer buf, int position)
-  {
-  }
-
-  @Override
-  public void aggregate(ByteBuffer buf, int position)
-  {
-  }
-
-  @Override
-  public Object get(ByteBuffer buf, int position)
-  {
-    return null;
-  }
-
-  @Override
-  public float getFloat(ByteBuffer buf, int position)
-  {
-    return 0;
-  }
-
-  @Override
-  public void close()
-  {
-    // no resources to cleanup
-  }
-}
diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java
index 5eb4552..fd4e36f 100644
--- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java
+++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java
@@ -25,9 +25,8 @@
 import com.metamx.common.IAE;
 import io.druid.query.aggregation.Aggregator;
 import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.aggregation.Aggregators;
 import io.druid.query.aggregation.BufferAggregator;
-import io.druid.query.aggregation.NoopAggregator;
-import io.druid.query.aggregation.NoopBufferAggregator;
 import io.druid.segment.ColumnSelectorFactory;
 import io.druid.segment.ObjectColumnSelector;
 import org.apache.commons.codec.binary.Base64;
@@ -71,7 +70,7 @@
     ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName);
 
     if (selector == null) {
-      return new NoopAggregator(name);
+      return Aggregators.noopAggregator();
     }
 
     if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) {
@@ -89,7 +88,7 @@
     ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName);
 
     if (selector == null) {
-      return new NoopBufferAggregator();
+      return Aggregators.noopBufferAggregator();
     }
 
     if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) {
diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java
index da1857b..28135a8 100644
--- a/server/src/main/java/io/druid/client/CachingClusteredClient.java
+++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java
@@ -53,8 +53,7 @@
 import io.druid.query.QueryToolChestWarehouse;
 import io.druid.query.Result;
 import io.druid.query.SegmentDescriptor;
-import io.druid.query.aggregation.AggregatorFactory;
-import io.druid.query.aggregation.MetricManipulationFn;
+import io.druid.query.aggregation.MetricManipulatorFns;
 import io.druid.query.spec.MultipleSpecificSegmentSpec;
 import io.druid.timeline.DataSegment;
 import io.druid.timeline.TimelineObjectHolder;
@@ -359,14 +358,7 @@
                                     ),
                                     toolChest.makePreComputeManipulatorFn(
                                         rewrittenQuery,
-                                        new MetricManipulationFn()
-                                        {
-                                          @Override
-                                          public Object manipulate(AggregatorFactory factory, Object object)
-                                          {
-                                            return factory.deserialize(object);
-                                          }
-                                        }
+                                        MetricManipulatorFns.deserializing()
                                     )
                                 ),
                                 new Runnable()
diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java
index 1194acf..ae994d1 100644
--- a/server/src/main/java/io/druid/client/DirectDruidClient.java
+++ b/server/src/main/java/io/druid/client/DirectDruidClient.java
@@ -49,8 +49,7 @@
 import io.druid.query.QueryToolChest;
 import io.druid.query.QueryToolChestWarehouse;
 import io.druid.query.Result;
-import io.druid.query.aggregation.AggregatorFactory;
-import io.druid.query.aggregation.MetricManipulationFn;
+import io.druid.query.aggregation.MetricManipulatorFns;
 import org.jboss.netty.handler.codec.http.HttpChunk;
 import org.jboss.netty.handler.codec.http.HttpHeaders;
 import org.jboss.netty.handler.codec.http.HttpResponse;
@@ -219,14 +218,7 @@
           retVal,
           toolChest.makePreComputeManipulatorFn(
               query,
-              new MetricManipulationFn()
-              {
-                @Override
-                public Object manipulate(AggregatorFactory factory, Object object)
-                {
-                  return factory.deserialize(object);
-                }
-              }
+              MetricManipulatorFns.deserializing()
           )
       );
     }