Merge pull request #514 from metamx/reduce-bloat

Reduce some bloat in the code base
This commit is contained in:
fjy 2014-04-30 11:21:35 -06:00
commit 411def5013
13 changed files with 159 additions and 214 deletions

View File

@ -24,9 +24,8 @@ import com.google.common.collect.ImmutableMap;
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 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
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>()

View File

@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicReference;
/**
*/
@Deprecated
public class ReflectionLoaderThingy<T>
{
private static final Logger log = new Logger(ReflectionLoaderThingy.class);

View File

@ -21,6 +21,7 @@ package io.druid.query;
/**
*/
@Deprecated
public class ReflectionQueryToolChestWarehouse implements QueryToolChestWarehouse
{
ReflectionLoaderThingy<QueryToolChest> loader = ReflectionLoaderThingy.create(QueryToolChest.class);

View File

@ -81,5 +81,4 @@ public class AggregatorUtil
}
return new Pair(condensedAggs, condensedPostAggs);
}
}

View File

@ -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()
{
}
};
}
}

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -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);
}
};
}
}

View File

@ -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
}
}

View File

@ -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
}
}

View File

@ -25,9 +25,8 @@ import com.google.common.base.Charsets;
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 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
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 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName);
if (selector == null) {
return new NoopBufferAggregator();
return Aggregators.noopBufferAggregator();
}
if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) {

View File

@ -53,8 +53,7 @@ import io.druid.query.QueryToolChest;
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 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
),
toolChest.makePreComputeManipulatorFn(
rewrittenQuery,
new MetricManipulationFn()
{
@Override
public Object manipulate(AggregatorFactory factory, Object object)
{
return factory.deserialize(object);
}
}
MetricManipulatorFns.deserializing()
)
),
new Runnable()

View File

@ -49,8 +49,7 @@ import io.druid.query.QueryRunner;
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 @@ public class DirectDruidClient<T> implements QueryRunner<T>
retVal,
toolChest.makePreComputeManipulatorFn(
query,
new MetricManipulationFn()
{
@Override
public Object manipulate(AggregatorFactory factory, Object object)
{
return factory.deserialize(object);
}
}
MetricManipulatorFns.deserializing()
)
);
}