bug fixes

This commit is contained in:
fjy 2013-04-23 18:06:41 -07:00
parent 13b617cfa9
commit 7f43f37441
3 changed files with 46 additions and 11 deletions

View File

@ -25,6 +25,7 @@ import com.google.common.base.Functions;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.metamx.common.ISE;
import com.metamx.common.guava.Accumulator;
import com.metamx.common.guava.ConcatSequence;
@ -33,8 +34,10 @@ import com.metamx.common.guava.Sequences;
import com.metamx.druid.Query;
import com.metamx.druid.QueryGranularity;
import com.metamx.druid.aggregation.AggregatorFactory;
import com.metamx.druid.aggregation.post.PostAggregator;
import com.metamx.druid.index.v1.IncrementalIndex;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.input.MapBasedInputRow;
import com.metamx.druid.input.MapBasedRow;
import com.metamx.druid.input.Row;
import com.metamx.druid.input.Rows;
@ -42,6 +45,8 @@ import com.metamx.druid.query.MetricManipulationFn;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.query.QueryToolChest;
import com.metamx.druid.query.dimension.DimensionSpec;
import com.metamx.druid.result.Result;
import com.metamx.druid.result.TimeseriesResultValue;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.service.ServiceMetricEvent;
@ -57,7 +62,6 @@ import java.util.Properties;
*/
public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery>
{
private static final TypeReference<Row> TYPE_REFERENCE = new TypeReference<Row>(){};
private static final String GROUP_BY_MERGE_KEY = "groupByMerge";
private static final Map<String, String> NO_MERGE_CONTEXT = ImmutableMap.of(GROUP_BY_MERGE_KEY, "false");
@ -183,9 +187,24 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
}
@Override
public Function<Row, Row> makeMetricManipulatorFn(GroupByQuery query, MetricManipulationFn fn)
public Function<Row, Row> makeMetricManipulatorFn(final GroupByQuery query, final MetricManipulationFn fn)
{
return Functions.identity();
return new Function<Row, Row>()
{
@Override
public Row apply(Row input)
{
if (input instanceof MapBasedRow) {
final MapBasedRow inputRow = (MapBasedRow) input;
final Map<String, Object> values = Maps.newHashMap(((MapBasedRow) input).getEvent());
for (AggregatorFactory agg : query.getAggregatorSpecs()) {
values.put(agg.getName(), fn.manipulate(agg, inputRow.getEvent().get(agg.getName())));
}
return new MapBasedRow(inputRow.getTimestamp(), values);
}
return input;
}
};
}
@Override

View File

@ -358,12 +358,14 @@ public class IndexIO
}
ByteBuffer spatialBuffer = smooshedFiles.mapFile("spatial.drd");
if (spatialBuffer != null) {
for (String dimension : IndexedIterable.create(availableDimensions)) {
spatialIndexed.put(
serializerUtils.readString(spatialBuffer),
GenericIndexed.read(spatialBuffer, IndexedRTree.objectStrategy)
);
}
}
final MMappedIndex retVal = new MMappedIndex(
availableDimensions,
@ -422,7 +424,7 @@ public class IndexIO
Map<String, GenericIndexed<ImmutableRTree>> spatialIndexes = Maps.newHashMap();
final ByteBuffer spatialBuffer = v8SmooshedFiles.mapFile("spatial.drd");
while (spatialBuffer.hasRemaining()) {
while (spatialBuffer != null && spatialBuffer.hasRemaining()) {
spatialIndexes.put(
serializerUtils.readString(spatialBuffer),
GenericIndexed.read(spatialBuffer, IndexedRTree.objectStrategy)

View File

@ -19,15 +19,26 @@
package com.metamx.druid.index.v1;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.google.common.primitives.Ints;
import com.metamx.common.logger.Logger;
import com.metamx.common.spatial.rtree.ImmutableRTree;
import com.metamx.druid.kv.ConciseCompressedIndexedInts;
import com.metamx.druid.kv.GenericIndexed;
import com.metamx.druid.kv.Indexed;
import com.metamx.druid.kv.IndexedList;
import com.metamx.druid.kv.IndexedLongs;
import com.metamx.druid.kv.IndexedRTree;
import com.metamx.druid.kv.VSizeIndexed;
import com.metamx.druid.kv.VSizeIndexedInts;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import org.joda.time.Interval;
import java.nio.ByteOrder;
import java.nio.LongBuffer;
import java.util.Arrays;
import java.util.Map;
/**
@ -158,7 +169,6 @@ public class MMappedIndex
return (retVal == null) ? emptySet : retVal;
}
/*
public static MMappedIndex fromIndex(Index index)
{
log.info("Converting timestamps");
@ -249,6 +259,11 @@ public class MMappedIndex
ConciseCompressedIndexedInts.objectStrategy
)
);
spatialIndexes.put(
dimension,
GenericIndexed.fromIterable(Arrays.asList(index.getSpatialIndex(dimension)), IndexedRTree.objectStrategy)
);
}
log.info("Making MMappedIndex");
@ -264,5 +279,4 @@ public class MMappedIndex
spatialIndexes
);
}
*/
}