mirror of
https://github.com/apache/druid.git
synced 2025-02-17 07:25:02 +00:00
Merge pull request #173 from metamx/dhruv
Add new demo firehose that is lower friction than twitter
This commit is contained in:
commit
b83bc14784
1
.gitignore
vendored
1
.gitignore
vendored
@ -13,3 +13,4 @@ target
|
||||
examples/rand/RealtimeNode.out
|
||||
examples/twitter/RealtimeNode.out
|
||||
*.log
|
||||
*.DS_Store
|
||||
|
@ -34,4 +34,10 @@ public class Runnables
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public static Runnable getNoopRunnable(){
|
||||
return new Runnable(){
|
||||
public void run(){}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
@ -1,5 +1,6 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<artifactId>druid-examples</artifactId>
|
||||
|
@ -16,6 +16,7 @@ import com.metamx.druid.realtime.SegmentPublisher;
|
||||
import druid.examples.flights.FlightsFirehoseFactory;
|
||||
import druid.examples.rand.RandomFirehoseFactory;
|
||||
import druid.examples.twitter.TwitterSpritzerFirehoseFactory;
|
||||
import druid.examples.webStream.WebFirehoseFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
@ -41,7 +42,9 @@ public class RealtimeStandaloneMain
|
||||
rn.registerJacksonSubtype(
|
||||
new NamedType(TwitterSpritzerFirehoseFactory.class, "twitzer"),
|
||||
new NamedType(FlightsFirehoseFactory.class, "flights"),
|
||||
new NamedType(RandomFirehoseFactory.class, "rand")
|
||||
new NamedType(RandomFirehoseFactory.class, "rand"),
|
||||
new NamedType(WebFirehoseFactory.class, "webstream")
|
||||
|
||||
);
|
||||
|
||||
// Create dummy objects for the various interfaces that interact with the DB, ZK and deep storage
|
||||
@ -50,7 +53,7 @@ public class RealtimeStandaloneMain
|
||||
rn.setDataSegmentPusher(new NoopDataSegmentPusher());
|
||||
rn.setServerView(new NoopServerView());
|
||||
rn.setInventoryView(new NoopInventoryView());
|
||||
|
||||
|
||||
Runtime.getRuntime().addShutdownHook(
|
||||
new Thread(
|
||||
new Runnable()
|
||||
|
@ -0,0 +1,133 @@
|
||||
/*
|
||||
* 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.fasterxml.jackson.core.JsonParseException;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.JsonMappingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.io.InputSupplier;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ArrayBlockingQueue;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class InputSupplierUpdateStream implements UpdateStream
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(InputSupplierUpdateStream.class);
|
||||
private static final long queueWaitTime = 15L;
|
||||
private final TypeReference<HashMap<String, Object>> typeRef;
|
||||
private final InputSupplier<BufferedReader> supplier;
|
||||
private final int QUEUE_SIZE = 10000;
|
||||
private final BlockingQueue<Map<String, Object>> queue = new ArrayBlockingQueue<Map<String, Object>>(QUEUE_SIZE);
|
||||
private final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
private final String timeDimension;
|
||||
private final Thread addToQueueThread;
|
||||
|
||||
public InputSupplierUpdateStream(
|
||||
final InputSupplier<BufferedReader> supplier,
|
||||
final String timeDimension
|
||||
)
|
||||
{
|
||||
addToQueueThread = new Thread()
|
||||
{
|
||||
public void run()
|
||||
{
|
||||
while (!isInterrupted()) {
|
||||
try {
|
||||
BufferedReader reader = supplier.getInput();
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
if (isValid(line)) {
|
||||
HashMap<String, Object> map = mapper.readValue(line, typeRef);
|
||||
if (map.get(timeDimension) != null) {
|
||||
queue.offer(map, queueWaitTime, TimeUnit.SECONDS);
|
||||
log.debug("Successfully added to queue");
|
||||
} else {
|
||||
log.info("missing timestamp");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
catch (InterruptedException e){
|
||||
log.info(e, "Thread adding events to the queue interrupted");
|
||||
return;
|
||||
}
|
||||
catch (JsonMappingException e) {
|
||||
log.info(e, "Error in converting json to map");
|
||||
}
|
||||
catch (JsonParseException e) {
|
||||
log.info(e, "Error in parsing json");
|
||||
}
|
||||
catch (IOException e) {
|
||||
log.info(e, "Error in connecting to InputStream");
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
addToQueueThread.setDaemon(true);
|
||||
|
||||
this.supplier = supplier;
|
||||
this.typeRef = new TypeReference<HashMap<String, Object>>()
|
||||
{
|
||||
};
|
||||
this.timeDimension = timeDimension;
|
||||
}
|
||||
|
||||
private boolean isValid(String s)
|
||||
{
|
||||
return !(s.isEmpty());
|
||||
}
|
||||
|
||||
public void start()
|
||||
{
|
||||
addToQueueThread.start();
|
||||
|
||||
}
|
||||
|
||||
public void stop()
|
||||
{
|
||||
addToQueueThread.interrupt();
|
||||
}
|
||||
|
||||
|
||||
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException
|
||||
{
|
||||
return queue.poll(waitTime, unit);
|
||||
}
|
||||
|
||||
public int getQueueSize()
|
||||
{
|
||||
return queue.size();
|
||||
}
|
||||
|
||||
public String getTimeDimension()
|
||||
{
|
||||
return timeDimension;
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,42 @@
|
||||
/*
|
||||
* 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.io.InputSupplier;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
|
||||
public class InputSupplierUpdateStreamFactory implements UpdateStreamFactory
|
||||
{
|
||||
private final InputSupplier<BufferedReader> inputSupplier;
|
||||
private final String timeDimension;
|
||||
|
||||
public InputSupplierUpdateStreamFactory(InputSupplier<BufferedReader> inputSupplier, String timeDimension)
|
||||
{
|
||||
this.inputSupplier = inputSupplier;
|
||||
this.timeDimension = timeDimension;
|
||||
}
|
||||
|
||||
public InputSupplierUpdateStream build()
|
||||
{
|
||||
return new InputSupplierUpdateStream(inputSupplier, timeDimension);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,111 @@
|
||||
/*
|
||||
* 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.io.InputSupplier;
|
||||
import junit.framework.Assert;
|
||||
import org.testng.annotations.BeforeClass;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class InputSupplierUpdateStreamTest
|
||||
{
|
||||
private final long waitTime = 1L;
|
||||
private final TimeUnit unit = TimeUnit.SECONDS;
|
||||
private final ArrayList<String> dimensions = new ArrayList<String>();
|
||||
private InputSupplier testCaseSupplier;
|
||||
Map<String, Object> expectedAnswer = new HashMap<String, Object>();
|
||||
String timeDimension;
|
||||
|
||||
@BeforeClass
|
||||
public void setUp()
|
||||
{
|
||||
timeDimension = "time";
|
||||
testCaseSupplier = new TestCaseSupplier(
|
||||
"{\"item1\": \"value1\","
|
||||
+ "\"item2\":2,"
|
||||
+ "\"time\":1372121562 }"
|
||||
);
|
||||
|
||||
dimensions.add("item1");
|
||||
dimensions.add("item2");
|
||||
dimensions.add("time");
|
||||
|
||||
expectedAnswer.put("item1", "value1");
|
||||
expectedAnswer.put("item2", 2);
|
||||
expectedAnswer.put("time", 1372121562);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void basicIngestionCheck() throws Exception
|
||||
{
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(
|
||||
testCaseSupplier,
|
||||
timeDimension
|
||||
);
|
||||
updateStream.start();
|
||||
Map<String, Object> insertedRow = updateStream.pollFromQueue(waitTime, unit);
|
||||
Assert.assertEquals(expectedAnswer, insertedRow);
|
||||
}
|
||||
|
||||
//If a timestamp is missing, we should throw away the event
|
||||
@Test
|
||||
public void missingTimeStampCheck()
|
||||
{
|
||||
testCaseSupplier = new TestCaseSupplier(
|
||||
"{\"item1\": \"value1\","
|
||||
+ "\"item2\":2}"
|
||||
);
|
||||
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(
|
||||
testCaseSupplier,
|
||||
timeDimension
|
||||
);
|
||||
updateStream.start();
|
||||
Assert.assertEquals(updateStream.getQueueSize(), 0);
|
||||
}
|
||||
|
||||
//If any other value is missing, we should still add the event and process it properly
|
||||
@Test
|
||||
public void otherNullValueCheck() throws Exception
|
||||
{
|
||||
testCaseSupplier = new TestCaseSupplier(
|
||||
"{\"item1\": \"value1\","
|
||||
+ "\"time\":1372121562 }"
|
||||
);
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(
|
||||
testCaseSupplier,
|
||||
timeDimension
|
||||
);
|
||||
updateStream.start();
|
||||
Map<String, Object> insertedRow = updateStream.pollFromQueue(waitTime, unit);
|
||||
Map<String, Object> expectedAnswer = new HashMap<String, Object>();
|
||||
expectedAnswer.put("item1", "value1");
|
||||
expectedAnswer.put("time", 1372121562);
|
||||
Assert.assertEquals(expectedAnswer, insertedRow);
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -0,0 +1,81 @@
|
||||
/*
|
||||
* 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 UpdateStream
|
||||
{
|
||||
|
||||
private final InputSupplierUpdateStream updateStream;
|
||||
private Map<String, String> renamedDimensions;
|
||||
|
||||
public RenamingKeysUpdateStream(
|
||||
InputSupplierUpdateStream 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();
|
||||
|
||||
}
|
||||
|
||||
public void start()
|
||||
{
|
||||
updateStream.start();
|
||||
}
|
||||
|
||||
public void stop(){
|
||||
updateStream.stop();
|
||||
}
|
||||
}
|
@ -0,0 +1,39 @@
|
||||
/*
|
||||
* 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 java.util.Map;
|
||||
|
||||
public class RenamingKeysUpdateStreamFactory implements UpdateStreamFactory
|
||||
{
|
||||
private InputSupplierUpdateStreamFactory updateStreamFactory;
|
||||
private Map<String, String> renamedDimensions;
|
||||
|
||||
public RenamingKeysUpdateStreamFactory(InputSupplierUpdateStreamFactory updateStreamFactory, Map<String, String> renamedDimensions)
|
||||
{
|
||||
this.updateStreamFactory = updateStreamFactory;
|
||||
this.renamedDimensions = renamedDimensions;
|
||||
}
|
||||
|
||||
public RenamingKeysUpdateStream build()
|
||||
{
|
||||
return new RenamingKeysUpdateStream(updateStreamFactory.build(), renamedDimensions);
|
||||
}
|
||||
}
|
@ -0,0 +1,94 @@
|
||||
/*
|
||||
* 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.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
|
||||
{
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(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.start();
|
||||
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
|
||||
{
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(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
|
||||
{
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(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());
|
||||
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,42 @@
|
||||
/*
|
||||
* 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.io.InputSupplier;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
|
||||
public class TestCaseSupplier implements InputSupplier<BufferedReader>
|
||||
{
|
||||
private final String testString;
|
||||
|
||||
public TestCaseSupplier(String testString)
|
||||
{
|
||||
this.testString = testString;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferedReader getInput() throws IOException
|
||||
{
|
||||
return new BufferedReader(new StringReader(testString));
|
||||
}
|
||||
}
|
@ -0,0 +1,31 @@
|
||||
/*
|
||||
* 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 java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public interface UpdateStream
|
||||
{
|
||||
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException;
|
||||
public String getTimeDimension();
|
||||
public void start();
|
||||
public void stop();
|
||||
|
||||
}
|
@ -0,0 +1,24 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
public interface UpdateStreamFactory
|
||||
{
|
||||
public UpdateStream build();
|
||||
}
|
@ -0,0 +1,134 @@
|
||||
/*
|
||||
* 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.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.metamx.common.parsers.TimestampParser;
|
||||
import com.metamx.druid.guava.Runnables;
|
||||
import com.metamx.druid.input.InputRow;
|
||||
import com.metamx.druid.input.MapBasedInputRow;
|
||||
import com.metamx.druid.realtime.firehose.Firehose;
|
||||
import com.metamx.druid.realtime.firehose.FirehoseFactory;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@JsonTypeName("webstream")
|
||||
public class WebFirehoseFactory implements FirehoseFactory
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(WebFirehoseFactory.class);
|
||||
private final String timeFormat;
|
||||
private final UpdateStreamFactory factory;
|
||||
private final long queueWaitTime = 15L;
|
||||
|
||||
@JsonCreator
|
||||
public WebFirehoseFactory(
|
||||
@JsonProperty("url") String url,
|
||||
@JsonProperty("renamedDimensions") Map<String, String> renamedDimensions,
|
||||
@JsonProperty("timeDimension") String timeDimension,
|
||||
@JsonProperty("timeFormat") String timeFormat
|
||||
)
|
||||
{
|
||||
this(
|
||||
new RenamingKeysUpdateStreamFactory(
|
||||
new InputSupplierUpdateStreamFactory(new WebJsonSupplier(url), timeDimension),
|
||||
renamedDimensions
|
||||
), timeFormat
|
||||
);
|
||||
}
|
||||
|
||||
public WebFirehoseFactory(UpdateStreamFactory factory, String timeFormat)
|
||||
{
|
||||
this.factory = factory;
|
||||
if (timeFormat == null) {
|
||||
this.timeFormat = "auto";
|
||||
} else {
|
||||
this.timeFormat = timeFormat;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
{
|
||||
|
||||
final UpdateStream updateStream = factory.build();
|
||||
updateStream.start();
|
||||
|
||||
return new Firehose()
|
||||
{
|
||||
Map<String, Object> map;
|
||||
private final Runnable doNothingRunnable = Runnables.getNoopRunnable();
|
||||
|
||||
@Override
|
||||
public boolean hasMore()
|
||||
{
|
||||
try {
|
||||
map = updateStream.pollFromQueue(queueWaitTime, TimeUnit.SECONDS);
|
||||
return map != null;
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public InputRow nextRow()
|
||||
{
|
||||
try {
|
||||
DateTime date = TimestampParser.createTimestampParser(timeFormat)
|
||||
.apply(map.get(updateStream.getTimeDimension()).toString());
|
||||
return new MapBasedInputRow(
|
||||
date.getMillis(),
|
||||
new ArrayList(map.keySet()),
|
||||
map
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
finally {
|
||||
map = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Runnable commit()
|
||||
{
|
||||
// ephemera in, ephemera out.
|
||||
return doNothingRunnable; // reuse the same object each time
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
updateStream.stop();
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
}
|
@ -0,0 +1,223 @@
|
||||
/*
|
||||
* 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.beust.jcommander.internal.Lists;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.metamx.druid.input.InputRow;
|
||||
import com.metamx.druid.realtime.firehose.Firehose;
|
||||
import junit.framework.Assert;
|
||||
import org.joda.time.DateTime;
|
||||
import org.testng.annotations.BeforeClass;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class WebFirehoseFactoryTest
|
||||
{
|
||||
private List<String> dimensions = Lists.newArrayList();
|
||||
private WebFirehoseFactory webbie;
|
||||
private WebFirehoseFactory webbie1;
|
||||
|
||||
@BeforeClass
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
dimensions.add("item1");
|
||||
dimensions.add("item2");
|
||||
dimensions.add("time");
|
||||
webbie = new WebFirehoseFactory(
|
||||
new UpdateStreamFactory()
|
||||
{
|
||||
@Override
|
||||
public UpdateStream build()
|
||||
{
|
||||
return new MyUpdateStream(ImmutableMap.<String,Object>of("item1", "value1", "item2", 2, "time", "1372121562"));
|
||||
}
|
||||
},
|
||||
"posix"
|
||||
);
|
||||
|
||||
webbie1 = new WebFirehoseFactory(
|
||||
new UpdateStreamFactory()
|
||||
{
|
||||
@Override
|
||||
public UpdateStream build()
|
||||
{
|
||||
return new MyUpdateStream(ImmutableMap.<String,Object>of("item1", "value1", "item2", 2, "time", "1373241600000"));
|
||||
}
|
||||
},
|
||||
"auto"
|
||||
);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDimensions() throws Exception
|
||||
{
|
||||
InputRow inputRow;
|
||||
Firehose firehose = webbie.connect();
|
||||
if (firehose.hasMore()) {
|
||||
inputRow = firehose.nextRow();
|
||||
} else {
|
||||
throw new RuntimeException("queue is empty");
|
||||
}
|
||||
List<String> actualAnswer = inputRow.getDimensions();
|
||||
Collections.sort(actualAnswer);
|
||||
Assert.assertEquals(actualAnswer, dimensions);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPosixTimeStamp() throws Exception
|
||||
{
|
||||
InputRow inputRow;
|
||||
Firehose firehose = webbie.connect();
|
||||
if (firehose.hasMore()) {
|
||||
inputRow = firehose.nextRow();
|
||||
} else {
|
||||
throw new RuntimeException("queue is empty");
|
||||
}
|
||||
long expectedTime = 1372121562L * 1000L;
|
||||
Assert.assertEquals(expectedTime, inputRow.getTimestampFromEpoch());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testISOTimeStamp() throws Exception
|
||||
{
|
||||
WebFirehoseFactory webbie3 = new WebFirehoseFactory(
|
||||
new UpdateStreamFactory()
|
||||
{
|
||||
@Override
|
||||
public UpdateStream build()
|
||||
{
|
||||
return new MyUpdateStream(ImmutableMap.<String,Object>of("item1", "value1", "item2", 2, "time", "2013-07-08"));
|
||||
}
|
||||
},
|
||||
"auto"
|
||||
);
|
||||
Firehose firehose1 = webbie3.connect();
|
||||
if (firehose1.hasMore()) {
|
||||
long milliSeconds = firehose1.nextRow().getTimestampFromEpoch();
|
||||
DateTime date = new DateTime("2013-07-08");
|
||||
Assert.assertEquals(date.getMillis(), milliSeconds);
|
||||
} else {
|
||||
Assert.assertFalse("hasMore returned false", true);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAutoIsoTimeStamp() throws Exception
|
||||
{
|
||||
WebFirehoseFactory webbie2 = new WebFirehoseFactory(
|
||||
new UpdateStreamFactory()
|
||||
{
|
||||
@Override
|
||||
public UpdateStream build()
|
||||
{
|
||||
return new MyUpdateStream(ImmutableMap.<String,Object>of("item1", "value1", "item2", 2, "time", "2013-07-08"));
|
||||
}
|
||||
},
|
||||
null
|
||||
);
|
||||
Firehose firehose2 = webbie2.connect();
|
||||
if (firehose2.hasMore()) {
|
||||
long milliSeconds = firehose2.nextRow().getTimestampFromEpoch();
|
||||
DateTime date = new DateTime("2013-07-08");
|
||||
Assert.assertEquals(date.getMillis(), milliSeconds);
|
||||
} else {
|
||||
Assert.assertFalse("hasMore returned false", true);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAutoMilliSecondsTimeStamp() throws Exception
|
||||
{
|
||||
Firehose firehose3 = webbie1.connect();
|
||||
if (firehose3.hasMore()) {
|
||||
long milliSeconds = firehose3.nextRow().getTimestampFromEpoch();
|
||||
DateTime date = new DateTime("2013-07-08");
|
||||
Assert.assertEquals(date.getMillis(), milliSeconds);
|
||||
} else {
|
||||
Assert.assertFalse("hasMore returned false", true);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetDimension() throws Exception
|
||||
{
|
||||
InputRow inputRow;
|
||||
Firehose firehose = webbie1.connect();
|
||||
if (firehose.hasMore()) {
|
||||
inputRow = firehose.nextRow();
|
||||
} else {
|
||||
throw new RuntimeException("queue is empty");
|
||||
}
|
||||
|
||||
List<String> column1 = Lists.newArrayList();
|
||||
column1.add("value1");
|
||||
Assert.assertEquals(column1, inputRow.getDimension("item1"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetFloatMetric() throws Exception
|
||||
{
|
||||
InputRow inputRow;
|
||||
Firehose firehose = webbie1.connect();
|
||||
if (firehose.hasMore()) {
|
||||
inputRow = firehose.nextRow();
|
||||
} else {
|
||||
throw new RuntimeException("queue is empty");
|
||||
}
|
||||
|
||||
Assert.assertEquals((float) 2.0, inputRow.getFloatMetric("item2"));
|
||||
}
|
||||
|
||||
private static class MyUpdateStream implements UpdateStream
|
||||
{
|
||||
private static ImmutableMap<String,Object> map;
|
||||
public MyUpdateStream(ImmutableMap<String,Object> map){
|
||||
this.map=map;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException
|
||||
{
|
||||
return map;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTimeDimension()
|
||||
{
|
||||
return "time";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start()
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop()
|
||||
{
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,58 @@
|
||||
/*
|
||||
* 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.io.InputSupplier;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.net.URL;
|
||||
import java.net.URLConnection;
|
||||
import java.util.List;
|
||||
|
||||
public class WebJsonSupplier implements InputSupplier<BufferedReader>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(WebJsonSupplier.class);
|
||||
|
||||
private String urlString;
|
||||
private URL url;
|
||||
|
||||
public WebJsonSupplier(String urlString)
|
||||
{
|
||||
this.urlString = urlString;
|
||||
try {
|
||||
this.url = new URL(urlString);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(e,"Malformed url");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferedReader getInput() throws IOException
|
||||
{
|
||||
URL url = new URL(urlString);
|
||||
URLConnection connection = url.openConnection();
|
||||
connection.setDoInput(true);
|
||||
return new BufferedReader(new InputStreamReader(url.openStream()));
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
Loading…
x
Reference in New Issue
Block a user