mirror of https://github.com/apache/druid.git
Merge pull request #514 from metamx/reduce-bloat
Reduce some bloat in the code base
This commit is contained in:
commit
411def5013
|
@ -24,9 +24,8 @@ import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.metamx.common.guava.Sequence;
|
import com.metamx.common.guava.Sequence;
|
||||||
import com.metamx.common.guava.Sequences;
|
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.MetricManipulationFn;
|
||||||
|
import io.druid.query.aggregation.MetricManipulatorFns;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
@ -58,11 +57,11 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
|
||||||
|
|
||||||
if (shouldFinalize) {
|
if (shouldFinalize) {
|
||||||
queryToRun = query.withOverriddenContext(ImmutableMap.<String, Object>of("finalize", false));
|
queryToRun = query.withOverriddenContext(ImmutableMap.<String, Object>of("finalize", false));
|
||||||
metricManipulationFn = new FinalizeMetricManipulationFn();
|
metricManipulationFn = MetricManipulatorFns.finalizing();
|
||||||
|
|
||||||
} else {
|
} else {
|
||||||
queryToRun = query;
|
queryToRun = query;
|
||||||
metricManipulationFn = new IdentityMetricManipulationFn();
|
metricManipulationFn = MetricManipulatorFns.identity();
|
||||||
}
|
}
|
||||||
if (isBySegment) {
|
if (isBySegment) {
|
||||||
finalizerFn = new Function<T, T>()
|
finalizerFn = new Function<T, T>()
|
||||||
|
|
|
@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@Deprecated
|
||||||
public class ReflectionLoaderThingy<T>
|
public class ReflectionLoaderThingy<T>
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(ReflectionLoaderThingy.class);
|
private static final Logger log = new Logger(ReflectionLoaderThingy.class);
|
||||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.query;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@Deprecated
|
||||||
public class ReflectionQueryToolChestWarehouse implements QueryToolChestWarehouse
|
public class ReflectionQueryToolChestWarehouse implements QueryToolChestWarehouse
|
||||||
{
|
{
|
||||||
ReflectionLoaderThingy<QueryToolChest> loader = ReflectionLoaderThingy.create(QueryToolChest.class);
|
ReflectionLoaderThingy<QueryToolChest> loader = ReflectionLoaderThingy.create(QueryToolChest.class);
|
||||||
|
|
|
@ -81,5 +81,4 @@ public class AggregatorUtil
|
||||||
}
|
}
|
||||||
return new Pair(condensedAggs, condensedPostAggs);
|
return new Pair(condensedAggs, condensedPostAggs);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
|
@ -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
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -25,9 +25,8 @@ import com.google.common.base.Charsets;
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
import io.druid.query.aggregation.Aggregator;
|
import io.druid.query.aggregation.Aggregator;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
|
import io.druid.query.aggregation.Aggregators;
|
||||||
import io.druid.query.aggregation.BufferAggregator;
|
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.ColumnSelectorFactory;
|
||||||
import io.druid.segment.ObjectColumnSelector;
|
import io.druid.segment.ObjectColumnSelector;
|
||||||
import org.apache.commons.codec.binary.Base64;
|
import org.apache.commons.codec.binary.Base64;
|
||||||
|
@ -71,7 +70,7 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
|
||||||
ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName);
|
ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName);
|
||||||
|
|
||||||
if (selector == null) {
|
if (selector == null) {
|
||||||
return new NoopAggregator(name);
|
return Aggregators.noopAggregator();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) {
|
if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) {
|
||||||
|
@ -89,7 +88,7 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
|
||||||
ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName);
|
ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName);
|
||||||
|
|
||||||
if (selector == null) {
|
if (selector == null) {
|
||||||
return new NoopBufferAggregator();
|
return Aggregators.noopBufferAggregator();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) {
|
if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) {
|
||||||
|
|
|
@ -53,8 +53,7 @@ import io.druid.query.QueryToolChest;
|
||||||
import io.druid.query.QueryToolChestWarehouse;
|
import io.druid.query.QueryToolChestWarehouse;
|
||||||
import io.druid.query.Result;
|
import io.druid.query.Result;
|
||||||
import io.druid.query.SegmentDescriptor;
|
import io.druid.query.SegmentDescriptor;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.MetricManipulatorFns;
|
||||||
import io.druid.query.aggregation.MetricManipulationFn;
|
|
||||||
import io.druid.query.spec.MultipleSpecificSegmentSpec;
|
import io.druid.query.spec.MultipleSpecificSegmentSpec;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
import io.druid.timeline.TimelineObjectHolder;
|
import io.druid.timeline.TimelineObjectHolder;
|
||||||
|
@ -359,14 +358,7 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
||||||
),
|
),
|
||||||
toolChest.makePreComputeManipulatorFn(
|
toolChest.makePreComputeManipulatorFn(
|
||||||
rewrittenQuery,
|
rewrittenQuery,
|
||||||
new MetricManipulationFn()
|
MetricManipulatorFns.deserializing()
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Object manipulate(AggregatorFactory factory, Object object)
|
|
||||||
{
|
|
||||||
return factory.deserialize(object);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
new Runnable()
|
new Runnable()
|
||||||
|
|
|
@ -49,8 +49,7 @@ import io.druid.query.QueryRunner;
|
||||||
import io.druid.query.QueryToolChest;
|
import io.druid.query.QueryToolChest;
|
||||||
import io.druid.query.QueryToolChestWarehouse;
|
import io.druid.query.QueryToolChestWarehouse;
|
||||||
import io.druid.query.Result;
|
import io.druid.query.Result;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.MetricManipulatorFns;
|
||||||
import io.druid.query.aggregation.MetricManipulationFn;
|
|
||||||
import org.jboss.netty.handler.codec.http.HttpChunk;
|
import org.jboss.netty.handler.codec.http.HttpChunk;
|
||||||
import org.jboss.netty.handler.codec.http.HttpHeaders;
|
import org.jboss.netty.handler.codec.http.HttpHeaders;
|
||||||
import org.jboss.netty.handler.codec.http.HttpResponse;
|
import org.jboss.netty.handler.codec.http.HttpResponse;
|
||||||
|
@ -219,14 +218,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
|
||||||
retVal,
|
retVal,
|
||||||
toolChest.makePreComputeManipulatorFn(
|
toolChest.makePreComputeManipulatorFn(
|
||||||
query,
|
query,
|
||||||
new MetricManipulationFn()
|
MetricManipulatorFns.deserializing()
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Object manipulate(AggregatorFactory factory, Object object)
|
|
||||||
{
|
|
||||||
return factory.deserialize(object);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue