Unit test for #3544: Avoid exceptions for dataSource spec when using s3. (#3571)

This commit is contained in:
Gian Merlino 2016-10-17 14:41:43 -05:00 committed by Fangjin Yang
parent 9611358f0a
commit dd0bb6da1e
1 changed files with 53 additions and 0 deletions

View File

@ -19,11 +19,14 @@
package io.druid.indexer.hadoop; package io.druid.indexer.hadoop;
import com.google.common.base.Charsets;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import com.google.common.io.Files;
import io.druid.indexer.JobHelper; import io.druid.indexer.JobHelper;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
@ -42,8 +45,11 @@ import org.easymock.EasyMock;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -52,6 +58,9 @@ import java.util.Map;
*/ */
public class DatasourceInputFormatTest public class DatasourceInputFormatTest
{ {
@Rule
public final TemporaryFolder temporaryFolder = new TemporaryFolder();
private List<WindowedDataSegment> segments; private List<WindowedDataSegment> segments;
private List<LocatedFileStatus> locations; private List<LocatedFileStatus> locations;
private JobConf config; private JobConf config;
@ -265,6 +274,50 @@ public class DatasourceInputFormatTest
Assert.assertArrayEquals(new String[]{"s1", "s2"}, splits.get(2).getLocations()); Assert.assertArrayEquals(new String[]{"s1", "s2"}, splits.get(2).getLocations());
} }
@Test
public void testGetSplitsUsingDefaultSupplier() throws Exception
{
// Use the builtin supplier, reading from the local filesystem, rather than testFormatter.
final File tmpFile = temporaryFolder.newFile("something:with:colons");
Files.write("dummy", tmpFile, Charsets.UTF_8);
final ImmutableList<WindowedDataSegment> mySegments = ImmutableList.of(
WindowedDataSegment.of(
new DataSegment(
"test1",
Interval.parse("2000/3000"),
"ver",
ImmutableMap.<String, Object>of(
"type", "local",
"path", tmpFile.getPath()
),
ImmutableList.of("host"),
ImmutableList.of("visited_sum", "unique_hosts"),
NoneShardSpec.instance(),
9,
2
)
)
);
final JobConf myConfig = new JobConf();
myConfig.set(
DatasourceInputFormat.CONF_INPUT_SEGMENTS,
new DefaultObjectMapper().writeValueAsString(mySegments)
);
final JobContext myContext = EasyMock.createMock(JobContext.class);
EasyMock.expect(myContext.getConfiguration()).andReturn(myConfig);
EasyMock.replay(myContext);
final List<InputSplit> splits = new DatasourceInputFormat().getSplits(myContext);
Assert.assertEquals(1, splits.size());
final DatasourceInputSplit theSplit = (DatasourceInputSplit) Iterables.getOnlyElement(splits);
Assert.assertEquals(mySegments.get(0).getSegment().getSize(), theSplit.getLength());
Assert.assertEquals(mySegments, theSplit.getSegments());
Assert.assertArrayEquals(new String[]{"localhost"}, theSplit.getLocations());
}
@Test @Test
public void testGetRecordReader() throws Exception public void testGetRecordReader() throws Exception
{ {