Fix and UT for testing segment analysis merge

This commit is contained in:
Himanshu Gupta 2015-12-19 00:24:02 -06:00
parent 1b46ea7b3d
commit 7ecad1be24
2 changed files with 49 additions and 2 deletions

View File

@ -36,7 +36,6 @@ import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.collections.OrderedMergeSequence; import io.druid.collections.OrderedMergeSequence;
import io.druid.common.guava.CombiningSequence; import io.druid.common.guava.CombiningSequence;
import io.druid.common.utils.JodaUtils; import io.druid.common.utils.JodaUtils;
import io.druid.data.input.Row;
import io.druid.query.CacheStrategy; import io.druid.query.CacheStrategy;
import io.druid.query.DruidMetrics; import io.druid.query.DruidMetrics;
import io.druid.query.Query; import io.druid.query.Query;
@ -53,6 +52,7 @@ import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -149,7 +149,7 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
List<Interval> newIntervals = null; List<Interval> newIntervals = null;
if (query.hasInterval()) { if (query.hasInterval()) {
newIntervals = arg1.getIntervals(); newIntervals = new ArrayList<>(arg1.getIntervals());
newIntervals.addAll(arg2.getIntervals()); newIntervals.addAll(arg2.getIntervals());
} }

View File

@ -122,6 +122,53 @@ public class SegmentMetadataQueryTest
Assert.assertEquals(Arrays.asList(expectedSegmentAnalysis), results); Assert.assertEquals(Arrays.asList(expectedSegmentAnalysis), results);
} }
@Test
public void testSegmentMetadataQueryWithDefaultAnalysisMerge()
{
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
"merged",
ImmutableList.of(
expectedSegmentAnalysis.getIntervals().get(0),
expectedSegmentAnalysis.getIntervals().get(0)
),
ImmutableMap.of(
"placement",
new ColumnAnalysis(
ValueType.STRING.toString(),
21762,
1,
null
)
),
expectedSegmentAnalysis.getSize()*2,
expectedSegmentAnalysis.getNumRows()*2
);
QueryToolChest toolChest = factory.getToolchest();
QueryRunner singleSegmentQueryRunner = toolChest.preMergeQueryDecoration(runner);
ExecutorService exec = Executors.newCachedThreadPool();
QueryRunner myRunner = new FinalizeResultsQueryRunner<>(
toolChest.mergeResults(
factory.mergeRunners(
Executors.newCachedThreadPool(),
Lists.<QueryRunner<SegmentAnalysis>>newArrayList(singleSegmentQueryRunner, singleSegmentQueryRunner)
)
),
toolChest
);
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(
testQuery,
Maps.newHashMap()
),
"failed SegmentMetadata bySegment query"
);
exec.shutdownNow();
}
@Test @Test
public void testBySegmentResults() public void testBySegmentResults()
{ {