make tests parameterised

This commit is contained in:
nishantmonu51 2014-12-02 23:55:29 +05:30
parent 6dc69c2f30
commit b65933ffb8
3 changed files with 137 additions and 38 deletions

View File

@ -51,7 +51,8 @@ public class IndexMergerTest
{
final long timestamp = System.currentTimeMillis();
IncrementalIndex toPersist = IncrementalIndexTest.createCaseInsensitiveIndex(timestamp);
IncrementalIndex toPersist = IncrementalIndexTest.createCaseInsensitiveIndex(true);
IncrementalIndexTest.populateIndex(timestamp, toPersist);
final File tempDir = Files.createTempDir();
try {
@ -70,7 +71,8 @@ public class IndexMergerTest
public void testPersistMerge() throws Exception
{
final long timestamp = System.currentTimeMillis();
IncrementalIndex toPersist1 = IncrementalIndexTest.createCaseInsensitiveIndex(timestamp);
IncrementalIndex toPersist1 = IncrementalIndexTest.createCaseInsensitiveIndex(true);
IncrementalIndexTest.populateIndex(timestamp, toPersist1);
IncrementalIndex toPersist2 = new OnheapIncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{});

View File

@ -25,14 +25,18 @@ import io.druid.data.input.Row;
import io.druid.granularity.QueryGranularity;
import io.druid.query.TestQueryRunners;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.OffheapIncrementalIndex;
import io.druid.segment.incremental.OnheapIncrementalIndex;
import junit.framework.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.CountDownLatch;
@ -42,15 +46,72 @@ import java.util.concurrent.TimeUnit;
/**
*/
@RunWith(Parameterized.class)
public class IncrementalIndexTest
{
public static IncrementalIndex createCaseInsensitiveIndex(long timestamp)
interface IndexCreator
{
IncrementalIndex index = new OnheapIncrementalIndex(
0L, QueryGranularity.NONE, new AggregatorFactory[]{}
);
public IncrementalIndex createIndex();
}
private final IndexCreator indexCreator;
public IncrementalIndexTest(
IndexCreator indexCreator
)
{
this.indexCreator = indexCreator;
}
@Parameterized.Parameters
public static Collection<?> constructorFeeder() throws IOException
{
return Arrays.asList(
new Object[][]{
{
new IndexCreator()
{
@Override
public IncrementalIndex createIndex()
{
return createCaseInsensitiveIndex(true);
}
}
},
{
new IndexCreator()
{
@Override
public IncrementalIndex createIndex()
{
return createCaseInsensitiveIndex(false);
}
}
}
}
);
}
public static IncrementalIndex createCaseInsensitiveIndex(boolean offheap)
{
if (offheap) {
return new OffheapIncrementalIndex(
0L,
QueryGranularity.NONE,
new AggregatorFactory[]{},
TestQueryRunners.pool,
true
);
} else {
return new OnheapIncrementalIndex(
0L, QueryGranularity.NONE, new AggregatorFactory[]{}
);
}
}
public static void populateIndex(long timestamp, IncrementalIndex index)
{
index.add(
new MapBasedInputRow(
timestamp,
@ -66,7 +127,6 @@ public class IncrementalIndexTest
ImmutableMap.<String, Object>of("dim1", "3", "dim2", "4")
)
);
return index;
}
public static MapBasedInputRow getRow(long timestamp, int rowID, int dimensionCount)
@ -84,10 +144,9 @@ public class IncrementalIndexTest
@Test
public void testCaseSensitivity() throws Exception
{
final long timestamp = System.currentTimeMillis();
IncrementalIndex index = createCaseInsensitiveIndex(timestamp);
long timestamp = System.currentTimeMillis();
IncrementalIndex index = indexCreator.createIndex();
populateIndex(timestamp, index);
Assert.assertEquals(Arrays.asList("dim1", "dim2"), index.getDimensions());
Assert.assertEquals(2, index.size());
@ -106,11 +165,7 @@ public class IncrementalIndexTest
@Test
public void testConcurrentAdd() throws Exception
{
final IncrementalIndex index = new OnheapIncrementalIndex(
0L,
QueryGranularity.NONE,
new AggregatorFactory[]{new CountAggregatorFactory("count")}
);
final IncrementalIndex index = indexCreator.createIndex();
final int threadCount = 10;
final int elementsPerThread = 200;
final int dimensionCount = 5;

View File

@ -49,24 +49,78 @@ import io.druid.segment.DimensionSelector;
import io.druid.segment.filter.SelectorFilter;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
/**
*/
@RunWith(Parameterized.class)
public class IncrementalIndexStorageAdapterTest
{
interface IndexCreator
{
public IncrementalIndex createIndex();
}
private final IndexCreator indexCreator;
public IncrementalIndexStorageAdapterTest(
IndexCreator IndexCreator
)
{
this.indexCreator = IndexCreator;
}
@Parameterized.Parameters
public static Collection<?> constructorFeeder() throws IOException
{
return Arrays.asList(
new Object[][]{
{ new IndexCreator()
{
@Override
public IncrementalIndex createIndex()
{
return new OnheapIncrementalIndex(
0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}
);
}
}
},
{
new IndexCreator()
{
@Override
public IncrementalIndex createIndex()
{
return new OffheapIncrementalIndex(
0,
QueryGranularity.MINUTE,
new AggregatorFactory[]{new CountAggregatorFactory("cnt")},
TestQueryRunners.pool,
true
);
}
}
}
}
);
}
@Test
public void testSanity() throws Exception
{
IncrementalIndex index = new OnheapIncrementalIndex(
0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}
);
IncrementalIndex index = indexCreator.createIndex();
index.add(
new MapBasedInputRow(
new DateTime().minus(1).getMillis(),
@ -110,10 +164,7 @@ public class IncrementalIndexStorageAdapterTest
@Test
public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception
{
IncrementalIndex index = new OnheapIncrementalIndex(
0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}
);
IncrementalIndex index = indexCreator.createIndex();
index.add(
new MapBasedInputRow(
new DateTime("2014-09-01T00:00:00"),
@ -196,11 +247,8 @@ public class IncrementalIndexStorageAdapterTest
@Test
public void testResetSanity() {
IncrementalIndex index = new OnheapIncrementalIndex(
0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}
);
IncrementalIndex index = indexCreator.createIndex();
DateTime t = DateTime.now();
Interval interval = new Interval(t.minusMinutes(1), t.plusMinutes(1));
@ -248,10 +296,7 @@ public class IncrementalIndexStorageAdapterTest
@Test
public void testSingleValueTopN()
{
IncrementalIndex index = new OnheapIncrementalIndex(
0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}
);
IncrementalIndex index = indexCreator.createIndex();
DateTime t = DateTime.now();
index.add(
new MapBasedInputRow(
@ -303,10 +348,7 @@ public class IncrementalIndexStorageAdapterTest
@Test
public void testFilterByNull() throws Exception
{
IncrementalIndex index = new OnheapIncrementalIndex(
0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}
);
IncrementalIndex index = indexCreator.createIndex();
index.add(
new MapBasedInputRow(
new DateTime().minus(1).getMillis(),