mirror of https://github.com/apache/druid.git
interfaces added and tests simplified
This commit is contained in:
parent
41cb115d60
commit
6d000fc4c2
|
@ -0,0 +1,103 @@
|
|||
/*
|
||||
* 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.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.io.InputSupplier;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
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;
|
||||
|
||||
public InputSupplierUpdateStream(
|
||||
InputSupplier<BufferedReader> supplier,
|
||||
String timeDimension
|
||||
)
|
||||
{
|
||||
this.supplier = supplier;
|
||||
this.typeRef = new TypeReference<HashMap<String, Object>>()
|
||||
{
|
||||
};
|
||||
this.timeDimension = timeDimension;
|
||||
}
|
||||
|
||||
private boolean isValid(String s)
|
||||
{
|
||||
return !(s.isEmpty());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
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.error("missing timestamp");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
|
@ -29,9 +29,9 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class UpdateStreamTest
|
||||
public class InputSupplierUpdateStreamTest
|
||||
{
|
||||
private final long waitTime = 15L;
|
||||
private final long waitTime = 1L;
|
||||
private final TimeUnit unit = TimeUnit.SECONDS;
|
||||
private final ArrayList<String> dimensions = new ArrayList<String>();
|
||||
private InputSupplier testCaseSupplier;
|
||||
|
@ -61,7 +61,7 @@ public class UpdateStreamTest
|
|||
@Test
|
||||
public void basicIngestionCheck() throws Exception
|
||||
{
|
||||
UpdateStream updateStream = new UpdateStream(
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(
|
||||
testCaseSupplier,
|
||||
timeDimension
|
||||
);
|
||||
|
@ -79,7 +79,7 @@ public class UpdateStreamTest
|
|||
+ "\"item2\":2}"
|
||||
);
|
||||
|
||||
UpdateStream updateStream = new UpdateStream(
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(
|
||||
testCaseSupplier,
|
||||
timeDimension
|
||||
);
|
||||
|
@ -95,7 +95,7 @@ public class UpdateStreamTest
|
|||
"{\"item1\": \"value1\","
|
||||
+ "\"time\":1372121562 }"
|
||||
);
|
||||
UpdateStream updateStream = new UpdateStream(
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(
|
||||
testCaseSupplier,
|
||||
timeDimension
|
||||
);
|
|
@ -24,14 +24,14 @@ import com.google.common.collect.Maps;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class RenamingKeysUpdateStream implements Runnable
|
||||
public class RenamingKeysUpdateStream implements UpdateStream
|
||||
{
|
||||
|
||||
private final UpdateStream updateStream;
|
||||
private final InputSupplierUpdateStream updateStream;
|
||||
private Map<String, String> renamedDimensions;
|
||||
|
||||
public RenamingKeysUpdateStream(
|
||||
UpdateStream updateStream,
|
||||
InputSupplierUpdateStream updateStream,
|
||||
Map<String, String> renamedDimensions
|
||||
)
|
||||
{
|
||||
|
|
|
@ -21,12 +21,12 @@ package druid.examples.webStream;
|
|||
|
||||
import java.util.Map;
|
||||
|
||||
public class RenamingKeysUpdateStreamFactory
|
||||
public class RenamingKeysUpdateStreamFactory implements UpdateStreamFactory
|
||||
{
|
||||
private UpdateStreamFactory updateStreamFactory;
|
||||
private InputSupplierUpdateStreamFactory updateStreamFactory;
|
||||
private Map<String, String> renamedDimensions;
|
||||
|
||||
public RenamingKeysUpdateStreamFactory(UpdateStreamFactory updateStreamFactory, Map<String, String> renamedDimensions)
|
||||
public RenamingKeysUpdateStreamFactory(InputSupplierUpdateStreamFactory updateStreamFactory, Map<String, String> renamedDimensions)
|
||||
{
|
||||
this.updateStreamFactory = updateStreamFactory;
|
||||
this.renamedDimensions = renamedDimensions;
|
||||
|
|
|
@ -49,7 +49,7 @@ public class RenamingKeysUpdateStreamTest
|
|||
@Test
|
||||
public void testPollFromQueue() throws Exception
|
||||
{
|
||||
UpdateStream updateStream = new UpdateStream(testCaseSupplier, timeDimension);
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(testCaseSupplier, timeDimension);
|
||||
Map<String, String> renamedKeys = new HashMap<String, String>();
|
||||
renamedKeys.put("item1", "i1");
|
||||
renamedKeys.put("item2", "i2");
|
||||
|
@ -69,7 +69,7 @@ public class RenamingKeysUpdateStreamTest
|
|||
@Test
|
||||
public void testGetTimeDimension() throws Exception
|
||||
{
|
||||
UpdateStream updateStream = new UpdateStream(testCaseSupplier, timeDimension);
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(testCaseSupplier, timeDimension);
|
||||
Map<String, String> renamedKeys = new HashMap<String, String>();
|
||||
renamedKeys.put("item1", "i1");
|
||||
renamedKeys.put("item2", "i2");
|
||||
|
@ -82,7 +82,7 @@ public class RenamingKeysUpdateStreamTest
|
|||
@Test
|
||||
public void testMissingTimeRename() throws Exception
|
||||
{
|
||||
UpdateStream updateStream = new UpdateStream(testCaseSupplier, timeDimension);
|
||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(testCaseSupplier, timeDimension);
|
||||
Map<String, String> renamedKeys = new HashMap<String, String>();
|
||||
renamedKeys.put("item1", "i1");
|
||||
renamedKeys.put("item2", "i2");
|
||||
|
|
|
@ -16,89 +16,15 @@
|
|||
* 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.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.InputSupplier;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
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 UpdateStream implements Runnable
|
||||
public interface UpdateStream extends Runnable
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(UpdateStream.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;
|
||||
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException;
|
||||
public String getTimeDimension();
|
||||
|
||||
public UpdateStream(
|
||||
InputSupplier<BufferedReader> supplier,
|
||||
String timeDimension
|
||||
)
|
||||
{
|
||||
this.supplier = supplier;
|
||||
this.typeRef = new TypeReference<HashMap<String, Object>>()
|
||||
{
|
||||
};
|
||||
this.timeDimension = timeDimension;
|
||||
}
|
||||
|
||||
private boolean isValid(String s)
|
||||
{
|
||||
return !(s.isEmpty());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
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.error("missing timestamp");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -16,25 +16,9 @@
|
|||
* 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;
|
||||
|
||||
public class UpdateStreamFactory
|
||||
public interface UpdateStreamFactory
|
||||
{
|
||||
private final InputSupplier inputSupplier;
|
||||
private final String timeDimension;
|
||||
|
||||
public UpdateStreamFactory(InputSupplier inputSupplier, String timeDimension)
|
||||
{
|
||||
this.inputSupplier = inputSupplier;
|
||||
this.timeDimension = timeDimension;
|
||||
}
|
||||
|
||||
public UpdateStream build()
|
||||
{
|
||||
return new UpdateStream(inputSupplier, timeDimension);
|
||||
}
|
||||
|
||||
public UpdateStream build();
|
||||
}
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.joda.time.DateTime;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
@ -45,7 +44,7 @@ public class WebFirehoseFactory implements FirehoseFactory
|
|||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(WebFirehoseFactory.class);
|
||||
private final String timeFormat;
|
||||
private final RenamingKeysUpdateStreamFactory factory;
|
||||
private final UpdateStreamFactory factory;
|
||||
private final long queueWaitTime = 15L;
|
||||
|
||||
@JsonCreator
|
||||
|
@ -58,13 +57,13 @@ public class WebFirehoseFactory implements FirehoseFactory
|
|||
{
|
||||
this(
|
||||
new RenamingKeysUpdateStreamFactory(
|
||||
new UpdateStreamFactory(new WebJsonSupplier(url), timeDimension),
|
||||
new InputSupplierUpdateStreamFactory(new WebJsonSupplier(url), timeDimension),
|
||||
renamedDimensions
|
||||
), timeFormat
|
||||
);
|
||||
}
|
||||
|
||||
public WebFirehoseFactory(RenamingKeysUpdateStreamFactory factory, String timeFormat)
|
||||
public WebFirehoseFactory(UpdateStreamFactory factory, String timeFormat)
|
||||
{
|
||||
this.factory = factory;
|
||||
if (timeFormat == null) {
|
||||
|
@ -78,9 +77,9 @@ public class WebFirehoseFactory implements FirehoseFactory
|
|||
public Firehose connect() throws IOException
|
||||
{
|
||||
|
||||
final RenamingKeysUpdateStream renamingUpdateStream = factory.build();
|
||||
final UpdateStream updateStream = factory.build();
|
||||
final ExecutorService service = Executors.newSingleThreadExecutor();
|
||||
service.submit(renamingUpdateStream);
|
||||
service.submit(updateStream);
|
||||
|
||||
return new Firehose()
|
||||
{
|
||||
|
@ -91,7 +90,7 @@ public class WebFirehoseFactory implements FirehoseFactory
|
|||
public boolean hasMore()
|
||||
{
|
||||
try {
|
||||
map = renamingUpdateStream.pollFromQueue(queueWaitTime, TimeUnit.SECONDS);
|
||||
map = updateStream.pollFromQueue(queueWaitTime, TimeUnit.SECONDS);
|
||||
return map != null;
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
|
@ -105,7 +104,7 @@ public class WebFirehoseFactory implements FirehoseFactory
|
|||
{
|
||||
try {
|
||||
DateTime date = TimestampParser.createTimestampParser(timeFormat)
|
||||
.apply(map.get(renamingUpdateStream.getTimeDimension()).toString());
|
||||
.apply(map.get(updateStream.getTimeDimension()).toString());
|
||||
return new MapBasedInputRow(
|
||||
date.getMillis(),
|
||||
new ArrayList(map.keySet()),
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
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;
|
||||
|
@ -27,36 +28,91 @@ import org.joda.time.DateTime;
|
|||
import org.testng.annotations.BeforeClass;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class WebFirehoseFactoryTest
|
||||
{
|
||||
private List<String> dimensions;
|
||||
private List<String> dimensions = Lists.newArrayList();
|
||||
private WebFirehoseFactory webbie;
|
||||
private TestCaseSupplier testCaseSupplier;
|
||||
private WebFirehoseFactory webbie1;
|
||||
|
||||
@BeforeClass
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
dimensions = new ArrayList<String>();
|
||||
dimensions.add("item1");
|
||||
dimensions.add("item2");
|
||||
dimensions.add("time");
|
||||
testCaseSupplier = new TestCaseSupplier(
|
||||
"{\"item1\":\"value1\","
|
||||
+ "\"item2\":2,"
|
||||
+ "\"time\":1372121562 }"
|
||||
webbie = new WebFirehoseFactory(
|
||||
new UpdateStreamFactory()
|
||||
{
|
||||
@Override
|
||||
public UpdateStream build()
|
||||
{
|
||||
return new UpdateStream()
|
||||
{
|
||||
@Override
|
||||
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException
|
||||
{
|
||||
return ImmutableMap.<String, Object>of("item1", "value1", "item2", 2, "time", "1372121562");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTimeDimension()
|
||||
{
|
||||
return "time";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
},
|
||||
"posix"
|
||||
);
|
||||
|
||||
webbie1 = new WebFirehoseFactory(
|
||||
new UpdateStreamFactory()
|
||||
{
|
||||
@Override
|
||||
public UpdateStream build()
|
||||
{
|
||||
return new UpdateStream()
|
||||
{
|
||||
@Override
|
||||
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException
|
||||
{
|
||||
return ImmutableMap.<String, Object>of("item1", "value1", "item2", 2, "time", "1373241600000");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTimeDimension()
|
||||
{
|
||||
return "time";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
},
|
||||
"auto"
|
||||
);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
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();
|
||||
|
@ -72,8 +128,6 @@ public class WebFirehoseFactoryTest
|
|||
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();
|
||||
|
@ -87,15 +141,37 @@ public class WebFirehoseFactoryTest
|
|||
@Test
|
||||
public void testISOTimeStamp() throws Exception
|
||||
{
|
||||
testCaseSupplier = new TestCaseSupplier(
|
||||
"{\"item1\": \"value1\","
|
||||
+ "\"item2\":2,"
|
||||
+ "\"time\":\"2013-07-08\"}"
|
||||
);
|
||||
WebFirehoseFactory webbie4 = new WebFirehoseFactory(
|
||||
new UpdateStreamFactory()
|
||||
{
|
||||
@Override
|
||||
public UpdateStream build()
|
||||
{
|
||||
return new UpdateStream()
|
||||
{
|
||||
@Override
|
||||
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException
|
||||
{
|
||||
return ImmutableMap.<String, Object>of("item1", "value1", "item2", 2, "time", "2013-07-08");
|
||||
}
|
||||
|
||||
UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, "time");
|
||||
webbie = new WebFirehoseFactory(new RenamingKeysUpdateStreamFactory(updateStreamFactory, null), "iso");
|
||||
Firehose firehose1 = webbie.connect();
|
||||
@Override
|
||||
public String getTimeDimension()
|
||||
{
|
||||
return "time";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
},
|
||||
"auto"
|
||||
);
|
||||
Firehose firehose1 = webbie4.connect();
|
||||
if (firehose1.hasMore()) {
|
||||
long milliSeconds = firehose1.nextRow().getTimestampFromEpoch();
|
||||
DateTime date = new DateTime("2013-07-08");
|
||||
|
@ -108,15 +184,37 @@ public class WebFirehoseFactoryTest
|
|||
@Test
|
||||
public void testAutoIsoTimeStamp() throws Exception
|
||||
{
|
||||
testCaseSupplier = new TestCaseSupplier(
|
||||
"{\"item1\": \"value1\","
|
||||
+ "\"item2\":2,"
|
||||
+ "\"time\":\"2013-07-08\"}"
|
||||
);
|
||||
WebFirehoseFactory webbie5 = new WebFirehoseFactory(
|
||||
new UpdateStreamFactory()
|
||||
{
|
||||
@Override
|
||||
public UpdateStream build()
|
||||
{
|
||||
return new UpdateStream()
|
||||
{
|
||||
@Override
|
||||
public Map<String, Object> pollFromQueue(long waitTime, TimeUnit unit) throws InterruptedException
|
||||
{
|
||||
return ImmutableMap.<String, Object>of("item1", "value1", "item2", 2, "time", "2013-07-08");
|
||||
}
|
||||
|
||||
UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, "time");
|
||||
webbie = new WebFirehoseFactory(new RenamingKeysUpdateStreamFactory(updateStreamFactory, null), null);
|
||||
Firehose firehose2 = webbie.connect();
|
||||
@Override
|
||||
public String getTimeDimension()
|
||||
{
|
||||
return "time";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
},
|
||||
null
|
||||
);
|
||||
Firehose firehose2 = webbie5.connect();
|
||||
if (firehose2.hasMore()) {
|
||||
long milliSeconds = firehose2.nextRow().getTimestampFromEpoch();
|
||||
DateTime date = new DateTime("2013-07-08");
|
||||
|
@ -129,15 +227,7 @@ public class WebFirehoseFactoryTest
|
|||
@Test
|
||||
public void testAutoMilliSecondsTimeStamp() throws Exception
|
||||
{
|
||||
testCaseSupplier = new TestCaseSupplier(
|
||||
"{\"item1\": \"value1\","
|
||||
+ "\"item2\":2,"
|
||||
+ "\"time\":1373241600000}"
|
||||
);
|
||||
|
||||
UpdateStreamFactory updateStreamFactory = new UpdateStreamFactory(testCaseSupplier, "time");
|
||||
webbie = new WebFirehoseFactory(new RenamingKeysUpdateStreamFactory(updateStreamFactory, null), null);
|
||||
Firehose firehose3 = webbie.connect();
|
||||
Firehose firehose3 = webbie1.connect();
|
||||
if (firehose3.hasMore()) {
|
||||
long milliSeconds = firehose3.nextRow().getTimestampFromEpoch();
|
||||
DateTime date = new DateTime("2013-07-08");
|
||||
|
@ -151,9 +241,7 @@ public class WebFirehoseFactoryTest
|
|||
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();
|
||||
Firehose firehose = webbie1.connect();
|
||||
if (firehose.hasMore()) {
|
||||
inputRow = firehose.nextRow();
|
||||
} else {
|
||||
|
@ -169,9 +257,7 @@ public class WebFirehoseFactoryTest
|
|||
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();
|
||||
Firehose firehose = webbie1.connect();
|
||||
if (firehose.hasMore()) {
|
||||
inputRow = firehose.nextRow();
|
||||
} else {
|
||||
|
|
Loading…
Reference in New Issue