Merge branch 'master' of github.com:metamx/druid

This commit is contained in:
Fangjin Yang 2013-03-15 14:29:19 -07:00
commit d316dfc8e1
92 changed files with 2438 additions and 1366 deletions

BIN
DruidCorporateCLA.pdf Normal file

Binary file not shown.

BIN
DruidIndividualCLA.pdf Normal file

Binary file not shown.

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.metamx.common.IAE;
import com.metamx.common.ISE;
import com.metamx.druid.client.DataSegment;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.InputStreamResponseHandler;
@ -106,6 +107,10 @@ public class IndexingServiceClient
{
try {
final ServiceInstance instance = serviceProvider.getInstance();
if (instance == null) {
throw new ISE("Cannot find instance of indexingService");
}
return String.format("http://%s:%s/mmx/merger/v1", instance.getAddress(), instance.getPort());
}
catch (Exception e) {

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -17,31 +17,36 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.realtime;
package com.metamx.druid.guava;
import org.skife.config.Config;
import org.skife.config.Default;
import java.util.concurrent.Callable;
/**
*/
public abstract class MetadataUpdaterConfig
public abstract class ThreadRenamingCallable<T> implements Callable<T>
{
@Config("druid.host")
public abstract String getServerName();
private final String name;
@Config("druid.host")
public abstract String getHost();
public ThreadRenamingCallable(
String name
)
{
this.name = name;
}
@Config("druid.server.maxSize")
@Default("0")
public abstract long getMaxSize();
@Override
public final T call()
{
final Thread currThread = Thread.currentThread();
String currName = currThread.getName();
try {
currThread.setName(name);
return doCall();
}
finally {
currThread.setName(currName);
}
}
@Config("druid.database.segmentTable")
public abstract String getSegmentTable();
@Config("druid.zk.paths.announcementsPath")
public abstract String getAnnounceLocation();
@Config("druid.zk.paths.servedSegmentsPath")
public abstract String getServedSegmentsLocation();
public abstract T doCall();
}

View File

@ -62,7 +62,7 @@ public class CompressionUtils
zipOut = new ZipOutputStream(new FileOutputStream(outputZipFile));
File[] files = directory.listFiles();
for (File file : files) {
log.info("Adding file[%s] with size[%,d]. Total size[%,d]", file, file.length(), totalSize);
log.info("Adding file[%s] with size[%,d]. Total size so far[%,d]", file, file.length(), totalSize);
if (file.length() >= Integer.MAX_VALUE) {
zipOut.close();
outputZipFile.delete();

View File

@ -24,11 +24,11 @@
<artifactId>druid-services</artifactId>
<name>druid-services</name>
<description>druid-services</description>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
</parent>
<modules>

View File

@ -9,7 +9,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid-examples</artifactId>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -1,21 +1,18 @@
package druid.examples;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.metamx.common.config.Config;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.ZKPhoneBook;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.log.LogLevelAdjuster;
import com.metamx.druid.realtime.MetadataUpdater;
import com.metamx.druid.realtime.MetadataUpdaterConfig;
import com.metamx.druid.realtime.RealtimeNode;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.log.LogLevelAdjuster;
import com.metamx.druid.realtime.RealtimeNode;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.SegmentPublisher;
import com.metamx.phonebook.PhoneBook;
import java.io.File;
import java.io.IOException;
@ -45,13 +42,11 @@ public class RealtimeStandaloneMain
};
rn.setPhoneBook(dummyPhoneBook);
MetadataUpdater dummyMetadataUpdater =
new MetadataUpdater(new DefaultObjectMapper(),
Config.createFactory(Initialization.loadProperties()).build(MetadataUpdaterConfig.class),
dummyPhoneBook,
null) {
SegmentAnnouncer dummySegmentAnnouncer =
new SegmentAnnouncer()
{
@Override
public void publishSegment(DataSegment segment) throws IOException
public void announceSegment(DataSegment segment) throws IOException
{
// do nothing
}
@ -61,17 +56,20 @@ public class RealtimeStandaloneMain
{
// do nothing
}
};
SegmentPublisher dummySegmentPublisher =
new SegmentPublisher()
{
@Override
public void announceSegment(DataSegment segment) throws IOException
public void publishSegment(DataSegment segment) throws IOException
{
// do nothing
}
};
// dummyMetadataUpdater will not send updates to db because standalone demo has no db
rn.setMetadataUpdater(dummyMetadataUpdater);
// dummySegmentPublisher will not send updates to db because standalone demo has no db
rn.setSegmentAnnouncer(dummySegmentAnnouncer);
rn.setSegmentPublisher(dummySegmentPublisher);
rn.setDataSegmentPusher(
new DataSegmentPusher()
{

View File

@ -9,7 +9,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid-examples</artifactId>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -1,22 +1,19 @@
package druid.examples;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.metamx.common.config.Config;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.ZKPhoneBook;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.log.LogLevelAdjuster;
import com.metamx.druid.realtime.MetadataUpdater;
import com.metamx.druid.realtime.MetadataUpdaterConfig;
import com.metamx.druid.realtime.RealtimeNode;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.log.LogLevelAdjuster;
import com.metamx.druid.realtime.RealtimeNode;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.SegmentPublisher;
import com.metamx.phonebook.PhoneBook;
import druid.examples.twitter.TwitterSpritzerFirehoseFactory;
import java.io.File;
import java.io.IOException;
@ -47,35 +44,34 @@ public class RealtimeStandaloneMain
};
rn.setPhoneBook(dummyPhoneBook);
MetadataUpdater dummyMetadataUpdater =
new MetadataUpdater(
new DefaultObjectMapper(),
Config.createFactory(Initialization.loadProperties()).build(MetadataUpdaterConfig.class),
dummyPhoneBook,
null
) {
final SegmentAnnouncer dummySegmentAnnouncer =
new SegmentAnnouncer()
{
@Override
public void announceSegment(DataSegment segment) throws IOException
{
// do nothing
}
@Override
public void unannounceSegment(DataSegment segment) throws IOException
{
// do nothing
}
};
SegmentPublisher dummySegmentPublisher =
new SegmentPublisher()
{
@Override
public void publishSegment(DataSegment segment) throws IOException
{
// do nothing
}
@Override
public void unannounceSegment(DataSegment segment) throws IOException
{
// do nothing
}
@Override
public void announceSegment(DataSegment segment) throws IOException
{
// do nothing
}
};
// dummyMetadataUpdater will not send updates to db because standalone demo has no db
rn.setMetadataUpdater(dummyMetadataUpdater);
// dummySegmentPublisher will not send updates to db because standalone demo has no db
rn.setSegmentAnnouncer(dummySegmentAnnouncer);
rn.setSegmentPublisher(dummySegmentPublisher);
rn.setDataSegmentPusher(
new DataSegmentPusher()
{

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
</parent>
<dependencies>
@ -71,6 +71,10 @@
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
<!-- Tests -->
<dependency>

View File

@ -204,10 +204,31 @@ public class IndexIO
final int version = getVersionFromDir(toConvert);
switch (version) {
case 1:
case 2:
case 3:
final String mappableDirName = "mappable";
if (toConvert.getName().equals(mappableDirName)) {
throw new ISE("Infinite recursion at play! OMFG quit it, please, it hurts!");
}
File mappable = new File(toConvert, mappableDirName);
final Index index = readIndex(toConvert);
storeLatest(index, mappable);
return convertSegment(mappable, converted);
case 4:
case 5:
case 6:
case 7:
log.info("Old version, re-persisting.");
IndexMerger.append(Arrays.<IndexableAdapter>asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))), converted);
return true;
case 8:
DefaultIndexIOHandler.convertV8toV9(toConvert, converted);
return true;
default:
log.info("Version[%s], skipping.", version);
return false;
}
}

View File

@ -310,9 +310,11 @@ public class IndexMerger
throw new ISE("Couldn't make outdir[%s].", outDir);
}
/*
if (indexes.size() < 2) {
throw new ISE("Too few indexes provided for append [%d].", indexes.size());
}
*/
final List<String> mergedDimensions = mergeIndexed(
Lists.transform(

View File

@ -19,12 +19,12 @@
package com.metamx.druid.index.v1;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.io.Closeables;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import com.metamx.druid.index.QueryableIndex;
import com.metamx.druid.index.column.BitmapIndex;
import com.metamx.druid.index.column.Column;
@ -44,6 +44,7 @@ import org.joda.time.Interval;
import java.io.Closeable;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
@ -52,13 +53,35 @@ import java.util.Set;
*/
public class QueryableIndexIndexableAdapter implements IndexableAdapter
{
private static final Logger log = new Logger(QueryableIndexIndexableAdapter.class);
private final int numRows;
private final QueryableIndex input;
private final List<String> availableDimensions;
public QueryableIndexIndexableAdapter(QueryableIndex input)
{
this.input = input;
numRows = input.getNumRows();
// It appears possible that the dimensions have some columns listed which do not have a DictionaryEncodedColumn
// This breaks current logic, but should be fine going forward. This is a work-around to make things work
// in the current state. This code shouldn't be needed once github tracker issue #55 is finished.
this.availableDimensions = Lists.newArrayList();
for (String dim : input.getAvailableDimensions()) {
final Column col = input.getColumn(dim);
if (col == null) {
log.warn("Wtf!? column[%s] didn't exist!?!?!?", dim);
}
else if (col.getDictionaryEncoding() != null) {
availableDimensions.add(dim);
}
else {
log.info("No dictionary on dimension[%s]", dim);
}
}
}
@Override
@ -76,7 +99,7 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
@Override
public Indexed<String> getAvailableDimensions()
{
return input.getAvailableDimensions();
return new ListIndexed<String>(availableDimensions, String.class);
}
@Override
@ -161,7 +184,7 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
{
dimensions = Maps.newLinkedHashMap();
for (String dim : input.getAvailableDimensions()) {
for (String dim : getAvailableDimensions()) {
dimensions.put(dim, input.getColumn(dim).getDictionaryEncoding());
}

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
</parent>
<dependencies>
@ -182,25 +182,4 @@
<artifactId>curator-test</artifactId>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<outputFile>
${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar
</outputFile>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View File

@ -17,9 +17,9 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator;
package com.metamx.druid.merger.common;
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
import com.metamx.emitter.EmittingLogger;
import org.joda.time.Duration;

View File

@ -17,9 +17,9 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator;
package com.metamx.druid.merger.common;
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
/**
*/

View File

@ -22,6 +22,7 @@ package com.metamx.druid.merger.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Maps;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.MutableServerView;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.MMappedQueryableIndexFactory;
import com.metamx.druid.loading.S3DataSegmentPuller;
@ -33,6 +34,8 @@ import com.metamx.druid.merger.common.actions.TaskActionClient;
import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.emitter.service.ServiceEmitter;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
@ -52,6 +55,9 @@ public class TaskToolbox
private final RestS3Service s3Client;
private final DataSegmentPusher segmentPusher;
private final DataSegmentKiller dataSegmentKiller;
private final SegmentAnnouncer segmentAnnouncer;
private final MutableServerView newSegmentServerView;
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
private final ObjectMapper objectMapper;
public TaskToolbox(
@ -62,6 +68,9 @@ public class TaskToolbox
RestS3Service s3Client,
DataSegmentPusher segmentPusher,
DataSegmentKiller dataSegmentKiller,
SegmentAnnouncer segmentAnnouncer,
MutableServerView newSegmentServerView,
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
ObjectMapper objectMapper
)
{
@ -72,6 +81,9 @@ public class TaskToolbox
this.s3Client = s3Client;
this.segmentPusher = segmentPusher;
this.dataSegmentKiller = dataSegmentKiller;
this.segmentAnnouncer = segmentAnnouncer;
this.newSegmentServerView = newSegmentServerView;
this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
this.objectMapper = objectMapper;
}
@ -100,6 +112,21 @@ public class TaskToolbox
return dataSegmentKiller;
}
public SegmentAnnouncer getSegmentAnnouncer()
{
return segmentAnnouncer;
}
public MutableServerView getNewSegmentServerView()
{
return newSegmentServerView;
}
public QueryRunnerFactoryConglomerate getQueryRunnerFactoryConglomerate()
{
return queryRunnerFactoryConglomerate;
}
public ObjectMapper getObjectMapper()
{
return objectMapper;

View File

@ -20,11 +20,14 @@
package com.metamx.druid.merger.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.druid.client.MutableServerView;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.emitter.service.ServiceEmitter;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
@ -39,6 +42,9 @@ public class TaskToolboxFactory
private final RestS3Service s3Client;
private final DataSegmentPusher segmentPusher;
private final DataSegmentKiller dataSegmentKiller;
private final SegmentAnnouncer segmentAnnouncer;
private final MutableServerView newSegmentServerView;
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
private final ObjectMapper objectMapper;
public TaskToolboxFactory(
@ -48,6 +54,9 @@ public class TaskToolboxFactory
RestS3Service s3Client,
DataSegmentPusher segmentPusher,
DataSegmentKiller dataSegmentKiller,
SegmentAnnouncer segmentAnnouncer,
MutableServerView newSegmentServerView,
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
ObjectMapper objectMapper
)
{
@ -57,6 +66,9 @@ public class TaskToolboxFactory
this.s3Client = s3Client;
this.segmentPusher = segmentPusher;
this.dataSegmentKiller = dataSegmentKiller;
this.segmentAnnouncer = segmentAnnouncer;
this.newSegmentServerView = newSegmentServerView;
this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
this.objectMapper = objectMapper;
}
@ -75,6 +87,9 @@ public class TaskToolboxFactory
s3Client,
segmentPusher,
dataSegmentKiller,
segmentAnnouncer,
newSegmentServerView,
queryRunnerFactoryConglomerate,
objectMapper
);
}

View File

@ -4,6 +4,8 @@ import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskStorage;
import com.metamx.emitter.EmittingLogger;
import java.io.IOException;
public class LocalTaskActionClient implements TaskActionClient
{
private final Task task;
@ -20,8 +22,10 @@ public class LocalTaskActionClient implements TaskActionClient
}
@Override
public <RetType> RetType submit(TaskAction<RetType> taskAction)
public <RetType> RetType submit(TaskAction<RetType> taskAction) throws IOException
{
log.info("Performing action for task[%s]: %s", task.getId(), taskAction);
final RetType ret = taskAction.perform(task, toolbox);
// Add audit log

View File

@ -1,15 +1,14 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Throwables;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import org.joda.time.Interval;
public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
public class LockAcquireAction implements TaskAction<TaskLock>
{
private final Interval interval;
@ -27,18 +26,29 @@ public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
return interval;
}
public TypeReference<Optional<TaskLock>> getReturnTypeReference()
public TypeReference<TaskLock> getReturnTypeReference()
{
return new TypeReference<Optional<TaskLock>>() {};
return new TypeReference<TaskLock>()
{
};
}
@Override
public Optional<TaskLock> perform(Task task, TaskActionToolbox toolbox)
public TaskLock perform(Task task, TaskActionToolbox toolbox)
{
try {
return toolbox.getTaskLockbox().tryLock(task, interval);
} catch (Exception e) {
return toolbox.getTaskLockbox().lock(task, interval);
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
}
}
@Override
public String toString()
{
return "LockAcquireAction{" +
"interval=" + interval +
'}';
}
}

View File

@ -20,10 +20,12 @@ public class LockListAction implements TaskAction<List<TaskLock>>
@Override
public List<TaskLock> perform(Task task, TaskActionToolbox toolbox)
{
try {
return toolbox.getTaskLockbox().findLocksForTask(task);
} catch (Exception e) {
throw Throwables.propagate(e);
}
return toolbox.getTaskLockbox().findLocksForTask(task);
}
@Override
public String toString()
{
return "LockListAction{}";
}
}

View File

@ -1,15 +1,11 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Throwables;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.metamx.druid.merger.common.task.Task;
import org.joda.time.Interval;
import java.util.List;
public class LockReleaseAction implements TaskAction<Void>
{
private final Interval interval;
@ -36,11 +32,15 @@ public class LockReleaseAction implements TaskAction<Void>
@Override
public Void perform(Task task, TaskActionToolbox toolbox)
{
try {
toolbox.getTaskLockbox().unlock(task, interval);
return null;
} catch (Exception e) {
throw Throwables.propagate(e);
}
toolbox.getTaskLockbox().unlock(task, interval);
return null;
}
@Override
public String toString()
{
return "LockReleaseAction{" +
"interval=" + interval +
'}';
}
}

View File

@ -5,54 +5,100 @@ import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import com.metamx.druid.merger.common.RetryPolicy;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.ToStringResponseHandler;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.netflix.curator.x.discovery.ServiceInstance;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.joda.time.Duration;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Map;
import java.util.concurrent.ExecutionException;
public class RemoteTaskActionClient implements TaskActionClient
{
private final Task task;
private final HttpClient httpClient;
private final ServiceProvider serviceProvider;
private final RetryPolicyFactory retryPolicyFactory;
private final ObjectMapper jsonMapper;
private static final Logger log = new Logger(RemoteTaskActionClient.class);
public RemoteTaskActionClient(Task task, HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper)
public RemoteTaskActionClient(
Task task,
HttpClient httpClient,
ServiceProvider serviceProvider,
RetryPolicyFactory retryPolicyFactory,
ObjectMapper jsonMapper
)
{
this.task = task;
this.httpClient = httpClient;
this.serviceProvider = serviceProvider;
this.retryPolicyFactory = retryPolicyFactory;
this.jsonMapper = jsonMapper;
}
@Override
public <RetType> RetType submit(TaskAction<RetType> taskAction)
public <RetType> RetType submit(TaskAction<RetType> taskAction) throws IOException
{
try {
byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction));
log.info("Performing action for task[%s]: %s", task.getId(), taskAction);
final String response = httpClient.post(getServiceUri().toURL())
.setContent("application/json", dataToSend)
.go(new ToStringResponseHandler(Charsets.UTF_8))
.get();
byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction));
final Map<String, Object> responseDict = jsonMapper.readValue(
response,
new TypeReference<Map<String, Object>>() {}
);
final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy();
return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference());
}
catch (Exception e) {
throw Throwables.propagate(e);
while (true) {
try {
final URI serviceUri;
try {
serviceUri = getServiceUri();
}
catch (Exception e) {
throw new IOException("Failed to locate service uri", e);
}
final String response;
try {
response = httpClient.post(serviceUri.toURL())
.setContent("application/json", dataToSend)
.go(new ToStringResponseHandler(Charsets.UTF_8))
.get();
}
catch (Exception e) {
Throwables.propagateIfInstanceOf(e.getCause(), IOException.class);
throw Throwables.propagate(e);
}
final Map<String, Object> responseDict = jsonMapper.readValue(
response,
new TypeReference<Map<String, Object>>() {}
);
return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference());
} catch(IOException e) {
log.warn(e, "Exception submitting action for task: %s", task.getId());
if (retryPolicy.hasExceededRetryThreshold()) {
throw e;
} else {
try {
final long sleepTime = retryPolicy.getAndIncrementRetryDelay().getMillis();
log.info("Will try again in %s.", new Duration(sleepTime).toString());
Thread.sleep(sleepTime);
}
catch (InterruptedException e2) {
throw Throwables.propagate(e2);
}
}
}
}
}

View File

@ -21,6 +21,7 @@ package com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.http.client.HttpClient;
import com.netflix.curator.x.discovery.ServiceProvider;
@ -30,18 +31,25 @@ public class RemoteTaskActionClientFactory implements TaskActionClientFactory
{
private final HttpClient httpClient;
private final ServiceProvider serviceProvider;
private final RetryPolicyFactory retryPolicyFactory;
private final ObjectMapper jsonMapper;
public RemoteTaskActionClientFactory(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper)
public RemoteTaskActionClientFactory(
HttpClient httpClient,
ServiceProvider serviceProvider,
RetryPolicyFactory retryPolicyFactory,
ObjectMapper jsonMapper
)
{
this.httpClient = httpClient;
this.serviceProvider = serviceProvider;
this.retryPolicyFactory = retryPolicyFactory;
this.jsonMapper = jsonMapper;
}
@Override
public TaskActionClient create(Task task)
{
return new RemoteTaskActionClient(task, httpClient, serviceProvider, jsonMapper);
return new RemoteTaskActionClient(task, httpClient, serviceProvider, retryPolicyFactory, jsonMapper);
}
}

View File

@ -1,23 +1,19 @@
package com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.metamx.common.ISE;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.emitter.service.ServiceMetricEvent;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
import com.metamx.common.ISE;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.emitter.service.ServiceMetricEvent;
import java.util.List;
import java.io.IOException;
import java.util.Set;
public class SegmentInsertAction implements TaskAction<Void>
public class SegmentInsertAction implements TaskAction<Set<DataSegment>>
{
@JsonIgnore
private final Set<DataSegment> segments;
@ -57,34 +53,38 @@ public class SegmentInsertAction implements TaskAction<Void>
return new SegmentInsertAction(segments, _allowOlderVersions);
}
public TypeReference<Void> getReturnTypeReference()
public TypeReference<Set<DataSegment>> getReturnTypeReference()
{
return new TypeReference<Void>() {};
return new TypeReference<Set<DataSegment>>() {};
}
@Override
public Void perform(Task task, TaskActionToolbox toolbox)
public Set<DataSegment> perform(Task task, TaskActionToolbox toolbox) throws IOException
{
if(!toolbox.taskLockCoversSegments(task, segments, allowOlderVersions)) {
throw new ISE("Segments not covered by locks for task: %s", task.getId());
throw new ISE("Segments not covered by locks for task[%s]: %s", task.getId(), segments);
}
try {
toolbox.getMergerDBCoordinator().announceHistoricalSegments(segments);
final Set<DataSegment> retVal = toolbox.getMergerDBCoordinator().announceHistoricalSegments(segments);
// Emit metrics
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
.setUser2(task.getDataSource())
.setUser4(task.getType());
// Emit metrics
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
.setUser2(task.getDataSource())
.setUser4(task.getType());
for (DataSegment segment : segments) {
metricBuilder.setUser5(segment.getInterval().toString());
toolbox.getEmitter().emit(metricBuilder.build("indexer/segment/bytes", segment.getSize()));
}
return null;
} catch (Exception e) {
throw Throwables.propagate(e);
for (DataSegment segment : segments) {
metricBuilder.setUser5(segment.getInterval().toString());
toolbox.getEmitter().emit(metricBuilder.build("indexer/segment/bytes", segment.getSize()));
}
return retVal;
}
@Override
public String toString()
{
return "SegmentInsertAction{" +
"segments=" + segments +
'}';
}
}

View File

@ -1,13 +1,13 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Throwables;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.task.Task;
import org.joda.time.Interval;
import java.io.IOException;
import java.util.List;
public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
@ -43,12 +43,17 @@ public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
}
@Override
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox)
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox) throws IOException
{
try {
return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval);
} catch (Exception e) {
throw Throwables.propagate(e);
}
return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval);
}
@Override
public String toString()
{
return "SegmentListUnusedAction{" +
"dataSource='" + dataSource + '\'' +
", interval=" + interval +
'}';
}
}

View File

@ -1,13 +1,13 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Throwables;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.task.Task;
import org.joda.time.Interval;
import java.io.IOException;
import java.util.List;
public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
@ -43,12 +43,17 @@ public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
}
@Override
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox)
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox) throws IOException
{
try {
return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval);
} catch (Exception e) {
throw Throwables.propagate(e);
}
return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval);
}
@Override
public String toString()
{
return "SegmentListUsedAction{" +
"dataSource='" + dataSource + '\'' +
", interval=" + interval +
'}';
}
}

View File

@ -1,19 +1,15 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.metamx.common.ISE;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.emitter.service.ServiceMetricEvent;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
import com.metamx.common.ISE;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.emitter.service.ServiceMetricEvent;
import java.util.List;
import java.io.IOException;
import java.util.Set;
public class SegmentNukeAction implements TaskAction<Void>
@ -40,28 +36,32 @@ public class SegmentNukeAction implements TaskAction<Void>
}
@Override
public Void perform(Task task, TaskActionToolbox toolbox)
public Void perform(Task task, TaskActionToolbox toolbox) throws IOException
{
if(!toolbox.taskLockCoversSegments(task, segments, true)) {
throw new ISE("Segments not covered by locks for task: %s", task.getId());
}
try {
toolbox.getMergerDBCoordinator().deleteSegments(segments);
toolbox.getMergerDBCoordinator().deleteSegments(segments);
// Emit metrics
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
.setUser2(task.getDataSource())
.setUser4(task.getType());
// Emit metrics
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
.setUser2(task.getDataSource())
.setUser4(task.getType());
for (DataSegment segment : segments) {
metricBuilder.setUser5(segment.getInterval().toString());
toolbox.getEmitter().emit(metricBuilder.build("indexer/segmentNuked/bytes", segment.getSize()));
}
return null;
} catch (Exception e) {
throw Throwables.propagate(e);
for (DataSegment segment : segments) {
metricBuilder.setUser5(segment.getInterval().toString());
toolbox.getEmitter().emit(metricBuilder.build("indexer/segmentNuked/bytes", segment.getSize()));
}
return null;
}
@Override
public String toString()
{
return "SegmentNukeAction{" +
"segments=" + segments +
'}';
}
}

View File

@ -1,11 +1,10 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableList;
import com.metamx.druid.merger.common.task.Task;
import java.util.List;
@ -35,14 +34,18 @@ public class SpawnTasksAction implements TaskAction<Void>
@Override
public Void perform(Task task, TaskActionToolbox toolbox)
{
try {
for(final Task newTask : newTasks) {
toolbox.getTaskQueue().add(newTask);
}
return null;
} catch (Exception e) {
throw Throwables.propagate(e);
for(final Task newTask : newTasks) {
toolbox.getTaskQueue().add(newTask);
}
return null;
}
@Override
public String toString()
{
return "SpawnTasksAction{" +
"newTasks=" + newTasks +
'}';
}
}

View File

@ -5,6 +5,8 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.core.type.TypeReference;
import java.io.IOException;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "lockAcquire", value = LockAcquireAction.class),
@ -19,5 +21,5 @@ import com.fasterxml.jackson.core.type.TypeReference;
public interface TaskAction<RetType>
{
public TypeReference<RetType> getReturnTypeReference(); // T_T
public RetType perform(Task task, TaskActionToolbox toolbox);
public RetType perform(Task task, TaskActionToolbox toolbox) throws IOException;
}

View File

@ -1,6 +1,8 @@
package com.metamx.druid.merger.common.actions;
import java.io.IOException;
public interface TaskActionClient
{
public <RetType> RetType submit(TaskAction<RetType> taskAction);
public <RetType> RetType submit(TaskAction<RetType> taskAction) throws IOException;
}

View File

@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator.config;
package com.metamx.druid.merger.common.config;
import org.joda.time.Duration;
import org.skife.config.Config;
@ -27,15 +27,15 @@ import org.skife.config.Default;
*/
public abstract class RetryPolicyConfig
{
@Config("druid.indexer.retry.minWaitMillis")
@Config("${base_path}.retry.minWaitMillis")
@Default("PT1M") // 1 minute
public abstract Duration getRetryMinDuration();
@Config("druid.indexer.retry.maxWaitMillis")
@Config("${base_path}.retry.maxWaitMillis")
@Default("PT10M") // 10 minutes
public abstract Duration getRetryMaxDuration();
@Config("druid.indexer.retry.maxRetryCount")
@Config("${base_path}.retry.maxRetryCount")
@Default("10")
public abstract long getMaxRetryCount();
}

View File

@ -39,10 +39,10 @@ import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.realtime.FireDepartmentMetrics;
import com.metamx.druid.realtime.FireHydrant;
import com.metamx.druid.realtime.Plumber;
import com.metamx.druid.realtime.PlumberSchool;
import com.metamx.druid.realtime.plumber.Plumber;
import com.metamx.druid.realtime.plumber.PlumberSchool;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.realtime.Sink;
import com.metamx.druid.realtime.plumber.Sink;
import org.apache.commons.io.FileUtils;
@ -84,16 +84,22 @@ public class YeOldePlumberSchool implements PlumberSchool
public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics)
{
// There can be only one.
final Sink theSink = new Sink(interval, schema);
final Sink theSink = new Sink(interval, schema, version);
// Temporary directory to hold spilled segments.
final File persistDir = new File(tmpSegmentDir, theSink.getSegment().withVersion(version).getIdentifier());
final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getIdentifier());
// Set of spilled segments. Will be merged at the end.
final Set<File> spilled = Sets.newHashSet();
return new Plumber()
{
@Override
public void startJob()
{
}
@Override
public Sink getSink(long timestamp)
{
@ -146,7 +152,6 @@ public class YeOldePlumberSchool implements PlumberSchool
final DataSegment segmentToUpload = theSink.getSegment()
.withDimensions(ImmutableList.copyOf(mappedSegment.getAvailableDimensions()))
.withVersion(version)
.withBinaryVersion(IndexIO.getVersionFromDir(fileToUpload));
dataSegmentPusher.push(fileToUpload, segmentToUpload);

View File

@ -25,9 +25,11 @@ import com.google.common.base.Joiner;
import com.google.common.base.Objects;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.metamx.druid.Query;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.SegmentListUsedAction;
import com.metamx.druid.query.QueryRunner;
import org.joda.time.Interval;
public abstract class AbstractTask implements Task
@ -87,6 +89,12 @@ public abstract class AbstractTask implements Task
return interval;
}
@Override
public <T> QueryRunner<T> getQueryRunner(Query<T> query)
{
return null;
}
@Override
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
{

View File

@ -39,9 +39,9 @@ import com.metamx.druid.merger.common.index.YeOldePlumberSchool;
import com.metamx.druid.realtime.FireDepartmentMetrics;
import com.metamx.druid.realtime.Firehose;
import com.metamx.druid.realtime.FirehoseFactory;
import com.metamx.druid.realtime.Plumber;
import com.metamx.druid.realtime.plumber.Plumber;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.realtime.Sink;
import com.metamx.druid.realtime.plumber.Sink;
import org.joda.time.DateTime;
import org.joda.time.Interval;

View File

@ -27,6 +27,7 @@ import com.google.common.base.Joiner;
import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
@ -49,6 +50,7 @@ import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -185,38 +187,42 @@ public abstract class MergeTaskBase extends AbstractTask
@Override
public TaskStatus preflight(TaskToolbox toolbox)
{
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
{
@Override
public String apply(DataSegment dataSegment)
try {
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
{
return dataSegment.getIdentifier();
@Override
public String apply(DataSegment dataSegment)
{
return dataSegment.getIdentifier();
}
};
final Set<String> current = ImmutableSet.copyOf(
Iterables.transform(toolbox.getTaskActionClient().submit(defaultListUsedAction()), toIdentifier)
);
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
final Set<String> missingFromRequested = Sets.difference(current, requested);
if (!missingFromRequested.isEmpty()) {
throw new ISE(
"Merge is invalid: current segment(s) are not in the requested set: %s",
Joiner.on(", ").join(missingFromRequested)
);
}
};
final Set<String> current = ImmutableSet.copyOf(
Iterables.transform(toolbox.getTaskActionClient().submit(defaultListUsedAction()), toIdentifier)
);
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
final Set<String> missingFromCurrent = Sets.difference(requested, current);
if (!missingFromCurrent.isEmpty()) {
throw new ISE(
"Merge is invalid: requested segment(s) are not in the current set: %s",
Joiner.on(", ").join(missingFromCurrent)
);
}
final Set<String> missingFromRequested = Sets.difference(current, requested);
if (!missingFromRequested.isEmpty()) {
throw new ISE(
"Merge is invalid: current segment(s) are not in the requested set: %s",
Joiner.on(", ").join(missingFromRequested)
);
return TaskStatus.running(getId());
}
final Set<String> missingFromCurrent = Sets.difference(requested, current);
if (!missingFromCurrent.isEmpty()) {
throw new ISE(
"Merge is invalid: requested segment(s) are not in the current set: %s",
Joiner.on(", ").join(missingFromCurrent)
);
catch (IOException e) {
throw Throwables.propagate(e);
}
return TaskStatus.running(getId());
}
protected abstract File merge(Map<DataSegment, File> segments, File outDir)
@ -274,12 +280,12 @@ public abstract class MergeTaskBase extends AbstractTask
DateTime start = null;
DateTime end = null;
for(final DataSegment segment : segments) {
if(start == null || segment.getInterval().getStart().isBefore(start)) {
for (final DataSegment segment : segments) {
if (start == null || segment.getInterval().getStart().isBefore(start)) {
start = segment.getInterval().getStart();
}
if(end == null || segment.getInterval().getEnd().isAfter(end)) {
if (end == null || segment.getInterval().getEnd().isAfter(end)) {
end = segment.getInterval().getEnd();
}
}

View File

@ -0,0 +1,310 @@
package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import com.google.common.io.Closeables;
import com.metamx.common.exception.FormattedException;
import com.metamx.druid.Query;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.index.v1.IndexGranularity;
import com.metamx.druid.input.InputRow;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.LockAcquireAction;
import com.metamx.druid.merger.common.actions.LockListAction;
import com.metamx.druid.merger.common.actions.LockReleaseAction;
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.realtime.FireDepartmentConfig;
import com.metamx.druid.realtime.FireDepartmentMetrics;
import com.metamx.druid.realtime.Firehose;
import com.metamx.druid.realtime.FirehoseFactory;
import com.metamx.druid.realtime.plumber.Plumber;
import com.metamx.druid.realtime.plumber.RealtimePlumberSchool;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.SegmentPublisher;
import com.metamx.druid.realtime.plumber.Sink;
import com.metamx.druid.realtime.plumber.VersioningPolicy;
import com.metamx.emitter.EmittingLogger;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.Period;
import java.io.File;
import java.io.IOException;
public class RealtimeIndexTask extends AbstractTask
{
@JsonIgnore
final Schema schema;
@JsonIgnore
final FirehoseFactory firehoseFactory;
@JsonIgnore
final FireDepartmentConfig fireDepartmentConfig;
@JsonIgnore
final Period windowPeriod;
@JsonIgnore
final IndexGranularity segmentGranularity;
@JsonIgnore
private volatile Plumber plumber = null;
private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class);
@JsonCreator
public RealtimeIndexTask(
@JsonProperty("id") String id,
@JsonProperty("schema") Schema schema,
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
@JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig, // TODO rename?
@JsonProperty("windowPeriod") Period windowPeriod,
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity
)
{
super(
id != null ? id : String.format(
"index_realtime_%s_%d_%s",
schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime()
),
String.format(
"index_realtime_%s",
schema.getDataSource()
),
schema.getDataSource(),
null
);
this.schema = schema;
this.firehoseFactory = firehoseFactory;
this.fireDepartmentConfig = fireDepartmentConfig;
this.windowPeriod = windowPeriod;
this.segmentGranularity = segmentGranularity;
}
@Override
public String getType()
{
return "index_realtime";
}
@Override
public <T> QueryRunner<T> getQueryRunner(Query<T> query)
{
if (plumber != null) {
return plumber.getQueryRunner(query);
} else {
return null;
}
}
@Override
public TaskStatus run(final TaskToolbox toolbox) throws Exception
{
if (this.plumber != null) {
throw new IllegalStateException("WTF?!? run with non-null plumber??!");
}
// Shed any locks we might have (e.g. if we were uncleanly killed and restarted) since we'll reacquire
// them if we actually need them
for (final TaskLock taskLock : toolbox.getTaskActionClient().submit(new LockListAction())) {
toolbox.getTaskActionClient().submit(new LockReleaseAction(taskLock.getInterval()));
}
boolean normalExit = true;
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
final Period intermediatePersistPeriod = fireDepartmentConfig.getIntermediatePersistPeriod();
final Firehose firehose = firehoseFactory.connect();
// TODO -- Take PlumberSchool in constructor (although that will need jackson injectables for stuff like
// TODO -- the ServerView, which seems kind of odd?)
final RealtimePlumberSchool realtimePlumberSchool = new RealtimePlumberSchool(
windowPeriod,
new File(toolbox.getTaskDir(), "persist"),
segmentGranularity
);
final SegmentPublisher segmentPublisher = new TaskActionSegmentPublisher(this, toolbox);
// TODO -- We're adding stuff to talk to the coordinator in various places in the plumber, and may
// TODO -- want to be more robust to coordinator downtime (currently we'll block/throw in whatever
// TODO -- thread triggered the coordinator behavior, which will typically be either the main
// TODO -- data processing loop or the persist thread)
// Wrap default SegmentAnnouncer such that we unlock intervals as we unannounce segments
final SegmentAnnouncer lockingSegmentAnnouncer = new SegmentAnnouncer()
{
@Override
public void announceSegment(final DataSegment segment) throws IOException
{
// NOTE: Side effect: Calling announceSegment causes a lock to be acquired
toolbox.getTaskActionClient().submit(new LockAcquireAction(segment.getInterval()));
toolbox.getSegmentAnnouncer().announceSegment(segment);
}
@Override
public void unannounceSegment(final DataSegment segment) throws IOException
{
try {
toolbox.getSegmentAnnouncer().unannounceSegment(segment);
} finally {
toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval()));
}
}
};
// NOTE: getVersion will block if there is lock contention, which will block plumber.getSink
// NOTE: (and thus the firehose)
// Shouldn't usually happen, since we don't expect people to submit tasks that intersect with the
// realtime window, but if they do it can be problematic. If we decide to care, we can use more threads in
// the plumber such that waiting for the coordinator doesn't block data processing.
final VersioningPolicy versioningPolicy = new VersioningPolicy()
{
@Override
public String getVersion(final Interval interval)
{
try {
// NOTE: Side effect: Calling getVersion causes a lock to be acquired
final TaskLock myLock = toolbox.getTaskActionClient()
.submit(new LockAcquireAction(interval));
return myLock.getVersion();
} catch (IOException e) {
throw Throwables.propagate(e);
}
}
};
// NOTE: This pusher selects path based purely on global configuration and the DataSegment, which means
// NOTE: that redundant realtime tasks will upload to the same location. This can cause index.zip and
// NOTE: descriptor.json to mismatch, or it can cause compute nodes to load different instances of the
// NOTE: "same" segment.
realtimePlumberSchool.setDataSegmentPusher(toolbox.getSegmentPusher());
realtimePlumberSchool.setConglomerate(toolbox.getQueryRunnerFactoryConglomerate());
realtimePlumberSchool.setVersioningPolicy(versioningPolicy);
realtimePlumberSchool.setSegmentAnnouncer(lockingSegmentAnnouncer);
realtimePlumberSchool.setSegmentPublisher(segmentPublisher);
realtimePlumberSchool.setServerView(toolbox.getNewSegmentServerView());
realtimePlumberSchool.setServiceEmitter(toolbox.getEmitter());
this.plumber = realtimePlumberSchool.findPlumber(schema, metrics);
try {
plumber.startJob();
long nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
while (firehose.hasMore()) {
final InputRow inputRow;
try {
inputRow = firehose.nextRow();
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
if (sink == null) {
metrics.incrementThrownAway();
log.debug("Throwing away event[%s]", inputRow);
if (System.currentTimeMillis() > nextFlush) {
plumber.persist(firehose.commit());
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
}
continue;
}
if (sink.isEmpty()) {
log.info("Task %s: New sink: %s", getId(), sink);
}
int currCount = sink.add(inputRow);
metrics.incrementProcessed();
if (currCount >= fireDepartmentConfig.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) {
plumber.persist(firehose.commit());
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
}
}
catch (FormattedException e) {
log.warn(e, "unparseable line");
metrics.incrementUnparseable();
}
}
}
catch (Exception e) {
log.makeAlert(e, "Exception aborted realtime processing[%s]", schema.getDataSource())
.emit();
normalExit = false;
throw Throwables.propagate(e);
}
finally {
Closeables.closeQuietly(firehose);
if (normalExit) {
try {
plumber.persist(firehose.commit());
plumber.finishJob();
} catch(Exception e) {
log.makeAlert(e, "Failed to finish realtime task").emit();
}
}
}
return TaskStatus.success(getId());
}
@JsonProperty
public Schema getSchema()
{
return schema;
}
@JsonProperty("firehose")
public FirehoseFactory getFirehoseFactory()
{
return firehoseFactory;
}
@JsonProperty
public FireDepartmentConfig getFireDepartmentConfig()
{
return fireDepartmentConfig;
}
@JsonProperty
public Period getWindowPeriod()
{
return windowPeriod;
}
@JsonProperty
public IndexGranularity getSegmentGranularity()
{
return segmentGranularity;
}
public static class TaskActionSegmentPublisher implements SegmentPublisher
{
final Task task;
final TaskToolbox taskToolbox;
public TaskActionSegmentPublisher(Task task, TaskToolbox taskToolbox)
{
this.task = task;
this.taskToolbox = taskToolbox;
}
@Override
public void publishSegment(DataSegment segment) throws IOException
{
taskToolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment)));
}
}
}

View File

@ -22,8 +22,10 @@ package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.common.base.Optional;
import com.metamx.druid.Query;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.query.QueryRunner;
import org.joda.time.Interval;
/**
@ -51,6 +53,7 @@ import org.joda.time.Interval;
@JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class),
@JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class),
@JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class),
@JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class),
@JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class),
@JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class)
})
@ -83,6 +86,12 @@ public interface Task
*/
public Optional<Interval> getImplicitLockInterval();
/**
* Returns query runners for this task. If this task is not meant to answer queries over its datasource, this method
* should return null.
*/
public <T> QueryRunner<T> getQueryRunner(Query<T> query);
/**
* Execute preflight checks for a task. This typically runs on the coordinator, and will be run while
* holding a lock on our dataSource and implicit lock interval (if any). If this method throws an exception, the

View File

@ -34,6 +34,7 @@ import com.metamx.druid.loading.SegmentLoadingException;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
import com.metamx.druid.merger.common.actions.SegmentListUsedAction;
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
import com.metamx.druid.merger.common.actions.TaskActionClient;
import org.joda.time.DateTime;
@ -77,7 +78,7 @@ public class VersionConverterTask extends AbstractTask
}
@JsonCreator
private VersionConverterTask(
private static VersionConverterTask createFromJson(
@JsonProperty("id") String id,
@JsonProperty("groupId") String groupId,
@JsonProperty("dataSource") String dataSource,
@ -85,12 +86,26 @@ public class VersionConverterTask extends AbstractTask
@JsonProperty("segment") DataSegment segment
)
{
super(
id,
groupId,
dataSource,
interval
);
if (id == null) {
if (segment == null) {
return create(dataSource, interval);
}
else {
return create(segment);
}
}
return new VersionConverterTask(id, groupId, dataSource, interval, segment);
}
private VersionConverterTask(
String id,
String groupId,
String dataSource,
Interval interval,
DataSegment segment
)
{
super(id, groupId, dataSource, interval);
this.segment = segment;
}
@ -224,6 +239,21 @@ public class VersionConverterTask extends AbstractTask
throws SegmentLoadingException, IOException
{
log.info("Converting segment[%s]", segment);
final TaskActionClient actionClient = toolbox.getTaskActionClient();
final List<DataSegment> currentSegments = actionClient.submit(
new SegmentListUsedAction(segment.getDataSource(), segment.getInterval())
);
for (DataSegment currentSegment : currentSegments) {
final String version = currentSegment.getVersion();
final Integer binaryVersion = currentSegment.getBinaryVersion();
if (version.startsWith(segment.getVersion()) && CURR_VERSION_INTEGER.equals(binaryVersion)) {
log.info("Skipping already updated segment[%s].", segment);
return;
}
}
final Map<DataSegment, File> localSegments = toolbox.getSegments(Arrays.asList(segment));
final File location = localSegments.get(segment);
@ -236,8 +266,7 @@ public class VersionConverterTask extends AbstractTask
DataSegment updatedSegment = segment.withVersion(String.format("%s_v%s", segment.getVersion(), outVersion));
updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment);
toolbox.getTaskActionClient()
.submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment)).withAllowOlderVersions(true));
actionClient.submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment)).withAllowOlderVersions(true));
} else {
log.info("Conversion failed.");
}

View File

@ -19,20 +19,33 @@
package com.metamx.druid.merger.coordinator;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.Collections2;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.merger.common.RetryPolicy;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.task.Task;
import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime;
import org.mortbay.thread.ThreadPool;
import javax.annotation.Nullable;
import java.io.File;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ThreadPoolExecutor;
/**
* Runs tasks in a JVM thread using an ExecutorService.
@ -42,6 +55,8 @@ public class LocalTaskRunner implements TaskRunner
private final TaskToolboxFactory toolboxFactory;
private final ExecutorService exec;
private final Set<TaskRunnerWorkItem> runningItems = new ConcurrentSkipListSet<TaskRunnerWorkItem>();
private static final Logger log = new Logger(LocalTaskRunner.class);
public LocalTaskRunner(
@ -64,65 +79,39 @@ public class LocalTaskRunner implements TaskRunner
{
final TaskToolbox toolbox = toolboxFactory.build(task);
exec.submit(
new Runnable()
{
@Override
public void run()
{
final long startTime = System.currentTimeMillis();
TaskStatus status;
try {
log.info("Running task: %s", task.getId());
status = task.run(toolbox);
}
catch (InterruptedException e) {
log.error(e, "Interrupted while running task[%s]", task);
throw Throwables.propagate(e);
}
catch (Exception e) {
log.error(e, "Exception while running task[%s]", task);
status = TaskStatus.failure(task.getId());
}
catch (Throwable t) {
log.error(t, "Uncaught Throwable while running task[%s]", task);
throw Throwables.propagate(t);
}
try {
final File taskDir = toolbox.getTaskDir();
if (taskDir.exists()) {
log.info("Removing task directory: %s", taskDir);
FileUtils.deleteDirectory(taskDir);
}
}
catch (Exception e) {
log.error(e, "Failed to delete task directory: %s", task.getId());
}
try {
callback.notify(status.withDuration(System.currentTimeMillis() - startTime));
} catch(Exception e) {
log.error(e, "Uncaught Exception during callback for task[%s]", task);
throw Throwables.propagate(e);
}
}
}
);
exec.submit(new LocalTaskRunnerRunnable(task, toolbox, callback));
}
@Override
public Collection<TaskRunnerWorkItem> getRunningTasks()
{
return Lists.newArrayList();
return runningItems;
}
@Override
public Collection<TaskRunnerWorkItem> getPendingTasks()
{
if (exec instanceof ThreadPoolExecutor) {
ThreadPoolExecutor tpe = (ThreadPoolExecutor) exec;
return Lists.newArrayList(
FunctionalIterable.create(tpe.getQueue())
.keep(
new Function<Runnable, TaskRunnerWorkItem>()
{
@Override
public TaskRunnerWorkItem apply(Runnable input)
{
if (input instanceof LocalTaskRunnerRunnable) {
return ((LocalTaskRunnerRunnable) input).getTaskRunnerWorkItem();
}
return null;
}
}
)
);
}
return Lists.newArrayList();
}
@ -131,4 +120,76 @@ public class LocalTaskRunner implements TaskRunner
{
return Lists.newArrayList();
}
private static class LocalTaskRunnerRunnable implements Runnable
{
private final Task task;
private final TaskToolbox toolbox;
private final TaskCallback callback;
private final DateTime createdTime;
public LocalTaskRunnerRunnable(Task task, TaskToolbox toolbox, TaskCallback callback)
{
this.task = task;
this.toolbox = toolbox;
this.callback = callback;
this.createdTime = new DateTime();
}
@Override
public void run()
{
final long startTime = System.currentTimeMillis();
TaskStatus status;
try {
log.info("Running task: %s", task.getId());
status = task.run(toolbox);
}
catch (InterruptedException e) {
log.error(e, "Interrupted while running task[%s]", task);
throw Throwables.propagate(e);
}
catch (Exception e) {
log.error(e, "Exception while running task[%s]", task);
status = TaskStatus.failure(task.getId());
}
catch (Throwable t) {
log.error(t, "Uncaught Throwable while running task[%s]", task);
throw Throwables.propagate(t);
}
try {
final File taskDir = toolbox.getTaskDir();
if (taskDir.exists()) {
log.info("Removing task directory: %s", taskDir);
FileUtils.deleteDirectory(taskDir);
}
}
catch (Exception e) {
log.error(e, "Failed to delete task directory: %s", task.getId());
}
try {
callback.notify(status.withDuration(System.currentTimeMillis() - startTime));
} catch(Exception e) {
log.error(e, "Uncaught Exception during callback for task[%s]", task);
throw Throwables.propagate(e);
}
}
public TaskRunnerWorkItem getTaskRunnerWorkItem()
{
return new TaskRunnerWorkItem(
task,
callback,
null,
createdTime
);
}
}
}

View File

@ -22,8 +22,10 @@ package com.metamx.druid.merger.coordinator;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.metamx.common.logger.Logger;
import com.metamx.druid.TimelineObjectHolder;
import com.metamx.druid.VersionedIntervalTimeline;
@ -71,13 +73,11 @@ public class MergerDBCoordinator
public List<DataSegment> getUsedSegmentsForInterval(final String dataSource, final Interval interval)
throws IOException
{
// XXX Could be reading from a cache if we can assume we're the only one editing the DB
final VersionedIntervalTimeline<String, DataSegment> timeline = dbi.withHandle(
new HandleCallback<VersionedIntervalTimeline<String, DataSegment>>()
{
@Override
public VersionedIntervalTimeline<String, DataSegment> withHandle(Handle handle) throws Exception
public VersionedIntervalTimeline<String, DataSegment> withHandle(Handle handle) throws IOException
{
final VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<String, DataSegment>(
Ordering.natural()
@ -129,31 +129,47 @@ public class MergerDBCoordinator
return segments;
}
public void announceHistoricalSegments(final Set<DataSegment> segments) throws Exception
/**
* Attempts to insert a set of segments to the database. Returns the set of segments actually added (segments
* with identifiers already in the database will not be added).
*
* @param segments set of segments to add
* @return set of segments actually added
*/
public Set<DataSegment> announceHistoricalSegments(final Set<DataSegment> segments) throws IOException
{
dbi.inTransaction(
new TransactionCallback<Void>()
return dbi.inTransaction(
new TransactionCallback<Set<DataSegment>>()
{
@Override
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception
public Set<DataSegment> inTransaction(Handle handle, TransactionStatus transactionStatus) throws IOException
{
for(final DataSegment segment : segments) {
announceHistoricalSegment(handle, segment);
final Set<DataSegment> inserted = Sets.newHashSet();
for (final DataSegment segment : segments) {
if (announceHistoricalSegment(handle, segment)) {
inserted.add(segment);
}
}
return null;
return ImmutableSet.copyOf(inserted);
}
}
);
}
private void announceHistoricalSegment(final Handle handle, final DataSegment segment) throws Exception
/**
* Attempts to insert a single segment to the database. If the segment already exists, will do nothing. Meant
* to be called from within a transaction.
*
* @return true if the segment was added, false otherwise
*/
private boolean announceHistoricalSegment(final Handle handle, final DataSegment segment) throws IOException
{
try {
final List<Map<String, Object>> exists = handle.createQuery(
String.format(
"SELECT id FROM %s WHERE id = ':identifier'",
"SELECT id FROM %s WHERE id = :identifier",
dbConnectorConfig.getSegmentTable()
)
).bind(
@ -163,7 +179,7 @@ public class MergerDBCoordinator
if (!exists.isEmpty()) {
log.info("Found [%s] in DB, not updating DB", segment.getIdentifier());
return;
return false;
}
handle.createStatement(
@ -185,19 +201,21 @@ public class MergerDBCoordinator
log.info("Published segment [%s] to DB", segment.getIdentifier());
}
catch (Exception e) {
catch (IOException e) {
log.error(e, "Exception inserting into DB");
throw e;
}
return true;
}
public void deleteSegments(final Set<DataSegment> segments) throws Exception
public void deleteSegments(final Set<DataSegment> segments) throws IOException
{
dbi.inTransaction(
new TransactionCallback<Void>()
{
@Override
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws IOException
{
for(final DataSegment segment : segments) {
deleteSegment(handle, segment);
@ -223,7 +241,7 @@ public class MergerDBCoordinator
new HandleCallback<List<DataSegment>>()
{
@Override
public List<DataSegment> withHandle(Handle handle) throws Exception
public List<DataSegment> withHandle(Handle handle) throws IOException
{
return handle.createQuery(
String.format(

View File

@ -31,6 +31,7 @@ import com.metamx.common.ISE;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.task.Task;
@ -274,20 +275,24 @@ public class RemoteTaskRunner implements TaskRunner
private void retryTask(final TaskRunnerWorkItem taskRunnerWorkItem, final String workerId)
{
final String taskId = taskRunnerWorkItem.getTask().getId();
log.info("Retry scheduled in %s for %s", taskRunnerWorkItem.getRetryPolicy().getRetryDelay(), taskId);
scheduledExec.schedule(
new Runnable()
{
@Override
public void run()
if (!taskRunnerWorkItem.getRetryPolicy().hasExceededRetryThreshold()) {
log.info("Retry scheduled in %s for %s", taskRunnerWorkItem.getRetryPolicy().getRetryDelay(), taskId);
scheduledExec.schedule(
new Runnable()
{
cleanup(workerId, taskId);
addPendingTask(taskRunnerWorkItem);
}
},
taskRunnerWorkItem.getRetryPolicy().getAndIncrementRetryDelay().getMillis(),
TimeUnit.MILLISECONDS
);
@Override
public void run()
{
cleanup(workerId, taskId);
addPendingTask(taskRunnerWorkItem);
}
},
taskRunnerWorkItem.getRetryPolicy().getAndIncrementRetryDelay().getMillis(),
TimeUnit.MILLISECONDS
);
} else {
log.makeAlert("Task exceeded retry threshold").addData("task", taskId).emit();
}
}
/**

View File

@ -45,6 +45,7 @@ import java.util.NavigableMap;
import java.util.NavigableSet;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
/**
@ -58,6 +59,7 @@ public class TaskLockbox
private final Map<String, NavigableMap<Interval, TaskLockPosse>> running = Maps.newHashMap();
private final TaskStorage taskStorage;
private final ReentrantLock giant = new ReentrantLock();
private final Condition lockReleaseCondition = giant.newCondition();
private static final EmittingLogger log = new EmittingLogger(TaskLockbox.class);
@ -66,6 +68,27 @@ public class TaskLockbox
this.taskStorage = taskStorage;
}
/**
* Locks a task without removing it from the queue. Blocks until the lock is acquired. Throws an exception
* if the lock cannot be acquired.
*/
public TaskLock lock(final Task task, final Interval interval) throws InterruptedException
{
giant.lock();
try {
Optional<TaskLock> taskLock;
while (!(taskLock = tryLock(task, interval)).isPresent()) {
lockReleaseCondition.await();
}
return taskLock.get();
} finally {
giant.unlock();
}
}
/**
* Attempt to lock a task, without removing it from the queue. Equivalent to the long form of {@code tryLock}
* with no preferred version.
@ -241,6 +264,9 @@ public class TaskLockbox
running.remove(dataSource);
}
// Wake up blocking-lock waiters
lockReleaseCondition.signalAll();
// Best effort to remove lock from storage
try {
taskStorage.removeLock(task.getId(), taskLock);

View File

@ -120,17 +120,17 @@ public class TaskQueue
}
// Sort locks by version
final Ordering<TaskLock> byVersionOrdering = new Ordering<TaskLock>()
final Ordering<Map.Entry<TaskLock, Task>> byVersionOrdering = new Ordering<Map.Entry<TaskLock, Task>>()
{
@Override
public int compare(TaskLock left, TaskLock right)
public int compare(Map.Entry<TaskLock, Task> left, Map.Entry<TaskLock, Task> right)
{
return left.getVersion().compareTo(right.getVersion());
return left.getKey().getVersion().compareTo(right.getKey().getVersion());
}
};
// Acquire as many locks as possible, in version order
for(final Map.Entry<TaskLock, Task> taskAndLock : tasksByLock.entries()) {
for(final Map.Entry<TaskLock, Task> taskAndLock : byVersionOrdering.sortedCopy(tasksByLock.entries())) {
final Task task = taskAndLock.getValue();
final TaskLock savedTaskLock = taskAndLock.getKey();

View File

@ -20,6 +20,7 @@
package com.metamx.druid.merger.coordinator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.druid.merger.common.RetryPolicy;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.task.Task;
import org.joda.time.DateTime;

View File

@ -23,8 +23,10 @@ import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.services.ec2.AmazonEC2Client;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.inject.Guice;
@ -38,14 +40,17 @@ import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.RegisteringNode;
import com.metamx.druid.BaseServerNode;
import com.metamx.druid.client.ClientConfig;
import com.metamx.druid.client.ClientInventoryManager;
import com.metamx.druid.client.MutableServerView;
import com.metamx.druid.client.OnlyNewSegmentWatcherServerView;
import com.metamx.druid.config.ConfigManager;
import com.metamx.druid.config.ConfigManagerConfig;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.db.DbConnector;
import com.metamx.druid.db.DbConnectorConfig;
import com.metamx.druid.http.GuiceServletConfig;
import com.metamx.druid.http.MasterMain;
import com.metamx.druid.http.RedirectFilter;
import com.metamx.druid.http.RedirectInfo;
import com.metamx.druid.http.StatusServlet;
@ -57,10 +62,12 @@ import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentKiller;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory;
import com.metamx.druid.merger.common.actions.TaskActionToolbox;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
import com.metamx.druid.merger.coordinator.DbTaskStorage;
@ -68,7 +75,6 @@ import com.metamx.druid.merger.coordinator.HeapMemoryTaskStorage;
import com.metamx.druid.merger.coordinator.LocalTaskRunner;
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
import com.metamx.druid.merger.coordinator.RemoteTaskRunner;
import com.metamx.druid.merger.coordinator.RetryPolicyFactory;
import com.metamx.druid.merger.coordinator.TaskLockbox;
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
import com.metamx.druid.merger.coordinator.TaskQueue;
@ -80,7 +86,6 @@ import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy;
import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy;
import com.metamx.druid.merger.coordinator.scaling.NoopAutoScalingStrategy;
@ -90,6 +95,9 @@ import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFa
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy;
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncerConfig;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.Emitters;
@ -118,7 +126,6 @@ import org.skife.config.ConfigurationObjectFactory;
import org.skife.jdbi.v2.DBI;
import java.net.URL;
import java.util.Arrays;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ExecutorService;
@ -128,7 +135,7 @@ import java.util.concurrent.atomic.AtomicReference;
/**
*/
public class IndexerCoordinatorNode extends RegisteringNode
public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNode>
{
private static final Logger log = new Logger(IndexerCoordinatorNode.class);
@ -137,7 +144,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
return new Builder();
}
private final ObjectMapper jsonMapper;
private final Lifecycle lifecycle;
private final Properties props;
private final ConfigurationObjectFactory configFactory;
@ -161,20 +167,21 @@ public class IndexerCoordinatorNode extends RegisteringNode
private TaskRunnerFactory taskRunnerFactory = null;
private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null;
private TaskMasterLifecycle taskMasterLifecycle = null;
private MutableServerView newSegmentServerView = null;
private Server server = null;
private boolean initialized = false;
public IndexerCoordinatorNode(
ObjectMapper jsonMapper,
Lifecycle lifecycle,
Properties props,
Lifecycle lifecycle,
ObjectMapper jsonMapper,
ObjectMapper smileMapper,
ConfigurationObjectFactory configFactory
)
{
super(Arrays.asList(jsonMapper));
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
this.jsonMapper = jsonMapper;
this.lifecycle = lifecycle;
this.props = props;
this.configFactory = configFactory;
@ -198,6 +205,12 @@ public class IndexerCoordinatorNode extends RegisteringNode
return this;
}
public IndexerCoordinatorNode setNewSegmentServerView(MutableServerView newSegmentServerView)
{
this.newSegmentServerView = newSegmentServerView;
return this;
}
public IndexerCoordinatorNode setS3Service(RestS3Service s3Service)
{
this.s3Service = s3Service;
@ -240,7 +253,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
return this;
}
public void init() throws Exception
public void doInit() throws Exception
{
scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
initializeDB();
@ -254,7 +267,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
dbi,
managerConfig
)
), jsonMapper
), getJsonMapper()
);
initializeEmitter();
@ -263,6 +276,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
initializeTaskConfig();
initializeS3Service();
initializeMergeDBCoordinator();
initializeNewSegmentServerView();
initializeTaskStorage();
initializeTaskLockbox();
initializeTaskQueue();
@ -288,7 +302,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
final Injector injector = Guice.createInjector(
new IndexerCoordinatorServletModule(
jsonMapper,
getJsonMapper(),
config,
emitter,
taskMasterLifecycle,
@ -306,6 +320,9 @@ public class IndexerCoordinatorNode extends RegisteringNode
});
staticContext.setBaseResource(resourceCollection);
// TODO -- Need a QueryServlet and some kind of QuerySegmentWalker if we want to support querying tasks
// TODO -- (e.g. for realtime) in local mode
final Context root = new Context(server, "/", Context.SESSIONS);
root.addServlet(new ServletHolder(new StatusServlet()), "/status");
root.addServlet(new ServletHolder(new DefaultServlet()), "/mmx/*");
@ -419,12 +436,12 @@ public class IndexerCoordinatorNode extends RegisteringNode
injectables.addValue("s3Client", s3Service)
.addValue("segmentPusher", segmentPusher);
jsonMapper.setInjectableValues(injectables);
getJsonMapper().setInjectableValues(injectables);
}
private void initializeJacksonSubtypes()
{
jsonMapper.registerSubtypes(StaticS3FirehoseFactory.class);
getJsonMapper().registerSubtypes(StaticS3FirehoseFactory.class);
}
private void initializeEmitter()
@ -437,7 +454,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
emitter = new ServiceEmitter(
PropUtils.getProperty(props, "druid.service"),
PropUtils.getProperty(props, "druid.host"),
Emitters.create(props, httpClient, jsonMapper, lifecycle)
Emitters.create(props, httpClient, getJsonMapper(), lifecycle)
);
}
EmittingLogger.registerEmitter(emitter);
@ -476,6 +493,21 @@ public class IndexerCoordinatorNode extends RegisteringNode
}
}
private void initializeNewSegmentServerView()
{
if (newSegmentServerView == null) {
final MutableServerView view = new OnlyNewSegmentWatcherServerView();
final ClientInventoryManager clientInventoryManager = new ClientInventoryManager(
getConfigFactory().build(ClientConfig.class),
getPhoneBook(),
view
);
lifecycle.addManagedInstance(clientInventoryManager);
this.newSegmentServerView = view;
}
}
public void initializeS3Service() throws S3ServiceException
{
this.s3Service = new RestS3Service(
@ -489,13 +521,17 @@ public class IndexerCoordinatorNode extends RegisteringNode
public void initializeDataSegmentPusher()
{
if (segmentPusher == null) {
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper);
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, getJsonMapper());
}
}
public void initializeTaskToolbox()
{
if (taskToolboxFactory == null) {
final SegmentAnnouncer segmentAnnouncer = new ZkSegmentAnnouncer(
configFactory.build(ZkSegmentAnnouncerConfig.class),
getPhoneBook()
);
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
taskToolboxFactory = new TaskToolboxFactory(
taskConfig,
@ -507,7 +543,10 @@ public class IndexerCoordinatorNode extends RegisteringNode
s3Service,
segmentPusher,
dataSegmentKiller,
jsonMapper
segmentAnnouncer,
newSegmentServerView,
getConglomerate(),
getJsonMapper()
);
}
}
@ -516,7 +555,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
{
if (mergerDBCoordinator == null) {
mergerDBCoordinator = new MergerDBCoordinator(
jsonMapper,
getJsonMapper(),
dbConnectorConfig,
dbi
);
@ -563,7 +602,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
taskStorage = new HeapMemoryTaskStorage();
} else if (config.getStorageImpl().equals("db")) {
final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class);
taskStorage = new DbTaskStorage(jsonMapper, dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI());
taskStorage = new DbTaskStorage(getJsonMapper(), dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI());
} else {
throw new ISE("Invalid storage implementation: %s", config.getStorageImpl());
}
@ -590,12 +629,17 @@ public class IndexerCoordinatorNode extends RegisteringNode
);
RemoteTaskRunner remoteTaskRunner = new RemoteTaskRunner(
jsonMapper,
getJsonMapper(),
configFactory.build(RemoteTaskRunnerConfig.class),
curatorFramework,
new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true),
retryScheduledExec,
new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)),
new RetryPolicyFactory(
configFactory.buildWithReplacements(
RetryPolicyConfig.class,
ImmutableMap.of("base_path", "druid.indexing")
)
),
configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class)
);
@ -641,7 +685,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
AutoScalingStrategy strategy;
if (config.getStrategyImpl().equalsIgnoreCase("ec2")) {
strategy = new EC2AutoScalingStrategy(
jsonMapper,
getJsonMapper(),
new AmazonEC2Client(
new BasicAWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
@ -675,6 +719,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
public static class Builder
{
private ObjectMapper jsonMapper = null;
private ObjectMapper smileMapper = null;
private Lifecycle lifecycle = null;
private Properties props = null;
private ConfigurationObjectFactory configFactory = null;
@ -705,8 +750,13 @@ public class IndexerCoordinatorNode extends RegisteringNode
public IndexerCoordinatorNode build()
{
if (jsonMapper == null) {
if (jsonMapper == null && smileMapper == null) {
jsonMapper = new DefaultObjectMapper();
smileMapper = new DefaultObjectMapper(new SmileFactory());
smileMapper.getJsonFactory().setCodec(smileMapper);
}
else if (jsonMapper == null || smileMapper == null) {
throw new ISE("Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", jsonMapper, smileMapper);
}
if (lifecycle == null) {
@ -721,7 +771,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
configFactory = Config.createFactory(props);
}
return new IndexerCoordinatorNode(jsonMapper, lifecycle, props, configFactory);
return new IndexerCoordinatorNode(props, lifecycle, jsonMapper, smileMapper, configFactory);
}
}
}

View File

@ -44,6 +44,7 @@ import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.core.Response;
import java.io.IOException;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
@ -187,12 +188,17 @@ public class IndexerCoordinatorResource
@Produces("application/json")
public <T> Response doAction(final TaskActionHolder<T> holder)
{
final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask())
.getTaskActionClient()
.submit(holder.getAction());
final Map<String, Object> retMap;
final Map<String, Object> retMap = Maps.newHashMap();
retMap.put("result", ret);
try {
final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask())
.getTaskActionClient()
.submit(holder.getAction());
retMap = Maps.newHashMap();
retMap.put("result", ret);
} catch(IOException e) {
return Response.serverError().build();
}
return Response.ok().entity(retMap).build();
}

View File

@ -21,35 +21,47 @@ package com.metamx.druid.merger.worker;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.Query;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.query.NoopQueryRunner;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.druid.query.segment.SegmentDescriptor;
import com.metamx.emitter.EmittingLogger;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent;
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
import org.apache.commons.io.FileUtils;
import org.joda.time.Interval;
import java.io.File;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutorService;
/**
* The monitor watches ZK at a specified path for new tasks to appear. Upon starting the monitor, a listener will be
* created that waits for new tasks. Tasks are executed as soon as they are seen.
*
* The monitor implements {@link QuerySegmentWalker} so tasks can offer up queryable data. This is useful for
* realtime index tasks.
*/
public class TaskMonitor
public class WorkerTaskMonitor implements QuerySegmentWalker
{
private static final EmittingLogger log = new EmittingLogger(TaskMonitor.class);
private static final EmittingLogger log = new EmittingLogger(WorkerTaskMonitor.class);
private final PathChildrenCache pathChildrenCache;
private final CuratorFramework cf;
private final WorkerCuratorCoordinator workerCuratorCoordinator;
private final TaskToolboxFactory toolboxFactory;
private final ExecutorService exec;
private final List<Task> running = new CopyOnWriteArrayList<Task>();
public TaskMonitor(
public WorkerTaskMonitor(
PathChildrenCache pathChildrenCache,
CuratorFramework cf,
WorkerCuratorCoordinator workerCuratorCoordinator,
@ -88,7 +100,7 @@ public class TaskMonitor
);
final TaskToolbox toolbox = toolboxFactory.build(task);
if (workerCuratorCoordinator.statusExists(task.getId())) {
if (isTaskRunning(task)) {
log.warn("Got task %s that I am already running...", task.getId());
workerCuratorCoordinator.unannounceTask(task.getId());
return;
@ -104,6 +116,7 @@ public class TaskMonitor
final File taskDir = toolbox.getTaskDir();
log.info("Running task [%s]", task.getId());
running.add(task);
TaskStatus taskStatus;
try {
@ -116,6 +129,8 @@ public class TaskMonitor
.addData("task", task.getId())
.emit();
taskStatus = TaskStatus.failure(task.getId());
} finally {
running.remove(task);
}
taskStatus = taskStatus.withDuration(System.currentTimeMillis() - startTime);
@ -151,12 +166,23 @@ public class TaskMonitor
);
}
catch (Exception e) {
log.makeAlert(e, "Exception starting TaskMonitor")
log.makeAlert(e, "Exception starting WorkerTaskMonitor")
.addData("exception", e.toString())
.emit();
}
}
private boolean isTaskRunning(final Task task)
{
for (final Task runningTask : running) {
if (runningTask.equals(task.getId())) {
return true;
}
}
return false;
}
@LifecycleStop
public void stop()
{
@ -165,9 +191,43 @@ public class TaskMonitor
exec.shutdown();
}
catch (Exception e) {
log.makeAlert(e, "Exception stopping TaskMonitor")
log.makeAlert(e, "Exception stopping WorkerTaskMonitor")
.addData("exception", e.toString())
.emit();
}
}
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<Interval> intervals)
{
return getQueryRunnerImpl(query);
}
@Override
public <T> QueryRunner<T> getQueryRunnerForSegments(Query<T> query, Iterable<SegmentDescriptor> specs)
{
return getQueryRunnerImpl(query);
}
private <T> QueryRunner<T> getQueryRunnerImpl(Query<T> query) {
QueryRunner<T> queryRunner = null;
for (final Task task : running) {
if (task.getDataSource().equals(query.getDataSource())) {
final QueryRunner<T> taskQueryRunner = task.getQueryRunner(query);
if (taskQueryRunner != null) {
if (queryRunner == null) {
queryRunner = taskQueryRunner;
} else {
log.makeAlert("Found too many query runners for datasource")
.addData("dataSource", query.getDataSource())
.emit();
}
}
}
}
return queryRunner == null ? new NoopQueryRunner<T>() : queryRunner;
}
}

View File

@ -21,16 +21,23 @@ package com.metamx.druid.merger.worker.http;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.inject.servlet.GuiceFilter;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutorFactory;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.config.Config;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.RegisteringNode;
import com.metamx.druid.BaseServerNode;
import com.metamx.druid.client.ClientConfig;
import com.metamx.druid.client.ClientInventoryManager;
import com.metamx.druid.client.MutableServerView;
import com.metamx.druid.client.OnlyNewSegmentWatcherServerView;
import com.metamx.druid.http.QueryServlet;
import com.metamx.druid.http.StatusServlet;
import com.metamx.druid.initialization.CuratorConfig;
import com.metamx.druid.initialization.Initialization;
@ -38,18 +45,23 @@ import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServerInit;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
import com.metamx.druid.merger.worker.TaskMonitor;
import com.metamx.druid.merger.worker.Worker;
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
import com.metamx.druid.merger.worker.WorkerTaskMonitor;
import com.metamx.druid.merger.worker.config.WorkerConfig;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncerConfig;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.Emitters;
@ -76,7 +88,6 @@ import org.mortbay.jetty.servlet.ServletHolder;
import org.skife.config.ConfigurationObjectFactory;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ExecutorService;
@ -85,16 +96,15 @@ import java.util.concurrent.ScheduledExecutorService;
/**
*/
public class WorkerNode extends RegisteringNode
public class WorkerNode extends BaseServerNode<WorkerNode>
{
private static final Logger log = new Logger(WorkerNode.class);
private static final EmittingLogger log = new EmittingLogger(WorkerNode.class);
public static Builder builder()
{
return new Builder();
}
private final ObjectMapper jsonMapper;
private final Lifecycle lifecycle;
private final Properties props;
private final ConfigurationObjectFactory configFactory;
@ -111,21 +121,22 @@ public class WorkerNode extends RegisteringNode
private ServiceDiscovery serviceDiscovery = null;
private ServiceProvider coordinatorServiceProvider = null;
private WorkerCuratorCoordinator workerCuratorCoordinator = null;
private TaskMonitor taskMonitor = null;
private WorkerTaskMonitor workerTaskMonitor = null;
private MutableServerView newSegmentServerView = null;
private Server server = null;
private boolean initialized = false;
public WorkerNode(
ObjectMapper jsonMapper,
Lifecycle lifecycle,
Properties props,
Lifecycle lifecycle,
ObjectMapper jsonMapper,
ObjectMapper smileMapper,
ConfigurationObjectFactory configFactory
)
{
super(Arrays.asList(jsonMapper));
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
this.jsonMapper = jsonMapper;
this.lifecycle = lifecycle;
this.props = props;
this.configFactory = configFactory;
@ -185,13 +196,20 @@ public class WorkerNode extends RegisteringNode
return this;
}
public WorkerNode setTaskMonitor(TaskMonitor taskMonitor)
public WorkerNode setNewSegmentServerView(MutableServerView newSegmentServerView)
{
this.taskMonitor = taskMonitor;
this.newSegmentServerView = newSegmentServerView;
return this;
}
public void init() throws Exception
public WorkerNode setWorkerTaskMonitor(WorkerTaskMonitor workerTaskMonitor)
{
this.workerTaskMonitor = workerTaskMonitor;
return this;
}
@Override
public void doInit() throws Exception
{
initializeHttpClient();
initializeEmitter();
@ -201,12 +219,13 @@ public class WorkerNode extends RegisteringNode
initializeCuratorFramework();
initializeServiceDiscovery();
initializeCoordinatorServiceProvider();
initializeNewSegmentServerView();
initializeDataSegmentPusher();
initializeTaskToolbox();
initializeJacksonInjections();
initializeJacksonSubtypes();
initializeCuratorCoordinator();
initializeTaskMonitor();
initializeWorkerTaskMonitor();
initializeServer();
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
@ -223,6 +242,12 @@ public class WorkerNode extends RegisteringNode
root.addServlet(new ServletHolder(new StatusServlet()), "/status");
root.addServlet(new ServletHolder(new DefaultServlet()), "/mmx/*");
root.addServlet(
new ServletHolder(
new QueryServlet(getJsonMapper(), getSmileMapper(), workerTaskMonitor, emitter, getRequestLogger())
),
"/druid/v2/*"
);
root.addFilter(GuiceFilter.class, "/mmx/indexer/worker/v1/*", 0);
}
@ -280,12 +305,12 @@ public class WorkerNode extends RegisteringNode
injectables.addValue("s3Client", s3Service)
.addValue("segmentPusher", segmentPusher);
jsonMapper.setInjectableValues(injectables);
getJsonMapper().setInjectableValues(injectables);
}
private void initializeJacksonSubtypes()
{
jsonMapper.registerSubtypes(StaticS3FirehoseFactory.class);
getJsonMapper().registerSubtypes(StaticS3FirehoseFactory.class);
}
private void initializeHttpClient()
@ -303,7 +328,7 @@ public class WorkerNode extends RegisteringNode
emitter = new ServiceEmitter(
PropUtils.getProperty(props, "druid.service"),
PropUtils.getProperty(props, "druid.host"),
Emitters.create(props, httpClient, jsonMapper, lifecycle)
Emitters.create(props, httpClient, getJsonMapper(), lifecycle)
);
}
EmittingLogger.registerEmitter(emitter);
@ -344,7 +369,7 @@ public class WorkerNode extends RegisteringNode
public void initializeDataSegmentPusher()
{
if (segmentPusher == null) {
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper);
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, getJsonMapper());
}
}
@ -352,14 +377,32 @@ public class WorkerNode extends RegisteringNode
{
if (taskToolboxFactory == null) {
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
final SegmentAnnouncer segmentAnnouncer = new ZkSegmentAnnouncer(
configFactory.build(ZkSegmentAnnouncerConfig.class),
getPhoneBook()
);
lifecycle.addManagedInstance(segmentAnnouncer);
taskToolboxFactory = new TaskToolboxFactory(
taskConfig,
new RemoteTaskActionClientFactory(httpClient, coordinatorServiceProvider, jsonMapper),
new RemoteTaskActionClientFactory(
httpClient,
coordinatorServiceProvider,
new RetryPolicyFactory(
configFactory.buildWithReplacements(
RetryPolicyConfig.class,
ImmutableMap.of("base_path", "druid.worker.taskActionClient")
)
),
getJsonMapper()
),
emitter,
s3Service,
segmentPusher,
dataSegmentKiller,
jsonMapper
segmentAnnouncer,
newSegmentServerView,
getConglomerate(),
getJsonMapper()
);
}
}
@ -402,7 +445,7 @@ public class WorkerNode extends RegisteringNode
{
if (workerCuratorCoordinator == null) {
workerCuratorCoordinator = new WorkerCuratorCoordinator(
jsonMapper,
getJsonMapper(),
configFactory.build(IndexerZkConfig.class),
curatorFramework,
new Worker(workerConfig)
@ -411,29 +454,45 @@ public class WorkerNode extends RegisteringNode
}
}
public void initializeTaskMonitor()
private void initializeNewSegmentServerView()
{
if (taskMonitor == null) {
if (newSegmentServerView == null) {
final MutableServerView view = new OnlyNewSegmentWatcherServerView();
final ClientInventoryManager clientInventoryManager = new ClientInventoryManager(
getConfigFactory().build(ClientConfig.class),
getPhoneBook(),
view
);
lifecycle.addManagedInstance(clientInventoryManager);
this.newSegmentServerView = view;
}
}
public void initializeWorkerTaskMonitor()
{
if (workerTaskMonitor == null) {
final ExecutorService workerExec = Executors.newFixedThreadPool(workerConfig.getNumThreads());
final PathChildrenCache pathChildrenCache = new PathChildrenCache(
curatorFramework,
workerCuratorCoordinator.getTaskPathForWorker(),
false
);
taskMonitor = new TaskMonitor(
workerTaskMonitor = new WorkerTaskMonitor(
pathChildrenCache,
curatorFramework,
workerCuratorCoordinator,
taskToolboxFactory,
workerExec
);
lifecycle.addManagedInstance(taskMonitor);
lifecycle.addManagedInstance(workerTaskMonitor);
}
}
public static class Builder
{
private ObjectMapper jsonMapper = null;
private ObjectMapper smileMapper = null;
private Lifecycle lifecycle = null;
private Properties props = null;
private ConfigurationObjectFactory configFactory = null;
@ -464,8 +523,13 @@ public class WorkerNode extends RegisteringNode
public WorkerNode build()
{
if (jsonMapper == null) {
if (jsonMapper == null && smileMapper == null) {
jsonMapper = new DefaultObjectMapper();
smileMapper = new DefaultObjectMapper(new SmileFactory());
smileMapper.getJsonFactory().setCodec(smileMapper);
}
else if (jsonMapper == null || smileMapper == null) {
throw new ISE("Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", jsonMapper, smileMapper);
}
if (lifecycle == null) {
@ -480,7 +544,7 @@ public class WorkerNode extends RegisteringNode
configFactory = Config.createFactory(props);
}
return new WorkerNode(jsonMapper, lifecycle, props, configFactory);
return new WorkerNode(props, lifecycle, jsonMapper, smileMapper, configFactory);
}
}
}

View File

@ -8,17 +8,20 @@ import com.metamx.druid.aggregation.AggregatorFactory;
import com.metamx.druid.aggregation.CountAggregatorFactory;
import com.metamx.druid.aggregation.DoubleSumAggregatorFactory;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.index.v1.IndexGranularity;
import com.metamx.druid.indexer.HadoopDruidIndexerConfig;
import com.metamx.druid.indexer.data.JSONDataSpec;
import com.metamx.druid.indexer.granularity.UniformGranularitySpec;
import com.metamx.druid.indexer.path.StaticPathSpec;
import com.metamx.druid.indexer.rollup.DataRollupSpec;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.shard.NoneShardSpec;
import junit.framework.Assert;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.joda.time.Interval;
import org.joda.time.Period;
import org.junit.Test;
public class TaskSerdeTest
@ -193,6 +196,40 @@ public class TaskSerdeTest
);
}
@Test
public void testRealtimeIndexTaskSerde() throws Exception
{
final Task task = new RealtimeIndexTask(
null,
new Schema("foo", new AggregatorFactory[0], QueryGranularity.NONE, new NoneShardSpec()),
null,
null,
new Period("PT10M"),
IndexGranularity.HOUR
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final Task task2 = jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.<Interval>absent(), task.getImplicitLockInterval());
Assert.assertEquals(new Period("PT10M"), ((RealtimeIndexTask) task).getWindowPeriod());
Assert.assertEquals(IndexGranularity.HOUR, ((RealtimeIndexTask) task).getSegmentGranularity());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
Assert.assertEquals(((RealtimeIndexTask) task).getWindowPeriod(), ((RealtimeIndexTask) task).getWindowPeriod());
Assert.assertEquals(
((RealtimeIndexTask) task).getSegmentGranularity(),
((RealtimeIndexTask) task).getSegmentGranularity()
);
}
@Test
public void testDeleteTaskSerde() throws Exception
{

View File

@ -9,17 +9,18 @@ import com.metamx.druid.aggregation.AggregatorFactory;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.merger.TestTask;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.worker.TaskMonitor;
import com.metamx.druid.merger.worker.Worker;
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
import com.metamx.druid.merger.worker.WorkerTaskMonitor;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.netflix.curator.framework.CuratorFramework;
@ -59,7 +60,7 @@ public class RemoteTaskRunnerTest
private CuratorFramework cf;
private PathChildrenCache pathChildrenCache;
private RemoteTaskRunner remoteTaskRunner;
private TaskMonitor taskMonitor;
private WorkerTaskMonitor workerTaskMonitor;
private ScheduledExecutorService scheduledExec;
@ -123,7 +124,7 @@ public class RemoteTaskRunnerTest
{
testingCluster.stop();
remoteTaskRunner.stop();
taskMonitor.stop();
workerTaskMonitor.stop();
}
@Test
@ -275,7 +276,7 @@ public class RemoteTaskRunnerTest
);
workerCuratorCoordinator.start();
taskMonitor = new TaskMonitor(
workerTaskMonitor = new WorkerTaskMonitor(
new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true),
cf,
workerCuratorCoordinator,
@ -304,12 +305,12 @@ public class RemoteTaskRunnerTest
{
return null;
}
}, null, null, null, null, null, jsonMapper
}, null, null, null, null, null, null, null, null, jsonMapper
),
Executors.newSingleThreadExecutor()
);
jsonMapper.registerSubtypes(new NamedType(TestTask.class, "test"));
taskMonitor.start();
workerTaskMonitor.start();
}
private void makeRemoteTaskRunner() throws Exception

View File

@ -1,6 +1,7 @@
package com.metamx.druid.merger.coordinator;
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
import com.metamx.druid.merger.common.RetryPolicy;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
import junit.framework.Assert;
import org.joda.time.Duration;
import org.junit.Test;

View File

@ -19,7 +19,6 @@
package com.metamx.druid.merger.coordinator;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
@ -153,6 +152,9 @@ public class TaskLifecycleTest
}
},
null, // segment announcer
null, // new segment server view
null, // query runner factory conglomerate corporation unionized collective
new DefaultObjectMapper()
);
@ -284,22 +286,20 @@ public class TaskLifecycleTest
// Sort of similar to what realtime tasks do:
// Acquire lock for first interval
final Optional<TaskLock> lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval1));
final TaskLock lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval1));
final List<TaskLock> locks1 = toolbox.getTaskActionClient().submit(new LockListAction());
// (Confirm lock sanity)
Assert.assertTrue("lock1 present", lock1.isPresent());
Assert.assertEquals("lock1 interval", interval1, lock1.get().getInterval());
Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1);
Assert.assertEquals("lock1 interval", interval1, lock1.getInterval());
Assert.assertEquals("locks1", ImmutableList.of(lock1), locks1);
// Acquire lock for second interval
final Optional<TaskLock> lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval2));
final TaskLock lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval2));
final List<TaskLock> locks2 = toolbox.getTaskActionClient().submit(new LockListAction());
// (Confirm lock sanity)
Assert.assertTrue("lock2 present", lock2.isPresent());
Assert.assertEquals("lock2 interval", interval2, lock2.get().getInterval());
Assert.assertEquals("locks2", ImmutableList.of(lock1.get(), lock2.get()), locks2);
Assert.assertEquals("lock2 interval", interval2, lock2.getInterval());
Assert.assertEquals("locks2", ImmutableList.of(lock1, lock2), locks2);
// Push first segment
toolbox.getTaskActionClient()
@ -309,7 +309,7 @@ public class TaskLifecycleTest
DataSegment.builder()
.dataSource("foo")
.interval(interval1)
.version(lock1.get().getVersion())
.version(lock1.getVersion())
.build()
)
)
@ -320,7 +320,7 @@ public class TaskLifecycleTest
final List<TaskLock> locks3 = toolbox.getTaskActionClient().submit(new LockListAction());
// (Confirm lock sanity)
Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3);
Assert.assertEquals("locks3", ImmutableList.of(lock2), locks3);
// Push second segment
toolbox.getTaskActionClient()
@ -330,7 +330,7 @@ public class TaskLifecycleTest
DataSegment.builder()
.dataSource("foo")
.interval(interval2)
.version(lock2.get().getVersion())
.version(lock2.getVersion())
.build()
)
)
@ -394,7 +394,7 @@ public class TaskLifecycleTest
}
@Test
public void testBadVersion() throws Exception
public void testBadInterval() throws Exception
{
final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
{
@ -428,7 +428,7 @@ public class TaskLifecycleTest
}
@Test
public void testBadInterval() throws Exception
public void testBadVersion() throws Exception
{
final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
{
@ -508,15 +508,22 @@ public class TaskLifecycleTest
}
@Override
public void announceHistoricalSegments(Set<DataSegment> segment)
public Set<DataSegment> announceHistoricalSegments(Set<DataSegment> segments)
{
published.addAll(segment);
Set<DataSegment> added = Sets.newHashSet();
for(final DataSegment segment : segments) {
if(published.add(segment)) {
added.add(segment);
}
}
return ImmutableSet.copyOf(added);
}
@Override
public void deleteSegments(Set<DataSegment> segment)
public void deleteSegments(Set<DataSegment> segments)
{
nuked.addAll(segment);
nuked.addAll(segments);
}
public Set<DataSegment> getPublished()

View File

@ -165,6 +165,9 @@ public class TaskQueueTest
null,
null,
null,
null,
null,
null,
null
);
@ -222,6 +225,9 @@ public class TaskQueueTest
null,
null,
null,
null,
null,
null,
null
);

View File

@ -23,7 +23,7 @@
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<packaging>pom</packaging>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
<name>druid</name>
<description>druid</description>
<scm>
@ -102,6 +102,11 @@
<artifactId>commons-logging</artifactId>
<version>1.1.1</version>
</dependency>
<dependency>
<groupId>commons-lang</groupId>
<artifactId>commons-lang</artifactId>
<version>2.6</version>
</dependency>
<dependency>
<groupId>com.ning</groupId>
<artifactId>compress-lzf</artifactId>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
</parent>
<properties>
@ -182,41 +182,4 @@
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.scala-tools</groupId>
<artifactId>maven-scala-plugin</artifactId>
<configuration>
<args>
<arg>-unchecked</arg>
<arg>-deprecation</arg>
</args>
</configuration>
<executions>
<execution>
<id>compile</id>
<goals>
<goal>compile</goal>
</goals>
<phase>compile</phase>
</execution>
<execution>
<id>test-compile</id>
<goals>
<goal>testCompile</goal>
</goals>
<phase>test-compile</phase>
</execution>
<execution>
<goals>
<goal>compile</goal>
</goals>
<phase>process-resources</phase>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,91 @@
package com.metamx.druid.realtime;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import org.joda.time.DateTime;
import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.tweak.HandleCallback;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class DbSegmentPublisher implements SegmentPublisher
{
private static final Logger log = new Logger(DbSegmentPublisher.class);
private final ObjectMapper jsonMapper;
private final DbSegmentPublisherConfig config;
private final DBI dbi;
public DbSegmentPublisher(
ObjectMapper jsonMapper,
DbSegmentPublisherConfig config,
DBI dbi
)
{
this.jsonMapper = jsonMapper;
this.config = config;
this.dbi = dbi;
}
public void publishSegment(final DataSegment segment) throws IOException
{
try {
List<Map<String, Object>> exists = dbi.withHandle(
new HandleCallback<List<Map<String, Object>>>()
{
@Override
public List<Map<String, Object>> withHandle(Handle handle) throws Exception
{
return handle.createQuery(
String.format("SELECT id FROM %s WHERE id=:id", config.getSegmentTable())
)
.bind("id", segment.getIdentifier())
.list();
}
}
);
if (!exists.isEmpty()) {
log.info("Found [%s] in DB, not updating DB", segment.getIdentifier());
return;
}
dbi.withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(
String.format(
"INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
config.getSegmentTable()
)
)
.bind("id", segment.getIdentifier())
.bind("dataSource", segment.getDataSource())
.bind("created_date", new DateTime().toString())
.bind("start", segment.getInterval().getStart().toString())
.bind("end", segment.getInterval().getEnd().toString())
.bind("partitioned", segment.getShardSpec().getPartitionNum())
.bind("version", segment.getVersion())
.bind("used", true)
.bind("payload", jsonMapper.writeValueAsString(segment))
.execute();
return null;
}
}
);
}
catch (Exception e) {
log.error(e, "Exception inserting into DB");
throw new RuntimeException(e);
}
}
}

View File

@ -0,0 +1,9 @@
package com.metamx.druid.realtime;
import org.skife.config.Config;
public abstract class DbSegmentPublisherConfig
{
@Config("druid.database.segmentTable")
public abstract String getSegmentTable();
}

View File

@ -24,6 +24,8 @@ package com.metamx.druid.realtime;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.druid.realtime.plumber.Plumber;
import com.metamx.druid.realtime.plumber.PlumberSchool;
import java.io.IOException;

View File

@ -50,8 +50,13 @@ public class KafkaFirehoseFactory implements FirehoseFactory
{
private static final Logger log = new Logger(KafkaFirehoseFactory.class);
@JsonProperty
private final Properties consumerProps;
@JsonProperty
private final String feed;
@JsonProperty
private final StringInputRowParser parser;
@JsonCreator

View File

@ -1,201 +0,0 @@
/*
* 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 com.metamx.druid.realtime;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.phonebook.PhoneBook;
import org.joda.time.DateTime;
import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.tweak.HandleCallback;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
/**
*/
public class MetadataUpdater
{
private static final Logger log = new Logger(MetadataUpdater.class);
private final Object lock = new Object();
private final ObjectMapper jsonMapper;
private final MetadataUpdaterConfig config;
private final PhoneBook yp;
private final String servedSegmentsLocation;
private final DBI dbi;
private volatile boolean started = false;
public MetadataUpdater(
ObjectMapper jsonMapper,
MetadataUpdaterConfig config,
PhoneBook yp,
DBI dbi
)
{
this.jsonMapper = jsonMapper;
this.config = config;
this.yp = yp;
this.servedSegmentsLocation = yp.combineParts(
Arrays.asList(
config.getServedSegmentsLocation(), config.getServerName()
)
);
this.dbi = dbi;
}
public Map<String, String> getStringProps()
{
return ImmutableMap.of(
"name", config.getServerName(),
"host", config.getHost(),
"maxSize", String.valueOf(config.getMaxSize()),
"type", "realtime"
);
}
public boolean hasStarted()
{
return started;
}
@LifecycleStart
public void start()
{
synchronized (lock) {
if (started) {
return;
}
log.info("Starting zkCoordinator for server[%s] with config[%s]", config.getServerName(), config);
if (yp.lookup(servedSegmentsLocation, Object.class) == null) {
yp.post(
config.getServedSegmentsLocation(),
config.getServerName(),
ImmutableMap.of("created", new DateTime().toString())
);
}
yp.announce(
config.getAnnounceLocation(),
config.getServerName(),
getStringProps()
);
started = true;
}
}
@LifecycleStop
public void stop()
{
synchronized (lock) {
if (!started) {
return;
}
log.info("Stopping MetadataUpdater with config[%s]", config);
yp.unannounce(config.getAnnounceLocation(), config.getServerName());
started = false;
}
}
public void announceSegment(DataSegment segment) throws IOException
{
log.info("Announcing realtime segment %s", segment.getIdentifier());
yp.announce(servedSegmentsLocation, segment.getIdentifier(), segment);
}
public void unannounceSegment(DataSegment segment) throws IOException
{
log.info("Unannouncing realtime segment %s", segment.getIdentifier());
yp.unannounce(servedSegmentsLocation, segment.getIdentifier());
}
public void publishSegment(final DataSegment segment) throws IOException
{
try {
List<Map<String, Object>> exists = dbi.withHandle(
new HandleCallback<List<Map<String, Object>>>()
{
@Override
public List<Map<String, Object>> withHandle(Handle handle) throws Exception
{
return handle.createQuery(
String.format("SELECT id FROM %s WHERE id=:id", config.getSegmentTable())
)
.bind("id", segment.getIdentifier())
.list();
}
}
);
if (!exists.isEmpty()) {
log.info("Found [%s] in DB, not updating DB", segment.getIdentifier());
return;
}
dbi.withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(
String.format(
"INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
config.getSegmentTable()
)
)
.bind("id", segment.getIdentifier())
.bind("dataSource", segment.getDataSource())
.bind("created_date", new DateTime().toString())
.bind("start", segment.getInterval().getStart().toString())
.bind("end", segment.getInterval().getEnd().toString())
.bind("partitioned", segment.getShardSpec().getPartitionNum())
.bind("version", segment.getVersion())
.bind("used", true)
.bind("payload", jsonMapper.writeValueAsString(segment))
.execute();
return null;
}
}
);
}
catch (Exception e) {
log.error(e, "Exception inserting into DB");
throw new RuntimeException(e);
}
}
}

View File

@ -36,6 +36,8 @@ import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.query.QueryToolChest;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.druid.query.segment.SegmentDescriptor;
import com.metamx.druid.realtime.plumber.Plumber;
import com.metamx.druid.realtime.plumber.Sink;
import com.metamx.emitter.EmittingLogger;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -154,6 +156,8 @@ public class RealtimeManager implements QuerySegmentWalker
final Period intermediatePersistPeriod = config.getIntermediatePersistPeriod();
try {
plumber.startJob();
long nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
while (firehose.hasMore()) {
final InputRow inputRow;

View File

@ -77,7 +77,8 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
private final Map<String, Object> injectablesMap = Maps.newLinkedHashMap();
private MetadataUpdater metadataUpdater = null;
private SegmentAnnouncer segmentAnnouncer = null;
private SegmentPublisher segmentPublisher = null;
private DataSegmentPusher dataSegmentPusher = null;
private List<FireDepartment> fireDepartments = null;
private ServerView view = null;
@ -102,10 +103,17 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
return this;
}
public RealtimeNode setMetadataUpdater(MetadataUpdater metadataUpdater)
public RealtimeNode setSegmentAnnouncer(SegmentAnnouncer segmentAnnouncer)
{
Preconditions.checkState(this.metadataUpdater == null, "Cannot set metadataUpdater once it has already been set.");
this.metadataUpdater = metadataUpdater;
Preconditions.checkState(this.segmentAnnouncer == null, "Cannot set segmentAnnouncer once it has already been set.");
this.segmentAnnouncer = segmentAnnouncer;
return this;
}
public RealtimeNode setSegmentPublisher(SegmentPublisher segmentPublisher)
{
Preconditions.checkState(this.segmentPublisher == null, "Cannot set segmentPublisher once it has already been set.");
this.segmentPublisher = segmentPublisher;
return this;
}
@ -130,10 +138,16 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
return this;
}
public MetadataUpdater getMetadataUpdater()
public SegmentAnnouncer getSegmentAnnouncer()
{
initializeMetadataUpdater();
return metadataUpdater;
initializeSegmentAnnouncer();
return segmentAnnouncer;
}
public SegmentPublisher getSegmentPublisher()
{
initializeSegmentPublisher();
return segmentPublisher;
}
public DataSegmentPusher getDataSegmentPusher()
@ -157,7 +171,8 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
protected void doInit() throws Exception
{
initializeView();
initializeMetadataUpdater();
initializeSegmentAnnouncer();
initializeSegmentPublisher();
initializeSegmentPusher();
initializeJacksonInjectables();
@ -213,7 +228,8 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
injectables.put("queryRunnerFactoryConglomerate", getConglomerate());
injectables.put("segmentPusher", dataSegmentPusher);
injectables.put("metadataUpdater", metadataUpdater);
injectables.put("segmentAnnouncer", segmentAnnouncer);
injectables.put("segmentPublisher", segmentPublisher);
injectables.put("serverView", view);
injectables.put("serviceEmitter", getEmitter());
@ -253,16 +269,25 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
}
}
protected void initializeMetadataUpdater()
protected void initializeSegmentAnnouncer()
{
if (metadataUpdater == null) {
metadataUpdater = new MetadataUpdater(
if (segmentAnnouncer == null) {
final ZkSegmentAnnouncerConfig zkSegmentAnnouncerConfig = getConfigFactory().build(ZkSegmentAnnouncerConfig.class);
segmentAnnouncer = new ZkSegmentAnnouncer(zkSegmentAnnouncerConfig, getPhoneBook());
getLifecycle().addManagedInstance(segmentAnnouncer);
}
}
protected void initializeSegmentPublisher()
{
if (segmentPublisher == null) {
final DbSegmentPublisherConfig dbSegmentPublisherConfig = getConfigFactory().build(DbSegmentPublisherConfig.class);
segmentPublisher = new DbSegmentPublisher(
getJsonMapper(),
getConfigFactory().build(MetadataUpdaterConfig.class),
getPhoneBook(),
dbSegmentPublisherConfig,
new DbConnector(getConfigFactory().build(DbConnectorConfig.class)).getDBI()
);
getLifecycle().addManagedInstance(metadataUpdater);
getLifecycle().addManagedInstance(segmentPublisher);
}
}

View File

@ -1,635 +0,0 @@
/*
* 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 com.metamx.druid.realtime;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.primitives.Ints;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.Pair;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.druid.Query;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.DruidServer;
import com.metamx.druid.client.ServerView;
import com.metamx.druid.guava.ThreadRenamingRunnable;
import com.metamx.druid.index.QueryableIndex;
import com.metamx.druid.index.QueryableIndexSegment;
import com.metamx.druid.index.Segment;
import com.metamx.druid.index.v1.IndexGranularity;
import com.metamx.druid.index.v1.IndexIO;
import com.metamx.druid.index.v1.IndexMerger;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.query.MetricsEmittingQueryRunner;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.query.QueryRunnerFactory;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.query.QueryToolChest;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Interval;
import org.joda.time.Period;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
/**
*/
public class RealtimePlumberSchool implements PlumberSchool
{
private static final EmittingLogger log = new EmittingLogger(RealtimePlumberSchool.class);
private static final ListeningExecutorService EXEC = MoreExecutors.sameThreadExecutor();
private final Period windowPeriod;
private final File basePersistDirectory;
private final IndexGranularity segmentGranularity;
private volatile Executor persistExecutor = null;
private volatile ScheduledExecutorService scheduledExecutor = null;
private volatile RejectionPolicyFactory rejectionPolicyFactory = null;
private volatile QueryRunnerFactoryConglomerate conglomerate = null;
private volatile DataSegmentPusher dataSegmentPusher = null;
private volatile MetadataUpdater metadataUpdater = null;
private volatile ServerView serverView = null;
private ServiceEmitter emitter;
@JsonCreator
public RealtimePlumberSchool(
@JsonProperty("windowPeriod") Period windowPeriod,
@JsonProperty("basePersistDirectory") File basePersistDirectory,
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity
)
{
this.windowPeriod = windowPeriod;
this.basePersistDirectory = basePersistDirectory;
this.segmentGranularity = segmentGranularity;
this.rejectionPolicyFactory = new ServerTimeRejectionPolicyFactory();
Preconditions.checkNotNull(windowPeriod, "RealtimePlumberSchool requires a windowPeriod.");
Preconditions.checkNotNull(basePersistDirectory, "RealtimePlumberSchool requires a basePersistDirectory.");
Preconditions.checkNotNull(segmentGranularity, "RealtimePlumberSchool requires a segmentGranularity.");
}
@JsonProperty("rejectionPolicy")
public void setRejectionPolicyFactory(RejectionPolicyFactory factory)
{
this.rejectionPolicyFactory = factory;
}
@JacksonInject("queryRunnerFactoryConglomerate")
public void setConglomerate(QueryRunnerFactoryConglomerate conglomerate)
{
this.conglomerate = conglomerate;
}
@JacksonInject("segmentPusher")
public void setDataSegmentPusher(DataSegmentPusher dataSegmentPusher)
{
this.dataSegmentPusher = dataSegmentPusher;
}
@JacksonInject("metadataUpdater")
public void setMetadataUpdater(MetadataUpdater metadataUpdater)
{
this.metadataUpdater = metadataUpdater;
}
@JacksonInject("serverView")
public void setServerView(ServerView serverView)
{
this.serverView = serverView;
}
@JacksonInject("serviceEmitter")
public void setServiceEmitter(ServiceEmitter emitter)
{
this.emitter = emitter;
}
@Override
public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics)
{
verifyState();
initializeExecutors();
computeBaseDir(schema).mkdirs();
final Map<Long, Sink> sinks = Maps.newConcurrentMap();
for (File sinkDir : computeBaseDir(schema).listFiles()) {
Interval sinkInterval = new Interval(sinkDir.getName().replace("_", "/"));
final File[] sinkFiles = sinkDir.listFiles();
Arrays.sort(
sinkFiles,
new Comparator<File>()
{
@Override
public int compare(File o1, File o2)
{
try {
return Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName()));
}
catch (NumberFormatException e) {
log.error(e, "Couldn't compare as numbers? [%s][%s]", o1, o2);
return o1.compareTo(o2);
}
}
}
);
try {
List<FireHydrant> hydrants = Lists.newArrayList();
for (File segmentDir : sinkFiles) {
log.info("Loading previously persisted segment at [%s]", segmentDir);
hydrants.add(
new FireHydrant(
new QueryableIndexSegment(null, IndexIO.loadIndex(segmentDir)),
Integer.parseInt(segmentDir.getName())
)
);
}
Sink currSink = new Sink(sinkInterval, schema, hydrants);
sinks.put(sinkInterval.getStartMillis(), currSink);
metadataUpdater.announceSegment(currSink.getSegment());
}
catch (IOException e) {
log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource())
.addData("interval", sinkInterval)
.emit();
}
}
serverView.registerSegmentCallback(
persistExecutor,
new ServerView.BaseSegmentCallback()
{
@Override
public ServerView.CallbackAction segmentAdded(DruidServer server, DataSegment segment)
{
if ("realtime".equals(server.getType())) {
return ServerView.CallbackAction.CONTINUE;
}
log.debug("Checking segment[%s] on server[%s]", segment, server);
if (schema.getDataSource().equals(segment.getDataSource())) {
final Interval interval = segment.getInterval();
for (Map.Entry<Long, Sink> entry : sinks.entrySet()) {
final Long sinkKey = entry.getKey();
if (interval.contains(sinkKey)) {
final Sink sink = entry.getValue();
log.info("Segment matches sink[%s]", sink);
if (segment.getVersion().compareTo(sink.getSegment().getVersion()) >= 0) {
try {
metadataUpdater.unannounceSegment(sink.getSegment());
FileUtils.deleteDirectory(computePersistDir(schema, sink.getInterval()));
sinks.remove(sinkKey);
}
catch (IOException e) {
log.makeAlert(e, "Unable to delete old segment for dataSource[%s].", schema.getDataSource())
.addData("interval", sink.getInterval())
.emit();
}
}
}
}
}
return ServerView.CallbackAction.CONTINUE;
}
}
);
final long truncatedNow = segmentGranularity.truncate(new DateTime()).getMillis();
final long windowMillis = windowPeriod.toStandardDuration().getMillis();
final RejectionPolicy rejectionPolicy = rejectionPolicyFactory.create(windowPeriod);
log.info("Creating plumber using rejectionPolicy[%s]", rejectionPolicy);
log.info(
"Expect to run at [%s]",
new DateTime().plus(
new Duration(System.currentTimeMillis(), segmentGranularity.increment(truncatedNow) + windowMillis)
)
);
ScheduledExecutors
.scheduleAtFixedRate(
scheduledExecutor,
new Duration(System.currentTimeMillis(), segmentGranularity.increment(truncatedNow) + windowMillis),
new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)),
new ThreadRenamingRunnable(String.format("%s-overseer", schema.getDataSource()))
{
@Override
public void doRun()
{
log.info("Starting merge and push.");
long minTimestamp = segmentGranularity.truncate(rejectionPolicy.getCurrMaxTime()).getMillis() - windowMillis;
List<Map.Entry<Long, Sink>> sinksToPush = Lists.newArrayList();
for (Map.Entry<Long, Sink> entry : sinks.entrySet()) {
final Long intervalStart = entry.getKey();
if (intervalStart < minTimestamp) {
log.info("Adding entry[%s] for merge and push.", entry);
sinksToPush.add(entry);
}
}
for (final Map.Entry<Long, Sink> entry : sinksToPush) {
final Sink sink = entry.getValue();
final String threadName = String.format(
"%s-%s-persist-n-merge", schema.getDataSource(), new DateTime(entry.getKey())
);
persistExecutor.execute(
new ThreadRenamingRunnable(threadName)
{
@Override
public void doRun()
{
final Interval interval = sink.getInterval();
for (FireHydrant hydrant : sink) {
if (!hydrant.hasSwapped()) {
log.info("Hydrant[%s] hasn't swapped yet, swapping. Sink[%s]", hydrant, sink);
final int rowCount = persistHydrant(hydrant, schema, interval);
metrics.incrementRowOutputCount(rowCount);
}
}
File mergedFile = null;
try {
List<QueryableIndex> indexes = Lists.newArrayList();
for (FireHydrant fireHydrant : sink) {
Segment segment = fireHydrant.getSegment();
final QueryableIndex queryableIndex = segment.asQueryableIndex();
log.info("Adding hydrant[%s]", fireHydrant);
indexes.add(queryableIndex);
}
mergedFile = IndexMerger.mergeQueryableIndex(
indexes,
schema.getAggregators(),
new File(computePersistDir(schema, interval), "merged")
);
QueryableIndex index = IndexIO.loadIndex(mergedFile);
DataSegment segment = dataSegmentPusher.push(
mergedFile,
sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions()))
);
metadataUpdater.publishSegment(segment);
}
catch (IOException e) {
log.makeAlert(e, "Failed to persist merged index[%s]", schema.getDataSource())
.addData("interval", interval)
.emit();
}
if (mergedFile != null) {
try {
if (mergedFile != null) {
log.info("Deleting Index File[%s]", mergedFile);
FileUtils.deleteDirectory(mergedFile);
}
}
catch (IOException e) {
log.warn(e, "Error deleting directory[%s]", mergedFile);
}
}
}
}
);
}
}
}
);
return new Plumber()
{
@Override
public Sink getSink(long timestamp)
{
if (!rejectionPolicy.accept(timestamp)) {
return null;
}
final long truncatedTime = segmentGranularity.truncate(timestamp);
Sink retVal = sinks.get(truncatedTime);
if (retVal == null) {
retVal = new Sink(
new Interval(new DateTime(truncatedTime), segmentGranularity.increment(new DateTime(truncatedTime))),
schema
);
try {
metadataUpdater.announceSegment(retVal.getSegment());
sinks.put(truncatedTime, retVal);
}
catch (IOException e) {
log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource())
.addData("interval", retVal.getInterval())
.emit();
}
}
return retVal;
}
@Override
public <T> QueryRunner<T> getQueryRunner(final Query<T> query)
{
final QueryRunnerFactory<T, Query<T>> factory = conglomerate.findFactory(query);
final Function<Query<T>, ServiceMetricEvent.Builder> builderFn =
new Function<Query<T>, ServiceMetricEvent.Builder>()
{
private final QueryToolChest<T,Query<T>> toolchest = factory.getToolchest();
@Override
public ServiceMetricEvent.Builder apply(@Nullable Query<T> input)
{
return toolchest.makeMetricBuilder(query);
}
};
return factory.mergeRunners(
EXEC,
FunctionalIterable
.create(sinks.values())
.transform(
new Function<Sink, QueryRunner<T>>()
{
@Override
public QueryRunner<T> apply(@Nullable Sink input)
{
return new MetricsEmittingQueryRunner<T>(
emitter,
builderFn,
factory.mergeRunners(
EXEC,
Iterables.transform(
input,
new Function<FireHydrant, QueryRunner<T>>()
{
@Override
public QueryRunner<T> apply(@Nullable FireHydrant input)
{
return factory.createRunner(input.getSegment());
}
}
)
)
);
}
}
)
);
}
@Override
public void persist(final Runnable commitRunnable)
{
final List<Pair<FireHydrant, Interval>> indexesToPersist = Lists.newArrayList();
for (Sink sink : sinks.values()) {
if (sink.swappable()) {
indexesToPersist.add(Pair.of(sink.swap(), sink.getInterval()));
}
}
log.info("Submitting persist runnable for dataSource[%s]", schema.getDataSource());
persistExecutor.execute(
new ThreadRenamingRunnable(String.format("%s-incremental-persist", schema.getDataSource()))
{
@Override
public void doRun()
{
for (Pair<FireHydrant, Interval> pair : indexesToPersist) {
metrics.incrementRowOutputCount(persistHydrant(pair.lhs, schema, pair.rhs));
}
commitRunnable.run();
}
}
);
}
@Override
public void finishJob()
{
throw new UnsupportedOperationException();
}
};
}
private File computeBaseDir(Schema schema)
{
return new File(basePersistDirectory, schema.getDataSource());
}
private File computePersistDir(Schema schema, Interval interval)
{
return new File(computeBaseDir(schema), interval.toString().replace("/", "_"));
}
/**
* Persists the given hydrant and returns the number of rows persisted
*
* @param indexToPersist
* @param schema
* @param interval
*
* @return the number of rows persisted
*/
private int persistHydrant(FireHydrant indexToPersist, Schema schema, Interval interval)
{
log.info("DataSource[%s], Interval[%s], persisting Hydrant[%s]", schema.getDataSource(), interval, indexToPersist);
try {
int numRows = indexToPersist.getIndex().size();
File persistedFile = IndexMerger.persist(
indexToPersist.getIndex(),
new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount()))
);
indexToPersist.swapSegment(new QueryableIndexSegment(null, IndexIO.loadIndex(persistedFile)));
return numRows;
}
catch (IOException e) {
log.makeAlert("dataSource[%s] -- incremental persist failed", schema.getDataSource())
.addData("interval", interval)
.addData("count", indexToPersist.getCount())
.emit();
throw Throwables.propagate(e);
}
}
private void verifyState()
{
Preconditions.checkNotNull(conglomerate, "must specify a queryRunnerFactoryConglomerate to do this action.");
Preconditions.checkNotNull(dataSegmentPusher, "must specify a segmentPusher to do this action.");
Preconditions.checkNotNull(metadataUpdater, "must specify a metadataUpdater to do this action.");
Preconditions.checkNotNull(serverView, "must specify a serverView to do this action.");
Preconditions.checkNotNull(emitter, "must specify a serviceEmitter to do this action.");
}
private void initializeExecutors()
{
if (persistExecutor == null) {
persistExecutor = Executors.newFixedThreadPool(
1,
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("plumber_persist_%d")
.build()
);
}
if (scheduledExecutor == null) {
scheduledExecutor = Executors.newScheduledThreadPool(
1,
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("plumber_scheduled_%d")
.build()
);
}
}
public interface RejectionPolicy
{
public DateTime getCurrMaxTime();
public boolean accept(long timestamp);
}
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "serverTime", value = ServerTimeRejectionPolicyFactory.class),
@JsonSubTypes.Type(name = "messageTime", value = MessageTimeRejectionPolicyFactory.class)
})
public static interface RejectionPolicyFactory
{
public RejectionPolicy create(Period windowPeriod);
}
public static class ServerTimeRejectionPolicyFactory implements RejectionPolicyFactory
{
@Override
public RejectionPolicy create(final Period windowPeriod)
{
final long windowMillis = windowPeriod.toStandardDuration().getMillis();
return new RejectionPolicy()
{
@Override
public DateTime getCurrMaxTime()
{
return new DateTime();
}
@Override
public boolean accept(long timestamp)
{
return timestamp >= (System.currentTimeMillis() - windowMillis);
}
@Override
public String toString()
{
return String.format("serverTime-%s", windowPeriod);
}
};
}
}
public static class MessageTimeRejectionPolicyFactory implements RejectionPolicyFactory
{
@Override
public RejectionPolicy create(final Period windowPeriod)
{
final long windowMillis = windowPeriod.toStandardDuration().getMillis();
return new RejectionPolicy()
{
private volatile long maxTimestamp = Long.MIN_VALUE;
@Override
public DateTime getCurrMaxTime()
{
return new DateTime(maxTimestamp);
}
@Override
public boolean accept(long timestamp)
{
maxTimestamp = Math.max(maxTimestamp, timestamp);
return timestamp >= (maxTimestamp - windowMillis);
}
@Override
public String toString()
{
return String.format("messageTime-%s", windowPeriod);
}
};
}
}
}

View File

@ -0,0 +1,11 @@
package com.metamx.druid.realtime;
import com.metamx.druid.client.DataSegment;
import java.io.IOException;
public interface SegmentAnnouncer
{
public void announceSegment(DataSegment segment) throws IOException;
public void unannounceSegment(DataSegment segment) throws IOException;
}

View File

@ -0,0 +1,10 @@
package com.metamx.druid.realtime;
import com.metamx.druid.client.DataSegment;
import java.io.IOException;
public interface SegmentPublisher
{
public void publishSegment(DataSegment segment) throws IOException;
}

View File

@ -0,0 +1,104 @@
package com.metamx.druid.realtime;
import com.google.common.collect.ImmutableMap;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.phonebook.PhoneBook;
import org.joda.time.DateTime;
import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
public class ZkSegmentAnnouncer implements SegmentAnnouncer
{
private static final Logger log = new Logger(ZkSegmentAnnouncer.class);
private final Object lock = new Object();
private final ZkSegmentAnnouncerConfig config;
private final PhoneBook yp;
private final String servedSegmentsLocation;
private volatile boolean started = false;
public ZkSegmentAnnouncer(
ZkSegmentAnnouncerConfig config,
PhoneBook yp
)
{
this.config = config;
this.yp = yp;
this.servedSegmentsLocation = yp.combineParts(
Arrays.asList(
config.getServedSegmentsLocation(), config.getServerName()
)
);
}
public Map<String, String> getStringProps()
{
return ImmutableMap.of(
"name", config.getServerName(),
"host", config.getHost(),
"maxSize", String.valueOf(config.getMaxSize()),
"type", "realtime"
);
}
@LifecycleStart
public void start()
{
synchronized (lock) {
if (started) {
return;
}
log.info("Starting zkCoordinator for server[%s] with config[%s]", config.getServerName(), config);
if (yp.lookup(servedSegmentsLocation, Object.class) == null) {
yp.post(
config.getServedSegmentsLocation(),
config.getServerName(),
ImmutableMap.of("created", new DateTime().toString())
);
}
yp.announce(
config.getAnnounceLocation(),
config.getServerName(),
getStringProps()
);
started = true;
}
}
@LifecycleStop
public void stop()
{
synchronized (lock) {
if (!started) {
return;
}
log.info("Stopping ZkSegmentAnnouncer with config[%s]", config);
yp.unannounce(config.getAnnounceLocation(), config.getServerName());
started = false;
}
}
public void announceSegment(DataSegment segment) throws IOException
{
log.info("Announcing realtime segment %s", segment.getIdentifier());
yp.announce(servedSegmentsLocation, segment.getIdentifier(), segment);
}
public void unannounceSegment(DataSegment segment) throws IOException
{
log.info("Unannouncing realtime segment %s", segment.getIdentifier());
yp.unannounce(servedSegmentsLocation, segment.getIdentifier());
}
}

View File

@ -0,0 +1,23 @@
package com.metamx.druid.realtime;
import org.skife.config.Config;
import org.skife.config.Default;
public abstract class ZkSegmentAnnouncerConfig
{
@Config("druid.host")
public abstract String getServerName();
@Config("druid.host")
public abstract String getHost();
@Config("druid.server.maxSize")
@Default("0")
public abstract long getMaxSize();
@Config("druid.zk.paths.announcementsPath")
public abstract String getAnnounceLocation();
@Config("druid.zk.paths.servedSegmentsPath")
public abstract String getServedSegmentsLocation();
}

View File

@ -0,0 +1,12 @@
package com.metamx.druid.realtime.plumber;
import org.joda.time.Interval;
public class IntervalStartVersioningPolicy implements VersioningPolicy
{
@Override
public String getVersion(Interval interval)
{
return interval.getStart().toString();
}
}

View File

@ -0,0 +1,39 @@
package com.metamx.druid.realtime.plumber;
import org.joda.time.DateTime;
import org.joda.time.Period;
public class MessageTimeRejectionPolicyFactory implements RejectionPolicyFactory
{
@Override
public RejectionPolicy create(final Period windowPeriod)
{
final long windowMillis = windowPeriod.toStandardDuration().getMillis();
return new RejectionPolicy()
{
private volatile long maxTimestamp = Long.MIN_VALUE;
@Override
public DateTime getCurrMaxTime()
{
return new DateTime(maxTimestamp);
}
@Override
public boolean accept(long timestamp)
{
maxTimestamp = Math.max(maxTimestamp, timestamp);
return timestamp >= (maxTimestamp - windowMillis);
}
@Override
public String toString()
{
return String.format("messageTime-%s", windowPeriod);
}
};
}
}

View File

@ -17,17 +17,33 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.realtime;
package com.metamx.druid.realtime.plumber;
import com.metamx.druid.Query;
import com.metamx.druid.query.QueryRunner;
/**
*/
public interface Plumber
{
/**
* Perform any initial setup. Should be called before using any other methods, and should be paired
* with a corresponding call to {@link #finishJob}.
*/
public void startJob();
public Sink getSink(long timestamp);
public <T> QueryRunner<T> getQueryRunner(Query<T> query);
/**
* Persist any in-memory indexed data to durable storage. This may be only somewhat durable, e.g. the
* machine's local disk.
*
* @param commitRunnable code to run after persisting data
*/
void persist(Runnable commitRunnable);
/**
* Perform any final processing and clean up after ourselves. Should be called after all data has been
* fed into sinks and persisted.
*/
public void finishJob();
}

View File

@ -17,11 +17,13 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.realtime;
package com.metamx.druid.realtime.plumber;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.metamx.druid.realtime.FireDepartmentMetrics;
import com.metamx.druid.realtime.Schema;
/**
*/

View File

@ -0,0 +1,631 @@
/*
* 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 com.metamx.druid.realtime.plumber;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.primitives.Ints;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.Pair;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.druid.Query;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.DruidServer;
import com.metamx.druid.client.ServerView;
import com.metamx.druid.guava.ThreadRenamingCallable;
import com.metamx.druid.guava.ThreadRenamingRunnable;
import com.metamx.druid.index.QueryableIndex;
import com.metamx.druid.index.QueryableIndexSegment;
import com.metamx.druid.index.Segment;
import com.metamx.druid.index.v1.IndexGranularity;
import com.metamx.druid.index.v1.IndexIO;
import com.metamx.druid.index.v1.IndexMerger;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.query.MetricsEmittingQueryRunner;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.query.QueryRunnerFactory;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.query.QueryToolChest;
import com.metamx.druid.realtime.FireDepartmentMetrics;
import com.metamx.druid.realtime.FireHydrant;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.SegmentPublisher;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Interval;
import org.joda.time.Period;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
/**
*/
public class RealtimePlumberSchool implements PlumberSchool
{
private static final EmittingLogger log = new EmittingLogger(RealtimePlumberSchool.class);
private static final ListeningExecutorService EXEC = MoreExecutors.sameThreadExecutor();
private final Period windowPeriod;
private final File basePersistDirectory;
private final IndexGranularity segmentGranularity;
private volatile VersioningPolicy versioningPolicy = null;
private volatile RejectionPolicyFactory rejectionPolicyFactory = null;
private volatile QueryRunnerFactoryConglomerate conglomerate = null;
private volatile DataSegmentPusher dataSegmentPusher = null;
private volatile SegmentAnnouncer segmentAnnouncer = null;
private volatile SegmentPublisher segmentPublisher = null;
private volatile ServerView serverView = null;
private ServiceEmitter emitter;
@JsonCreator
public RealtimePlumberSchool(
@JsonProperty("windowPeriod") Period windowPeriod,
@JsonProperty("basePersistDirectory") File basePersistDirectory,
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity
)
{
this.windowPeriod = windowPeriod;
this.basePersistDirectory = basePersistDirectory;
this.segmentGranularity = segmentGranularity;
this.versioningPolicy = new IntervalStartVersioningPolicy();
this.rejectionPolicyFactory = new ServerTimeRejectionPolicyFactory();
Preconditions.checkNotNull(windowPeriod, "RealtimePlumberSchool requires a windowPeriod.");
Preconditions.checkNotNull(basePersistDirectory, "RealtimePlumberSchool requires a basePersistDirectory.");
Preconditions.checkNotNull(segmentGranularity, "RealtimePlumberSchool requires a segmentGranularity.");
}
@JsonProperty("versioningPolicy")
public void setVersioningPolicy(VersioningPolicy versioningPolicy)
{
this.versioningPolicy = versioningPolicy;
}
@JsonProperty("rejectionPolicy")
public void setRejectionPolicyFactory(RejectionPolicyFactory factory)
{
this.rejectionPolicyFactory = factory;
}
@JacksonInject("queryRunnerFactoryConglomerate")
public void setConglomerate(QueryRunnerFactoryConglomerate conglomerate)
{
this.conglomerate = conglomerate;
}
@JacksonInject("segmentPusher")
public void setDataSegmentPusher(DataSegmentPusher dataSegmentPusher)
{
this.dataSegmentPusher = dataSegmentPusher;
}
@JacksonInject("segmentAnnouncer")
public void setSegmentAnnouncer(SegmentAnnouncer segmentAnnouncer)
{
this.segmentAnnouncer = segmentAnnouncer;
}
@JacksonInject("segmentPublisher")
public void setSegmentPublisher(SegmentPublisher segmentPublisher)
{
this.segmentPublisher = segmentPublisher;
}
@JacksonInject("serverView")
public void setServerView(ServerView serverView)
{
this.serverView = serverView;
}
@JacksonInject("serviceEmitter")
public void setServiceEmitter(ServiceEmitter emitter)
{
this.emitter = emitter;
}
@Override
public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics)
{
verifyState();
final RejectionPolicy rejectionPolicy = rejectionPolicyFactory.create(windowPeriod);
log.info("Creating plumber using rejectionPolicy[%s]", rejectionPolicy);
return new Plumber()
{
private volatile boolean stopped = false;
private volatile ExecutorService persistExecutor = null;
private volatile ScheduledExecutorService scheduledExecutor = null;
private final Map<Long, Sink> sinks = Maps.newConcurrentMap();
@Override
public void startJob()
{
computeBaseDir(schema).mkdirs();
initializeExecutors();
bootstrapSinksFromDisk();
registerServerViewCallback();
startPersistThread();
}
@Override
public Sink getSink(long timestamp)
{
if (!rejectionPolicy.accept(timestamp)) {
return null;
}
final long truncatedTime = segmentGranularity.truncate(timestamp);
Sink retVal = sinks.get(truncatedTime);
if (retVal == null) {
final Interval sinkInterval = new Interval(
new DateTime(truncatedTime),
segmentGranularity.increment(new DateTime(truncatedTime))
);
retVal = new Sink(sinkInterval, schema, versioningPolicy.getVersion(sinkInterval));
try {
segmentAnnouncer.announceSegment(retVal.getSegment());
sinks.put(truncatedTime, retVal);
}
catch (IOException e) {
log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource())
.addData("interval", retVal.getInterval())
.emit();
}
}
return retVal;
}
@Override
public <T> QueryRunner<T> getQueryRunner(final Query<T> query)
{
final QueryRunnerFactory<T, Query<T>> factory = conglomerate.findFactory(query);
final Function<Query<T>, ServiceMetricEvent.Builder> builderFn =
new Function<Query<T>, ServiceMetricEvent.Builder>()
{
private final QueryToolChest<T,Query<T>> toolchest = factory.getToolchest();
@Override
public ServiceMetricEvent.Builder apply(@Nullable Query<T> input)
{
return toolchest.makeMetricBuilder(query);
}
};
return factory.mergeRunners(
EXEC,
FunctionalIterable
.create(sinks.values())
.transform(
new Function<Sink, QueryRunner<T>>()
{
@Override
public QueryRunner<T> apply(@Nullable Sink input)
{
return new MetricsEmittingQueryRunner<T>(
emitter,
builderFn,
factory.mergeRunners(
EXEC,
Iterables.transform(
input,
new Function<FireHydrant, QueryRunner<T>>()
{
@Override
public QueryRunner<T> apply(@Nullable FireHydrant input)
{
return factory.createRunner(input.getSegment());
}
}
)
)
);
}
}
)
);
}
@Override
public void persist(final Runnable commitRunnable)
{
final List<Pair<FireHydrant, Interval>> indexesToPersist = Lists.newArrayList();
for (Sink sink : sinks.values()) {
if (sink.swappable()) {
indexesToPersist.add(Pair.of(sink.swap(), sink.getInterval()));
}
}
log.info("Submitting persist runnable for dataSource[%s]", schema.getDataSource());
persistExecutor.execute(
new ThreadRenamingRunnable(String.format("%s-incremental-persist", schema.getDataSource()))
{
@Override
public void doRun()
{
for (Pair<FireHydrant, Interval> pair : indexesToPersist) {
metrics.incrementRowOutputCount(persistHydrant(pair.lhs, schema, pair.rhs));
}
commitRunnable.run();
}
}
);
}
@Override
public void finishJob()
{
stopped = true;
for (final Sink sink : sinks.values()) {
try {
segmentAnnouncer.unannounceSegment(sink.getSegment());
}
catch (Exception e) {
log.makeAlert("Failed to unannounce segment on shutdown")
.addData("segment", sink.getSegment())
.emit();
}
}
// scheduledExecutor is shutdown here, but persistExecutor is shutdown when the
// ServerView sends it a new segment callback
if (scheduledExecutor != null) {
scheduledExecutor.shutdown();
}
}
private void initializeExecutors()
{
if (persistExecutor == null) {
persistExecutor = Executors.newFixedThreadPool(
1,
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("plumber_persist_%d")
.build()
);
}
if (scheduledExecutor == null) {
scheduledExecutor = Executors.newScheduledThreadPool(
1,
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("plumber_scheduled_%d")
.build()
);
}
}
private void bootstrapSinksFromDisk()
{
for (File sinkDir : computeBaseDir(schema).listFiles()) {
Interval sinkInterval = new Interval(sinkDir.getName().replace("_", "/"));
final File[] sinkFiles = sinkDir.listFiles();
Arrays.sort(
sinkFiles,
new Comparator<File>()
{
@Override
public int compare(File o1, File o2)
{
try {
return Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName()));
}
catch (NumberFormatException e) {
log.error(e, "Couldn't compare as numbers? [%s][%s]", o1, o2);
return o1.compareTo(o2);
}
}
}
);
try {
List<FireHydrant> hydrants = Lists.newArrayList();
for (File segmentDir : sinkFiles) {
log.info("Loading previously persisted segment at [%s]", segmentDir);
hydrants.add(
new FireHydrant(
new QueryableIndexSegment(null, IndexIO.loadIndex(segmentDir)),
Integer.parseInt(segmentDir.getName())
)
);
}
Sink currSink = new Sink(sinkInterval, schema, versioningPolicy.getVersion(sinkInterval), hydrants);
sinks.put(sinkInterval.getStartMillis(), currSink);
segmentAnnouncer.announceSegment(currSink.getSegment());
}
catch (IOException e) {
log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource())
.addData("interval", sinkInterval)
.emit();
}
}
}
private void registerServerViewCallback()
{
serverView.registerSegmentCallback(
persistExecutor,
new ServerView.BaseSegmentCallback()
{
@Override
public ServerView.CallbackAction segmentAdded(DruidServer server, DataSegment segment)
{
if (stopped) {
log.info("Unregistering ServerViewCallback");
persistExecutor.shutdown();
return ServerView.CallbackAction.UNREGISTER;
}
if ("realtime".equals(server.getType())) {
return ServerView.CallbackAction.CONTINUE;
}
log.debug("Checking segment[%s] on server[%s]", segment, server);
if (schema.getDataSource().equals(segment.getDataSource())) {
final Interval interval = segment.getInterval();
for (Map.Entry<Long, Sink> entry : sinks.entrySet()) {
final Long sinkKey = entry.getKey();
if (interval.contains(sinkKey)) {
final Sink sink = entry.getValue();
log.info("Segment matches sink[%s]", sink);
if (segment.getVersion().compareTo(sink.getSegment().getVersion()) >= 0) {
try {
segmentAnnouncer.unannounceSegment(sink.getSegment());
FileUtils.deleteDirectory(computePersistDir(schema, sink.getInterval()));
sinks.remove(sinkKey);
}
catch (IOException e) {
log.makeAlert(e, "Unable to delete old segment for dataSource[%s].", schema.getDataSource())
.addData("interval", sink.getInterval())
.emit();
}
}
}
}
}
return ServerView.CallbackAction.CONTINUE;
}
}
);
}
private void startPersistThread()
{
final long truncatedNow = segmentGranularity.truncate(new DateTime()).getMillis();
final long windowMillis = windowPeriod.toStandardDuration().getMillis();
log.info(
"Expect to run at [%s]",
new DateTime().plus(
new Duration(System.currentTimeMillis(), segmentGranularity.increment(truncatedNow) + windowMillis)
)
);
ScheduledExecutors
.scheduleAtFixedRate(
scheduledExecutor,
new Duration(System.currentTimeMillis(), segmentGranularity.increment(truncatedNow) + windowMillis),
new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)),
new ThreadRenamingCallable<ScheduledExecutors.Signal>(
String.format(
"%s-overseer-%d",
schema.getDataSource(),
schema.getShardSpec().getPartitionNum()
)
)
{
@Override
public ScheduledExecutors.Signal doCall()
{
if (stopped) {
log.info("Stopping merge-n-push overseer thread");
return ScheduledExecutors.Signal.STOP;
}
log.info("Starting merge and push.");
long minTimestamp = segmentGranularity.truncate(rejectionPolicy.getCurrMaxTime()).getMillis()
- windowMillis;
List<Map.Entry<Long, Sink>> sinksToPush = Lists.newArrayList();
for (Map.Entry<Long, Sink> entry : sinks.entrySet()) {
final Long intervalStart = entry.getKey();
if (intervalStart < minTimestamp) {
log.info("Adding entry[%s] for merge and push.", entry);
sinksToPush.add(entry);
}
}
for (final Map.Entry<Long, Sink> entry : sinksToPush) {
final Sink sink = entry.getValue();
final String threadName = String.format(
"%s-%s-persist-n-merge", schema.getDataSource(), new DateTime(entry.getKey())
);
persistExecutor.execute(
new ThreadRenamingRunnable(threadName)
{
@Override
public void doRun()
{
final Interval interval = sink.getInterval();
for (FireHydrant hydrant : sink) {
if (!hydrant.hasSwapped()) {
log.info("Hydrant[%s] hasn't swapped yet, swapping. Sink[%s]", hydrant, sink);
final int rowCount = persistHydrant(hydrant, schema, interval);
metrics.incrementRowOutputCount(rowCount);
}
}
File mergedFile = null;
try {
List<QueryableIndex> indexes = Lists.newArrayList();
for (FireHydrant fireHydrant : sink) {
Segment segment = fireHydrant.getSegment();
final QueryableIndex queryableIndex = segment.asQueryableIndex();
log.info("Adding hydrant[%s]", fireHydrant);
indexes.add(queryableIndex);
}
mergedFile = IndexMerger.mergeQueryableIndex(
indexes,
schema.getAggregators(),
new File(computePersistDir(schema, interval), "merged")
);
QueryableIndex index = IndexIO.loadIndex(mergedFile);
DataSegment segment = dataSegmentPusher.push(
mergedFile,
sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions()))
);
segmentPublisher.publishSegment(segment);
}
catch (IOException e) {
log.makeAlert(e, "Failed to persist merged index[%s]", schema.getDataSource())
.addData("interval", interval)
.emit();
}
if (mergedFile != null) {
try {
if (mergedFile != null) {
log.info("Deleting Index File[%s]", mergedFile);
FileUtils.deleteDirectory(mergedFile);
}
}
catch (IOException e) {
log.warn(e, "Error deleting directory[%s]", mergedFile);
}
}
}
}
);
}
if (stopped) {
log.info("Stopping merge-n-push overseer thread");
return ScheduledExecutors.Signal.STOP;
} else {
return ScheduledExecutors.Signal.REPEAT;
}
}
}
);
}
};
}
private File computeBaseDir(Schema schema)
{
return new File(basePersistDirectory, schema.getDataSource());
}
private File computePersistDir(Schema schema, Interval interval)
{
return new File(computeBaseDir(schema), interval.toString().replace("/", "_"));
}
/**
* Persists the given hydrant and returns the number of rows persisted
*
* @param indexToPersist
* @param schema
* @param interval
*
* @return the number of rows persisted
*/
private int persistHydrant(FireHydrant indexToPersist, Schema schema, Interval interval)
{
log.info("DataSource[%s], Interval[%s], persisting Hydrant[%s]", schema.getDataSource(), interval, indexToPersist);
try {
int numRows = indexToPersist.getIndex().size();
File persistedFile = IndexMerger.persist(
indexToPersist.getIndex(),
new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount()))
);
indexToPersist.swapSegment(new QueryableIndexSegment(null, IndexIO.loadIndex(persistedFile)));
return numRows;
}
catch (IOException e) {
log.makeAlert("dataSource[%s] -- incremental persist failed", schema.getDataSource())
.addData("interval", interval)
.addData("count", indexToPersist.getCount())
.emit();
throw Throwables.propagate(e);
}
}
private void verifyState()
{
Preconditions.checkNotNull(conglomerate, "must specify a queryRunnerFactoryConglomerate to do this action.");
Preconditions.checkNotNull(dataSegmentPusher, "must specify a segmentPusher to do this action.");
Preconditions.checkNotNull(segmentAnnouncer, "must specify a segmentAnnouncer to do this action.");
Preconditions.checkNotNull(segmentPublisher, "must specify a segmentPublisher to do this action.");
Preconditions.checkNotNull(serverView, "must specify a serverView to do this action.");
Preconditions.checkNotNull(emitter, "must specify a serviceEmitter to do this action.");
}
}

View File

@ -0,0 +1,9 @@
package com.metamx.druid.realtime.plumber;
import org.joda.time.DateTime;
public interface RejectionPolicy
{
public DateTime getCurrMaxTime();
public boolean accept(long timestamp);
}

View File

@ -0,0 +1,15 @@
package com.metamx.druid.realtime.plumber;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import org.joda.time.Period;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "serverTime", value = ServerTimeRejectionPolicyFactory.class),
@JsonSubTypes.Type(name = "messageTime", value = MessageTimeRejectionPolicyFactory.class)
})
public interface RejectionPolicyFactory
{
public RejectionPolicy create(Period windowPeriod);
}

View File

@ -0,0 +1,34 @@
package com.metamx.druid.realtime.plumber;
import org.joda.time.DateTime;
import org.joda.time.Period;
public class ServerTimeRejectionPolicyFactory implements RejectionPolicyFactory
{
@Override
public RejectionPolicy create(final Period windowPeriod)
{
final long windowMillis = windowPeriod.toStandardDuration().getMillis();
return new RejectionPolicy()
{
@Override
public DateTime getCurrMaxTime()
{
return new DateTime();
}
@Override
public boolean accept(long timestamp)
{
return timestamp >= (System.currentTimeMillis() - windowMillis);
}
@Override
public String toString()
{
return String.format("serverTime-%s", windowPeriod);
}
};
}
}

View File

@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.realtime;
package com.metamx.druid.realtime.plumber;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
@ -32,6 +32,8 @@ import com.metamx.druid.client.DataSegment;
import com.metamx.druid.index.v1.IncrementalIndex;
import com.metamx.druid.index.v1.IndexIO;
import com.metamx.druid.input.InputRow;
import com.metamx.druid.realtime.FireHydrant;
import com.metamx.druid.realtime.Schema;
import org.joda.time.Interval;
import javax.annotation.Nullable;
@ -50,16 +52,19 @@ public class Sink implements Iterable<FireHydrant>
private final Interval interval;
private final Schema schema;
private final String version;
private final CopyOnWriteArrayList<FireHydrant> hydrants = new CopyOnWriteArrayList<FireHydrant>();
public Sink(
Interval interval,
Schema schema
Schema schema,
String version
)
{
this.schema = schema;
this.interval = interval;
this.version = version;
makeNewCurrIndex(interval.getStartMillis(), schema);
}
@ -67,11 +72,13 @@ public class Sink implements Iterable<FireHydrant>
public Sink(
Interval interval,
Schema schema,
String version,
List<FireHydrant> hydrants
)
{
this.schema = schema;
this.interval = interval;
this.version = version;
for (int i = 0; i < hydrants.size(); ++i) {
final FireHydrant hydrant = hydrants.get(i);
@ -100,6 +107,13 @@ public class Sink implements Iterable<FireHydrant>
}
}
public boolean isEmpty()
{
synchronized (currIndex) {
return hydrants.size() == 1 && currIndex.getIndex().isEmpty();
}
}
/**
* If currIndex is A, creates a new index B, sets currIndex to B and returns A.
*
@ -122,7 +136,7 @@ public class Sink implements Iterable<FireHydrant>
return new DataSegment(
schema.getDataSource(),
interval,
interval.getStart().toString(),
version,
ImmutableMap.<String, Object>of(),
Lists.<String>newArrayList(),
Lists.transform(

View File

@ -0,0 +1,14 @@
package com.metamx.druid.realtime.plumber;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import org.joda.time.Interval;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "intervalStart", value = IntervalStartVersioningPolicy.class)
})
public interface VersioningPolicy
{
public String getVersion(Interval interval);
}

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.22-SNAPSHOT</version>
<version>0.3.23-SNAPSHOT</version>
</parent>
<dependencies>
@ -63,12 +63,10 @@
<dependency>
<groupId>commons-cli</groupId>
<artifactId>commons-cli</artifactId>
<version>1.2</version>
</dependency>
<dependency>
<groupId>commons-lang</groupId>
<artifactId>commons-lang</artifactId>
<version>2.6</version>
</dependency>
<dependency>
<groupId>commons-io</groupId>

View File

@ -204,7 +204,7 @@ public enum IndexGranularity
@Override
public long increment(long timeMillis)
{
return timeMillis - MILLIS_IN;
return timeMillis + MILLIS_IN;
}
@Override
@ -273,7 +273,7 @@ public enum IndexGranularity
@Override
public long increment(long timeMillis)
{
return timeMillis - MILLIS_IN;
return timeMillis + MILLIS_IN;
}
@Override