added a RenameKeysUpdateStream wrapper

This commit is contained in:
Dhruv Parthasarathy 2013-07-08 18:53:08 -07:00
parent 142271aad2
commit 72fbc516bc
10 changed files with 328 additions and 111 deletions

View File

@ -0,0 +1,79 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 druid.examples.webStream;
import com.google.common.collect.Maps;
import java.util.Map;
import java.util.concurrent.TimeUnit;
public class RenamingKeysUpdateStream implements Runnable
{
private final UpdateStream updateStream;
private Map<String, String> renamedDimensions;
public RenamingKeysUpdateStream(
UpdateStream updateStream,
Map<String, String> renamedDimensions
)
{
this.renamedDimensions = renamedDimensions;
this.updateStream = updateStream;
}
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException
{
return renameKeys(updateStream.pollFromQueue(waitTime, unit));
}
private Map<String, Object> renameKeys(Map<String, Object> update)
{
if (renamedDimensions != null) {
Map<String, Object> renamedMap = Maps.newHashMap();
for (String key : renamedDimensions.keySet()) {
if (update.get(key) != null) {
Object obj = update.get(key);
renamedMap.put(renamedDimensions.get(key), obj);
}
}
return renamedMap;
} else {
return update;
}
}
public String getTimeDimension()
{
if (renamedDimensions != null && renamedDimensions.get(updateStream.getTimeDimension()) != null) {
return renamedDimensions.get(updateStream.getTimeDimension());
}
return updateStream.getTimeDimension();
}
@Override
public void run()
{
updateStream.run();
}
}

View File

@ -0,0 +1,37 @@
package druid.examples.webStream;/*
* Druid - a distributed column store.
* Copyright (C) 2012 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.
*/
import java.util.Map;
public class RenamingKeysUpdateStreamFactory
{
private UpdateStreamFactory updateStreamFactory;
private Map<String, String> renamedDimensions;
public RenamingKeysUpdateStreamFactory(UpdateStreamFactory updateStreamFactory, Map<String, String> renamedDimensions)
{
this.updateStreamFactory = updateStreamFactory;
this.renamedDimensions = renamedDimensions;
}
public RenamingKeysUpdateStream build()
{
return new RenamingKeysUpdateStream(updateStreamFactory.build(), renamedDimensions);
}
}

View File

@ -0,0 +1,92 @@
package druid.examples.webStream;/*
* Druid - a distributed column store.
* Copyright (C) 2012 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.
*/
import com.google.common.io.InputSupplier;
import junit.framework.Assert;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
public class RenamingKeysUpdateStreamTest
{
private final long waitTime = 15L;
private final TimeUnit unit = TimeUnit.SECONDS;
private InputSupplier testCaseSupplier;
String timeDimension;
@BeforeClass
public void setUp()
{
timeDimension = "time";
testCaseSupplier = new TestCaseSupplier(
"{\"item1\": \"value1\","
+ "\"item2\":2,"
+ "\"time\":1372121562 }"
);
}
@Test
public void testPollFromQueue() throws Exception
{
UpdateStream updateStream = new UpdateStream(testCaseSupplier, timeDimension);
Map<String, String> renamedKeys = new HashMap<String, String>();
renamedKeys.put("item1", "i1");
renamedKeys.put("item2", "i2");
renamedKeys.put("time", "t");
RenamingKeysUpdateStream renamer = new RenamingKeysUpdateStream(updateStream, renamedKeys);
renamer.run();
Map<String, Object> expectedAnswer = new HashMap<String, Object>();
expectedAnswer.put("i1", "value1");
expectedAnswer.put("i2", 2);
expectedAnswer.put("t", 1372121562);
Assert.assertEquals(expectedAnswer, renamer.pollFromQueue(waitTime, unit));
}
@Test
public void testGetTimeDimension() throws Exception
{
UpdateStream updateStream = new UpdateStream(testCaseSupplier, timeDimension);
Map<String, String> renamedKeys = new HashMap<String, String>();
renamedKeys.put("item1", "i1");
renamedKeys.put("item2", "i2");
renamedKeys.put("time", "t");
RenamingKeysUpdateStream renamer = new RenamingKeysUpdateStream(updateStream, renamedKeys);
Assert.assertEquals("t", renamer.getTimeDimension());
}
@Test
public void testMissingTimeRename() throws Exception
{
UpdateStream updateStream = new UpdateStream(testCaseSupplier, timeDimension);
Map<String, String> renamedKeys = new HashMap<String, String>();
renamedKeys.put("item1", "i1");
renamedKeys.put("item2", "i2");
RenamingKeysUpdateStream renamer = new RenamingKeysUpdateStream(updateStream, renamedKeys);
Assert.assertEquals("time", renamer.getTimeDimension());
}
}

View File

@ -43,13 +43,10 @@ public class UpdateStream implements Runnable
private final int QUEUE_SIZE = 10000; private final int QUEUE_SIZE = 10000;
private final BlockingQueue<Map<String, Object>> queue = new ArrayBlockingQueue<Map<String, Object>>(QUEUE_SIZE); private final BlockingQueue<Map<String, Object>> queue = new ArrayBlockingQueue<Map<String, Object>>(QUEUE_SIZE);
private final ObjectMapper mapper = new DefaultObjectMapper(); private final ObjectMapper mapper = new DefaultObjectMapper();
private final Map<String, String> renamedDimensions;
private final String timeDimension; private final String timeDimension;
private final long waitTime = 15L;
public UpdateStream( public UpdateStream(
InputSupplier<BufferedReader> supplier, InputSupplier<BufferedReader> supplier,
Map<String, String> renamedDimensions,
String timeDimension String timeDimension
) )
{ {
@ -58,7 +55,6 @@ public class UpdateStream implements Runnable
{ {
}; };
this.timeDimension = timeDimension; this.timeDimension = timeDimension;
this.renamedDimensions = renamedDimensions;
} }
private boolean isValid(String s) private boolean isValid(String s)
@ -76,8 +72,7 @@ public class UpdateStream implements Runnable
if (isValid(line)) { if (isValid(line)) {
HashMap<String, Object> map = mapper.readValue(line, typeRef); HashMap<String, Object> map = mapper.readValue(line, typeRef);
if (map.get(timeDimension) != null) { if (map.get(timeDimension) != null) {
Map<String, Object> renamedMap = renameKeys(map); queue.offer(map, queueWaitTime, TimeUnit.SECONDS);
queue.offer(renamedMap, queueWaitTime, TimeUnit.SECONDS);
log.debug("Successfully added to queue"); log.debug("Successfully added to queue");
} else { } else {
log.error("missing timestamp"); log.error("missing timestamp");
@ -91,25 +86,10 @@ public class UpdateStream implements Runnable
} }
private Map<String, Object> renameKeys(Map<String, Object> update)
{
if (renamedDimensions != null) {
Map<String, Object> renamedMap = Maps.newHashMap();
for (String key : renamedDimensions.keySet()) {
if (update.get(key) != null) {
Object obj = update.get(key);
renamedMap.put(renamedDimensions.get(key), obj);
}
}
return renamedMap;
} else {
return update;
}
}
public Map<String, Object> pollFromQueue() throws InterruptedException public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException
{ {
return queue.poll(waitTime, TimeUnit.SECONDS); return queue.poll(waitTime, unit);
} }
public int getQueueSize() public int getQueueSize()
@ -117,13 +97,8 @@ public class UpdateStream implements Runnable
return queue.size(); return queue.size();
} }
public String getNewTimeDimension() public String getTimeDimension(){
{ return timeDimension;
if (renamedDimensions != null) {
return renamedDimensions.get(timeDimension);
} else {
return timeDimension;
}
} }
} }

View File

@ -21,24 +21,20 @@ package druid.examples.webStream;
import com.google.common.io.InputSupplier; import com.google.common.io.InputSupplier;
import java.util.Map;
public class UpdateStreamFactory public class UpdateStreamFactory
{ {
private final InputSupplier inputSupplier; private final InputSupplier inputSupplier;
private final Map<String, String> renamedDimensions;
private final String timeDimension; private final String timeDimension;
public UpdateStreamFactory(InputSupplier inputSupplier, Map<String, String> renamedDimensions, String timeDimension) public UpdateStreamFactory(InputSupplier inputSupplier, String timeDimension)
{ {
this.inputSupplier = inputSupplier; this.inputSupplier = inputSupplier;
this.renamedDimensions = renamedDimensions;
this.timeDimension = timeDimension; this.timeDimension = timeDimension;
} }
public UpdateStream build() public UpdateStream build()
{ {
return new UpdateStream(inputSupplier, renamedDimensions, timeDimension); return new UpdateStream(inputSupplier, timeDimension);
} }
} }

View File

@ -20,21 +20,21 @@
package druid.examples.webStream; package druid.examples.webStream;
import com.google.common.io.InputSupplier; import com.google.common.io.InputSupplier;
import com.metamx.druid.jackson.DefaultObjectMapper;
import junit.framework.Assert; import junit.framework.Assert;
import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test; import org.testng.annotations.Test;
import java.net.UnknownHostException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.concurrent.TimeUnit;
public class UpdateStreamTest public class UpdateStreamTest
{ {
private final long waitTime = 15L;
private final TimeUnit unit = TimeUnit.SECONDS;
private final ArrayList<String> dimensions = new ArrayList<String>(); private final ArrayList<String> dimensions = new ArrayList<String>();
private InputSupplier testCaseSupplier; private InputSupplier testCaseSupplier;
DefaultObjectMapper mapper = new DefaultObjectMapper();
Map<String, Object> expectedAnswer = new HashMap<String, Object>(); Map<String, Object> expectedAnswer = new HashMap<String, Object>();
String timeDimension; String timeDimension;
@ -57,25 +57,16 @@ public class UpdateStreamTest
expectedAnswer.put("time", 1372121562); expectedAnswer.put("time", 1372121562);
} }
@Test(expectedExceptions = UnknownHostException.class)
public void checkInvalidUrl() throws Exception
{
String invalidURL = "http://invalid.url";
WebJsonSupplier supplier = new WebJsonSupplier(invalidURL);
supplier.getInput();
}
@Test @Test
public void basicIngestionCheck() throws Exception public void basicIngestionCheck() throws Exception
{ {
UpdateStream updateStream = new UpdateStream( UpdateStream updateStream = new UpdateStream(
testCaseSupplier, testCaseSupplier,
null,
timeDimension timeDimension
); );
updateStream.run(); updateStream.run();
Map<String, Object> insertedRow = updateStream.pollFromQueue(); Map<String, Object> insertedRow = updateStream.pollFromQueue(waitTime, unit);
Assert.assertEquals(expectedAnswer, insertedRow); Assert.assertEquals(expectedAnswer, insertedRow);
} }
@ -90,7 +81,6 @@ public class UpdateStreamTest
UpdateStream updateStream = new UpdateStream( UpdateStream updateStream = new UpdateStream(
testCaseSupplier, testCaseSupplier,
null,
timeDimension timeDimension
); );
updateStream.run(); updateStream.run();
@ -105,40 +95,17 @@ public class UpdateStreamTest
"{\"item1\": \"value1\"," "{\"item1\": \"value1\","
+ "\"time\":1372121562 }" + "\"time\":1372121562 }"
); );
UpdateStream updateStream = new UpdateStream(
testCaseSupplier,
timeDimension
);
updateStream.run();
Map<String, Object> insertedRow = updateStream.pollFromQueue(waitTime, unit);
Map<String, Object> expectedAnswer = new HashMap<String, Object>(); Map<String, Object> expectedAnswer = new HashMap<String, Object>();
expectedAnswer.put("item1", "value1"); expectedAnswer.put("item1", "value1");
expectedAnswer.put("time", 1372121562); expectedAnswer.put("time", 1372121562);
UpdateStream updateStream = new UpdateStream(
testCaseSupplier,
null,
timeDimension
);
updateStream.run();
Map<String, Object> insertedRow = updateStream.pollFromQueue();
Assert.assertEquals(expectedAnswer, insertedRow); Assert.assertEquals(expectedAnswer, insertedRow);
} }
@Test
public void checkRenameKeys() throws Exception
{
Map<String, Object> expectedAnswer = new HashMap<String, Object>();
Map<String, String> renamedDimensions = new HashMap<String, String>();
renamedDimensions.put("item1", "i1");
renamedDimensions.put("item2", "i2");
renamedDimensions.put("time", "t");
expectedAnswer.put("i1", "value1");
expectedAnswer.put("i2", 2);
expectedAnswer.put("t", 1372121562);
UpdateStream updateStream = new UpdateStream(
testCaseSupplier,
renamedDimensions,
timeDimension
);
updateStream.run();
Map<String, Object> inputRow = updateStream.pollFromQueue();
Assert.assertEquals(expectedAnswer, inputRow);
}
} }

View File

@ -34,17 +34,19 @@ import org.joda.time.DateTime;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
@JsonTypeName("webstream") @JsonTypeName("webstream")
public class WebFirehoseFactory implements FirehoseFactory public class WebFirehoseFactory implements FirehoseFactory
{ {
private static final EmittingLogger log = new EmittingLogger(WebFirehoseFactory.class); private static final EmittingLogger log = new EmittingLogger(WebFirehoseFactory.class);
private final String timeFormat; private final String timeFormat;
private final UpdateStreamFactory updateStreamFactory; private final RenamingKeysUpdateStreamFactory factory;
private final long queueWaitTime = 15L;
@JsonCreator @JsonCreator
public WebFirehoseFactory( public WebFirehoseFactory(
@ -54,12 +56,17 @@ public class WebFirehoseFactory implements FirehoseFactory
@JsonProperty("timeFormat") String timeFormat @JsonProperty("timeFormat") String timeFormat
) )
{ {
this(new UpdateStreamFactory(new WebJsonSupplier(url), renamedDimensions, timeDimension), timeFormat); this(
new RenamingKeysUpdateStreamFactory(
new UpdateStreamFactory(new WebJsonSupplier(url), timeDimension),
renamedDimensions
), timeFormat
);
} }
public WebFirehoseFactory(UpdateStreamFactory updateStreamFactory, String timeFormat) public WebFirehoseFactory(RenamingKeysUpdateStreamFactory factory, String timeFormat)
{ {
this.updateStreamFactory = updateStreamFactory; this.factory = factory;
if (timeFormat == null) { if (timeFormat == null) {
this.timeFormat = "auto"; this.timeFormat = "auto";
} else { } else {
@ -71,9 +78,9 @@ public class WebFirehoseFactory implements FirehoseFactory
public Firehose connect() throws IOException public Firehose connect() throws IOException
{ {
final UpdateStream updateStream = updateStreamFactory.build(); final RenamingKeysUpdateStream renamingUpdateStream = factory.build();
final ExecutorService service = Executors.newSingleThreadExecutor(); final ExecutorService service = Executors.newSingleThreadExecutor();
service.submit(updateStream); service.submit(renamingUpdateStream);
return new Firehose() return new Firehose()
{ {
@ -84,10 +91,10 @@ public class WebFirehoseFactory implements FirehoseFactory
public boolean hasMore() public boolean hasMore()
{ {
try { try {
map = updateStream.pollFromQueue(); map = renamingUpdateStream.pollFromQueue(queueWaitTime, TimeUnit.SECONDS);
return map != null; return map != null;
} }
catch (Exception e) { catch (InterruptedException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
} }
} }
@ -98,7 +105,7 @@ public class WebFirehoseFactory implements FirehoseFactory
{ {
try { try {
DateTime date = TimestampParser.createTimestampParser(timeFormat) DateTime date = TimestampParser.createTimestampParser(timeFormat)
.apply(map.get(updateStream.getNewTimeDimension()).toString()); .apply(map.get(renamingUpdateStream.getTimeDimension()).toString());
return new MapBasedInputRow( return new MapBasedInputRow(
date.getMillis(), date.getMillis(),
new ArrayList(map.keySet()), new ArrayList(map.keySet()),
@ -108,6 +115,9 @@ public class WebFirehoseFactory implements FirehoseFactory
catch (Exception e) { catch (Exception e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
} }
finally {
map = null;
}
} }
@Override @Override

View File

@ -35,8 +35,6 @@ public class WebFirehoseFactoryTest
{ {
private List<String> dimensions; private List<String> dimensions;
private WebFirehoseFactory webbie; private WebFirehoseFactory webbie;
private Firehose firehose;
private InputRow inputRow;
private TestCaseSupplier testCaseSupplier; private TestCaseSupplier testCaseSupplier;
@BeforeClass @BeforeClass
@ -51,21 +49,20 @@ public class WebFirehoseFactoryTest
+ "\"item2\":2," + "\"item2\":2,"
+ "\"time\":1372121562 }" + "\"time\":1372121562 }"
); );
UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, null, "time");
webbie = new WebFirehoseFactory(updateStreamFactory, "posix");
firehose = webbie.connect();
if (firehose.hasMore()) {
inputRow = firehose.nextRow();
} else {
throw new RuntimeException("queue is empty");
}
} }
@Test @Test
public void testDimensions() throws Exception public void testDimensions() throws Exception
{ {
InputRow inputRow;
UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, "time");
webbie = new WebFirehoseFactory(new RenamingKeysUpdateStreamFactory(updateStreamFactory, null), "posix");
Firehose firehose = webbie.connect();
if (firehose.hasMore()) {
inputRow = firehose.nextRow();
} else {
throw new RuntimeException("queue is empty");
}
List<String> actualAnswer = inputRow.getDimensions(); List<String> actualAnswer = inputRow.getDimensions();
Collections.sort(actualAnswer); Collections.sort(actualAnswer);
Assert.assertEquals(actualAnswer, dimensions); Assert.assertEquals(actualAnswer, dimensions);
@ -74,6 +71,15 @@ public class WebFirehoseFactoryTest
@Test @Test
public void testPosixTimeStamp() throws Exception public void testPosixTimeStamp() throws Exception
{ {
InputRow inputRow;
UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, "time");
webbie = new WebFirehoseFactory(new RenamingKeysUpdateStreamFactory(updateStreamFactory, null), "posix");
Firehose firehose = webbie.connect();
if (firehose.hasMore()) {
inputRow = firehose.nextRow();
} else {
throw new RuntimeException("queue is empty");
}
long expectedTime = 1372121562L * 1000L; long expectedTime = 1372121562L * 1000L;
Assert.assertEquals(expectedTime, inputRow.getTimestampFromEpoch()); Assert.assertEquals(expectedTime, inputRow.getTimestampFromEpoch());
} }
@ -87,8 +93,8 @@ public class WebFirehoseFactoryTest
+ "\"time\":\"2013-07-08\"}" + "\"time\":\"2013-07-08\"}"
); );
UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, null, "time"); UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, "time");
webbie = new WebFirehoseFactory(updateStreamFactory, "iso"); webbie = new WebFirehoseFactory(new RenamingKeysUpdateStreamFactory(updateStreamFactory, null), "iso");
Firehose firehose1 = webbie.connect(); Firehose firehose1 = webbie.connect();
if (firehose1.hasMore()) { if (firehose1.hasMore()) {
long milliSeconds = firehose1.nextRow().getTimestampFromEpoch(); long milliSeconds = firehose1.nextRow().getTimestampFromEpoch();
@ -108,8 +114,8 @@ public class WebFirehoseFactoryTest
+ "\"time\":\"2013-07-08\"}" + "\"time\":\"2013-07-08\"}"
); );
UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, null, "time"); UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, "time");
webbie = new WebFirehoseFactory(updateStreamFactory, null); webbie = new WebFirehoseFactory(new RenamingKeysUpdateStreamFactory(updateStreamFactory, null), null);
Firehose firehose2 = webbie.connect(); Firehose firehose2 = webbie.connect();
if (firehose2.hasMore()) { if (firehose2.hasMore()) {
long milliSeconds = firehose2.nextRow().getTimestampFromEpoch(); long milliSeconds = firehose2.nextRow().getTimestampFromEpoch();
@ -129,8 +135,8 @@ public class WebFirehoseFactoryTest
+ "\"time\":1373241600000}" + "\"time\":1373241600000}"
); );
UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, null, "time"); UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, "time");
webbie = new WebFirehoseFactory(updateStreamFactory, null); webbie = new WebFirehoseFactory(new RenamingKeysUpdateStreamFactory(updateStreamFactory, null), null);
Firehose firehose3 = webbie.connect(); Firehose firehose3 = webbie.connect();
if (firehose3.hasMore()) { if (firehose3.hasMore()) {
long milliSeconds = firehose3.nextRow().getTimestampFromEpoch(); long milliSeconds = firehose3.nextRow().getTimestampFromEpoch();
@ -142,16 +148,36 @@ public class WebFirehoseFactoryTest
} }
@Test @Test
public void testGetDimension() public void testGetDimension() throws Exception
{ {
InputRow inputRow;
UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, "time");
webbie = new WebFirehoseFactory(new RenamingKeysUpdateStreamFactory(updateStreamFactory, null), "posix");
Firehose firehose = webbie.connect();
if (firehose.hasMore()) {
inputRow = firehose.nextRow();
} else {
throw new RuntimeException("queue is empty");
}
List<String> column1 = Lists.newArrayList(); List<String> column1 = Lists.newArrayList();
column1.add("value1"); column1.add("value1");
Assert.assertEquals(column1, inputRow.getDimension("item1")); Assert.assertEquals(column1, inputRow.getDimension("item1"));
} }
@Test @Test
public void testGetFloatMetric() public void testGetFloatMetric() throws Exception
{ {
InputRow inputRow;
UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, "time");
webbie = new WebFirehoseFactory(new RenamingKeysUpdateStreamFactory(updateStreamFactory, null), "posix");
Firehose firehose = webbie.connect();
if (firehose.hasMore()) {
inputRow = firehose.nextRow();
} else {
throw new RuntimeException("queue is empty");
}
Assert.assertEquals((float) 2.0, inputRow.getFloatMetric("item2")); Assert.assertEquals((float) 2.0, inputRow.getFloatMetric("item2"));
} }
} }

View File

@ -31,7 +31,6 @@ import java.util.List;
public class WebJsonSupplier implements InputSupplier<BufferedReader> public class WebJsonSupplier implements InputSupplier<BufferedReader>
{ {
private List<String> dimensions;
private static final EmittingLogger log = new EmittingLogger(WebJsonSupplier.class); private static final EmittingLogger log = new EmittingLogger(WebJsonSupplier.class);
private String urlString; private String urlString;

View File

@ -0,0 +1,36 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 druid.examples.webStream;
import org.testng.annotations.Test;
import java.net.UnknownHostException;
public class WebJsonSupplierTest
{
@Test(expectedExceptions = UnknownHostException.class)
public void checkInvalidUrl() throws Exception
{
String invalidURL = "http://invalid.url";
WebJsonSupplier supplier = new WebJsonSupplier(invalidURL);
supplier.getInput();
}
}