mirror of https://github.com/apache/druid.git
Merge branch 'master' of github.com:metamx/druid
This commit is contained in:
commit
39449e6a3e
|
@ -33,7 +33,8 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
|||
@JsonSubTypes.Type(name="selector", value=SelectorDimFilter.class),
|
||||
@JsonSubTypes.Type(name="extraction", value=ExtractionDimFilter.class),
|
||||
@JsonSubTypes.Type(name="regex", value=RegexDimFilter.class),
|
||||
@JsonSubTypes.Type(name="search", value=SearchQueryDimFilter.class)
|
||||
@JsonSubTypes.Type(name="search", value=SearchQueryDimFilter.class),
|
||||
@JsonSubTypes.Type(name="javascript", value=JavaScriptDimFilter.class)
|
||||
})
|
||||
public interface DimFilter
|
||||
{
|
||||
|
|
|
@ -34,6 +34,7 @@ class DimFilterCacheHelper
|
|||
static final byte EXTRACTION_CACHE_ID = 0x4;
|
||||
static final byte REGEX_CACHE_ID = 0x5;
|
||||
static final byte SEARCH_QUERY_TYPE_ID = 0x6;
|
||||
static final byte JAVASCRIPT_CACHE_ID = 0x7;
|
||||
|
||||
static byte[] computeCacheKey(byte cacheIdKey, List<DimFilter> filters)
|
||||
{
|
||||
|
|
|
@ -0,0 +1,48 @@
|
|||
package com.metamx.druid.query.filter;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Charsets;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
public class JavaScriptDimFilter implements DimFilter
|
||||
{
|
||||
private final String dimension;
|
||||
private final String function;
|
||||
|
||||
@JsonCreator
|
||||
public JavaScriptDimFilter(
|
||||
@JsonProperty("dimension") String dimension,
|
||||
@JsonProperty("function") String function
|
||||
)
|
||||
{
|
||||
this.dimension = dimension;
|
||||
this.function = function;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDimension()
|
||||
{
|
||||
return dimension;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getFunction()
|
||||
{
|
||||
return function;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
final byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8);
|
||||
final byte[] functionBytes = function.getBytes(Charsets.UTF_8);
|
||||
|
||||
return ByteBuffer.allocate(1 + dimensionBytes.length + functionBytes.length)
|
||||
.put(DimFilterCacheHelper.JAVASCRIPT_CACHE_ID)
|
||||
.put(dimensionBytes)
|
||||
.put(functionBytes)
|
||||
.array();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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;
|
||||
|
|
@ -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;
|
||||
|
||||
/**
|
||||
*/
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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{}";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,31 +1,39 @@
|
|||
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.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;
|
||||
|
||||
@JsonIgnore
|
||||
private final boolean allowOlderVersions;
|
||||
|
||||
public SegmentInsertAction(Set<DataSegment> segments)
|
||||
{
|
||||
this(segments, false);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public SegmentInsertAction(
|
||||
@JsonProperty("segments") Set<DataSegment> segments
|
||||
@JsonProperty("segments") Set<DataSegment> segments,
|
||||
@JsonProperty("allowOlderVersions") boolean allowOlderVersions
|
||||
)
|
||||
{
|
||||
this.segments = ImmutableSet.copyOf(segments);
|
||||
this.allowOlderVersions = allowOlderVersions;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -34,34 +42,49 @@ public class SegmentInsertAction implements TaskAction<Void>
|
|||
return segments;
|
||||
}
|
||||
|
||||
public TypeReference<Void> getReturnTypeReference()
|
||||
@JsonProperty
|
||||
public boolean isAllowOlderVersions()
|
||||
{
|
||||
return new TypeReference<Void>() {};
|
||||
return allowOlderVersions;
|
||||
}
|
||||
|
||||
public SegmentInsertAction withAllowOlderVersions(boolean _allowOlderVersions)
|
||||
{
|
||||
return new SegmentInsertAction(segments, _allowOlderVersions);
|
||||
}
|
||||
|
||||
public TypeReference<Set<DataSegment>> getReturnTypeReference()
|
||||
{
|
||||
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, false)) {
|
||||
throw new ISE("Segments not covered by locks for task: %s", task.getId());
|
||||
if(!toolbox.taskLockCoversSegments(task, segments, allowOlderVersions)) {
|
||||
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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -19,23 +19,33 @@
|
|||
|
||||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
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
|
||||
{
|
||||
private static final Joiner ID_JOINER = Joiner.on("_");
|
||||
|
||||
@JsonIgnore
|
||||
private final String id;
|
||||
|
||||
@JsonIgnore
|
||||
private final String groupId;
|
||||
|
||||
@JsonIgnore
|
||||
private final String dataSource;
|
||||
|
||||
@JsonIgnore
|
||||
private final Optional<Interval> interval;
|
||||
|
||||
protected AbstractTask(String id, String dataSource, Interval interval)
|
||||
|
@ -79,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
|
||||
{
|
||||
|
|
|
@ -48,11 +48,12 @@ public class AppendTask extends MergeTaskBase
|
|||
{
|
||||
@JsonCreator
|
||||
public AppendTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("segments") List<DataSegment> segments
|
||||
)
|
||||
{
|
||||
super(dataSource, segments);
|
||||
super(id, dataSource, segments);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -50,12 +50,13 @@ public class DeleteTask extends AbstractTask
|
|||
|
||||
@JsonCreator
|
||||
public DeleteTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
super(
|
||||
String.format(
|
||||
id != null ? id : String.format(
|
||||
"delete_%s_%s_%s_%s",
|
||||
dataSource,
|
||||
interval.getStart(),
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
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.Preconditions;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
@ -41,7 +42,7 @@ import java.util.List;
|
|||
|
||||
public class HadoopIndexTask extends AbstractTask
|
||||
{
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final HadoopDruidIndexerConfig config;
|
||||
|
||||
private static final Logger log = new Logger(HadoopIndexTask.class);
|
||||
|
@ -58,11 +59,12 @@ public class HadoopIndexTask extends AbstractTask
|
|||
|
||||
@JsonCreator
|
||||
public HadoopIndexTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("config") HadoopDruidIndexerConfig config
|
||||
)
|
||||
{
|
||||
super(
|
||||
String.format("index_hadoop_%s_%s", config.getDataSource(), new DateTime()),
|
||||
id != null ? id : String.format("index_hadoop_%s_%s", config.getDataSource(), new DateTime()),
|
||||
config.getDataSource(),
|
||||
JodaUtils.umbrellaInterval(config.getIntervals())
|
||||
);
|
||||
|
@ -133,4 +135,10 @@ public class HadoopIndexTask extends AbstractTask
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public HadoopDruidIndexerConfig getConfig()
|
||||
{
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
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.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
@ -48,22 +49,23 @@ import java.util.Set;
|
|||
|
||||
public class IndexDeterminePartitionsTask extends AbstractTask
|
||||
{
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final FirehoseFactory firehoseFactory;
|
||||
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final Schema schema;
|
||||
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final long targetPartitionSize;
|
||||
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final int rowFlushBoundary;
|
||||
|
||||
private static final Logger log = new Logger(IndexTask.class);
|
||||
|
||||
@JsonCreator
|
||||
public IndexDeterminePartitionsTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("groupId") String groupId,
|
||||
@JsonProperty("interval") Interval interval,
|
||||
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
|
||||
|
@ -73,7 +75,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||
)
|
||||
{
|
||||
super(
|
||||
String.format(
|
||||
id != null ? id : String.format(
|
||||
"%s_partitions_%s_%s",
|
||||
groupId,
|
||||
interval.getStart(),
|
||||
|
@ -243,6 +245,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||
public Task apply(ShardSpec shardSpec)
|
||||
{
|
||||
return new IndexGeneratorTask(
|
||||
null,
|
||||
getGroupId(),
|
||||
getImplicitLockInterval().get(),
|
||||
firehoseFactory,
|
||||
|
@ -262,4 +265,28 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public FirehoseFactory getFirehoseFactory()
|
||||
{
|
||||
return firehoseFactory;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Schema getSchema()
|
||||
{
|
||||
return schema;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getTargetPartitionSize()
|
||||
{
|
||||
return targetPartitionSize;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getRowFlushBoundary()
|
||||
{
|
||||
return rowFlushBoundary;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
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.Preconditions;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
@ -38,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;
|
||||
|
||||
|
@ -52,19 +53,20 @@ import java.util.concurrent.CopyOnWriteArrayList;
|
|||
|
||||
public class IndexGeneratorTask extends AbstractTask
|
||||
{
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final FirehoseFactory firehoseFactory;
|
||||
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final Schema schema;
|
||||
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final int rowFlushBoundary;
|
||||
|
||||
private static final Logger log = new Logger(IndexTask.class);
|
||||
|
||||
@JsonCreator
|
||||
public IndexGeneratorTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("groupId") String groupId,
|
||||
@JsonProperty("interval") Interval interval,
|
||||
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
|
||||
|
@ -73,7 +75,7 @@ public class IndexGeneratorTask extends AbstractTask
|
|||
)
|
||||
{
|
||||
super(
|
||||
String.format(
|
||||
id != null ? id : String.format(
|
||||
"%s_generator_%s_%s_%s",
|
||||
groupId,
|
||||
interval.getStart(),
|
||||
|
@ -216,4 +218,22 @@ public class IndexGeneratorTask extends AbstractTask
|
|||
|
||||
return schema.getShardSpec().isInChunk(eventDimensions);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public FirehoseFactory getFirehoseFactory()
|
||||
{
|
||||
return firehoseFactory;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Schema getSchema()
|
||||
{
|
||||
return schema;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getRowFlushBoundary()
|
||||
{
|
||||
return rowFlushBoundary;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
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.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -40,28 +41,29 @@ import java.util.List;
|
|||
|
||||
public class IndexTask extends AbstractTask
|
||||
{
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final GranularitySpec granularitySpec;
|
||||
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final AggregatorFactory[] aggregators;
|
||||
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final QueryGranularity indexGranularity;
|
||||
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final long targetPartitionSize;
|
||||
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final FirehoseFactory firehoseFactory;
|
||||
|
||||
@JsonProperty
|
||||
@JsonIgnore
|
||||
private final int rowFlushBoundary;
|
||||
|
||||
private static final Logger log = new Logger(IndexTask.class);
|
||||
|
||||
@JsonCreator
|
||||
public IndexTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("granularitySpec") GranularitySpec granularitySpec,
|
||||
@JsonProperty("aggregators") AggregatorFactory[] aggregators,
|
||||
|
@ -73,7 +75,7 @@ public class IndexTask extends AbstractTask
|
|||
{
|
||||
super(
|
||||
// _not_ the version, just something uniqueish
|
||||
String.format("index_%s_%s", dataSource, new DateTime().toString()),
|
||||
id != null ? id : String.format("index_%s_%s", dataSource, new DateTime().toString()),
|
||||
dataSource,
|
||||
new Interval(
|
||||
granularitySpec.bucketIntervals().first().getStart(),
|
||||
|
@ -98,6 +100,7 @@ public class IndexTask extends AbstractTask
|
|||
// Need to do one pass over the data before indexing in order to determine good partitions
|
||||
retVal.add(
|
||||
new IndexDeterminePartitionsTask(
|
||||
null,
|
||||
getGroupId(),
|
||||
interval,
|
||||
firehoseFactory,
|
||||
|
@ -115,6 +118,7 @@ public class IndexTask extends AbstractTask
|
|||
// Jump straight into indexing
|
||||
retVal.add(
|
||||
new IndexGeneratorTask(
|
||||
null,
|
||||
getGroupId(),
|
||||
interval,
|
||||
firehoseFactory,
|
||||
|
@ -151,4 +155,41 @@ public class IndexTask extends AbstractTask
|
|||
{
|
||||
throw new IllegalStateException("IndexTasks should not be run!");
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public GranularitySpec getGranularitySpec()
|
||||
{
|
||||
return granularitySpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public AggregatorFactory[] getAggregators()
|
||||
{
|
||||
return aggregators;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public QueryGranularity getIndexGranularity()
|
||||
{
|
||||
return indexGranularity;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getTargetPartitionSize()
|
||||
{
|
||||
return targetPartitionSize;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public FirehoseFactory getFirehoseFactory()
|
||||
{
|
||||
return firehoseFactory;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getRowFlushBoundary()
|
||||
{
|
||||
return rowFlushBoundary;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -45,12 +45,13 @@ public class KillTask extends AbstractTask
|
|||
|
||||
@JsonCreator
|
||||
public KillTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
super(
|
||||
String.format(
|
||||
id != null ? id : String.format(
|
||||
"kill_%s_%s_%s_%s",
|
||||
dataSource,
|
||||
interval.getStart(),
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
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.Function;
|
||||
import com.google.common.base.Throwables;
|
||||
|
@ -42,16 +43,18 @@ import java.util.Map;
|
|||
*/
|
||||
public class MergeTask extends MergeTaskBase
|
||||
{
|
||||
@JsonIgnore
|
||||
private final List<AggregatorFactory> aggregators;
|
||||
|
||||
@JsonCreator
|
||||
public MergeTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("segments") List<DataSegment> segments,
|
||||
@JsonProperty("aggregations") List<AggregatorFactory> aggregators
|
||||
)
|
||||
{
|
||||
super(dataSource, segments);
|
||||
super(id, dataSource, segments);
|
||||
this.aggregators = aggregators;
|
||||
}
|
||||
|
||||
|
@ -86,4 +89,10 @@ public class MergeTask extends MergeTaskBase
|
|||
{
|
||||
return "merge";
|
||||
}
|
||||
|
||||
@JsonProperty("aggregations")
|
||||
public List<AggregatorFactory> getAggregators()
|
||||
{
|
||||
return aggregators;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Function;
|
||||
|
@ -26,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;
|
||||
|
@ -48,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;
|
||||
|
@ -56,15 +59,18 @@ import java.util.Set;
|
|||
*/
|
||||
public abstract class MergeTaskBase extends AbstractTask
|
||||
{
|
||||
@JsonIgnore
|
||||
private final List<DataSegment> segments;
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(MergeTaskBase.class);
|
||||
|
||||
protected MergeTaskBase(final String dataSource, final List<DataSegment> segments)
|
||||
protected MergeTaskBase(final String id, final String dataSource, final List<DataSegment> segments)
|
||||
{
|
||||
super(
|
||||
// _not_ the version, just something uniqueish
|
||||
String.format("merge_%s_%s", computeProcessingID(dataSource, segments), new DateTime().toString()),
|
||||
id != null ? id : String.format(
|
||||
"merge_%s_%s", computeProcessingID(dataSource, segments), new DateTime().toString()
|
||||
),
|
||||
dataSource,
|
||||
computeMergedInterval(segments)
|
||||
);
|
||||
|
@ -181,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)
|
||||
|
@ -270,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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
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.Function;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -52,6 +53,8 @@ public class VersionConverterTask extends AbstractTask
|
|||
private static final Integer CURR_VERSION_INTEGER = new Integer(IndexIO.CURRENT_VERSION_ID);
|
||||
|
||||
private static final Logger log = new Logger(VersionConverterTask.class);
|
||||
|
||||
@JsonIgnore
|
||||
private final DataSegment segment;
|
||||
|
||||
public static VersionConverterTask create(String dataSource, Interval interval)
|
||||
|
@ -172,6 +175,7 @@ public class VersionConverterTask extends AbstractTask
|
|||
|
||||
public static class SubTask extends AbstractTask
|
||||
{
|
||||
@JsonIgnore
|
||||
private final DataSegment segment;
|
||||
|
||||
@JsonCreator
|
||||
|
@ -232,7 +236,8 @@ 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)));
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment)).withAllowOlderVersions(true));
|
||||
} else {
|
||||
log.info("Conversion failed.");
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ import com.metamx.druid.merger.common.TaskLock;
|
|||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig;
|
||||
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import org.joda.time.DateTime;
|
||||
import org.skife.jdbi.v2.DBI;
|
||||
import org.skife.jdbi.v2.Handle;
|
||||
|
@ -52,7 +53,7 @@ public class DbTaskStorage implements TaskStorage
|
|||
private final IndexerDbConnectorConfig dbConnectorConfig;
|
||||
private final DBI dbi;
|
||||
|
||||
private static final Logger log = new Logger(DbTaskStorage.class);
|
||||
private static final EmittingLogger log = new EmittingLogger(DbTaskStorage.class);
|
||||
|
||||
public DbTaskStorage(ObjectMapper jsonMapper, IndexerDbConnectorConfig dbConnectorConfig, DBI dbi)
|
||||
{
|
||||
|
@ -203,18 +204,18 @@ public class DbTaskStorage implements TaskStorage
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<Task> getRunningTasks()
|
||||
public List<String> getRunningTaskIds()
|
||||
{
|
||||
return dbi.withHandle(
|
||||
new HandleCallback<List<Task>>()
|
||||
new HandleCallback<List<String>>()
|
||||
{
|
||||
@Override
|
||||
public List<Task> withHandle(Handle handle) throws Exception
|
||||
public List<String> withHandle(Handle handle) throws Exception
|
||||
{
|
||||
final List<Map<String, Object>> dbTasks =
|
||||
handle.createQuery(
|
||||
String.format(
|
||||
"SELECT payload FROM %s WHERE status_code = :status_code",
|
||||
"SELECT id FROM %s WHERE status_code = :status_code",
|
||||
dbConnectorConfig.getTaskTable()
|
||||
)
|
||||
)
|
||||
|
@ -222,16 +223,12 @@ public class DbTaskStorage implements TaskStorage
|
|||
.list();
|
||||
|
||||
return Lists.transform(
|
||||
dbTasks, new Function<Map<String, Object>, Task>()
|
||||
dbTasks, new Function<Map<String, Object>, String>()
|
||||
{
|
||||
@Override
|
||||
public Task apply(Map<String, Object> row)
|
||||
public String apply(Map<String, Object> row)
|
||||
{
|
||||
try {
|
||||
return jsonMapper.readValue(row.get("payload").toString(), Task.class);
|
||||
} catch(Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
return row.get("id").toString();
|
||||
}
|
||||
}
|
||||
);
|
||||
|
|
|
@ -128,15 +128,15 @@ public class HeapMemoryTaskStorage implements TaskStorage
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<Task> getRunningTasks()
|
||||
public List<String> getRunningTaskIds()
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
final ImmutableList.Builder<Task> listBuilder = ImmutableList.builder();
|
||||
final ImmutableList.Builder<String> listBuilder = ImmutableList.builder();
|
||||
for(final TaskStuff taskStuff : tasks.values()) {
|
||||
if(taskStuff.getStatus().isRunnable()) {
|
||||
listBuilder.add(taskStuff.getTask());
|
||||
listBuilder.add(taskStuff.getTask().getId());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package com.metamx.druid.merger.coordinator;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Throwables;
|
||||
|
@ -28,8 +29,8 @@ import com.google.common.collect.Multimap;
|
|||
import com.google.common.collect.Ordering;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
||||
|
@ -89,33 +90,47 @@ public class TaskQueue
|
|||
queue.clear();
|
||||
taskLockbox.clear();
|
||||
|
||||
// Add running tasks to the queue
|
||||
final List<Task> runningTasks = taskStorage.getRunningTasks();
|
||||
|
||||
for(final Task task : runningTasks) {
|
||||
queue.add(task);
|
||||
}
|
||||
|
||||
// Get all locks, along with which tasks they belong to
|
||||
// Get all running tasks and their locks
|
||||
final Multimap<TaskLock, Task> tasksByLock = ArrayListMultimap.create();
|
||||
for(final Task runningTask : runningTasks) {
|
||||
for(final TaskLock taskLock : taskStorage.getLocks(runningTask.getId())) {
|
||||
tasksByLock.put(taskLock, runningTask);
|
||||
|
||||
for (final String taskId : taskStorage.getRunningTaskIds()) {
|
||||
try {
|
||||
// .get since TaskStorage semantics should mean this task is always found
|
||||
final Task task = taskStorage.getTask(taskId).get();
|
||||
final List<TaskLock> taskLocks = taskStorage.getLocks(task.getId());
|
||||
|
||||
queue.add(task);
|
||||
|
||||
for (final TaskLock taskLock : taskLocks) {
|
||||
tasksByLock.put(taskLock, task);
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert("Failed to bootstrap task").addData("task", taskId).emit();
|
||||
|
||||
// A bit goofy to special-case JsonProcessingException, but we don't want to suppress bootstrap problems on
|
||||
// any old Exception or even IOException...
|
||||
if (e instanceof JsonProcessingException || e.getCause() instanceof JsonProcessingException) {
|
||||
// Mark this task a failure, and continue bootstrapping
|
||||
taskStorage.setStatus(TaskStatus.failure(taskId));
|
||||
} else {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// 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();
|
||||
|
||||
|
@ -150,7 +165,7 @@ public class TaskQueue
|
|||
}
|
||||
}
|
||||
|
||||
log.info("Bootstrapped %,d tasks. Ready to go!", runningTasks.size());
|
||||
log.info("Bootstrapped %,d tasks with %,d locks. Ready to go!", queue.size(), tasksByLock.keySet().size());
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
|
@ -214,7 +229,7 @@ public class TaskQueue
|
|||
// insert the task into our queue.
|
||||
try {
|
||||
taskStorage.insert(task, TaskStatus.running(task.getId()));
|
||||
} catch(TaskExistsException e) {
|
||||
} catch (TaskExistsException e) {
|
||||
log.warn("Attempt to add task twice: %s", task.getId());
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -20,9 +20,9 @@
|
|||
package com.metamx.druid.merger.coordinator;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.actions.TaskAction;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -77,9 +77,9 @@ public interface TaskStorage
|
|||
public List<TaskAction> getAuditLogs(String taskid);
|
||||
|
||||
/**
|
||||
* Returns a list of currently-running tasks as stored in the storage facility, in no particular order.
|
||||
* Returns a list of currently-running task IDs as stored in the storage facility, in no particular order.
|
||||
*/
|
||||
public List<Task> getRunningTasks();
|
||||
public List<String> getRunningTaskIds();
|
||||
|
||||
/**
|
||||
* Returns a list of locks for a particular task.
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,7 +35,6 @@ import java.util.List;
|
|||
@JsonTypeName("test")
|
||||
public class TestTask extends MergeTask
|
||||
{
|
||||
private final String id;
|
||||
private final TaskStatus status;
|
||||
|
||||
@JsonCreator
|
||||
|
@ -47,19 +46,10 @@ public class TestTask extends MergeTask
|
|||
@JsonProperty("taskStatus") TaskStatus status
|
||||
)
|
||||
{
|
||||
super(dataSource, segments, aggregators);
|
||||
|
||||
this.id = id;
|
||||
super(id, dataSource, segments, aggregators);
|
||||
this.status = status;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getId()
|
||||
{
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getType()
|
||||
|
|
|
@ -43,7 +43,7 @@ public class MergeTaskBaseTest
|
|||
.add(segmentBuilder.interval(new Interval("2012-01-03/2012-01-05")).build())
|
||||
.build();
|
||||
|
||||
final MergeTaskBase testMergeTaskBase = new MergeTaskBase("foo", segments)
|
||||
final MergeTaskBase testMergeTaskBase = new MergeTaskBase(null, "foo", segments)
|
||||
{
|
||||
@Override
|
||||
protected File merge(Map<DataSegment, File> segments, File outDir) throws Exception
|
||||
|
|
|
@ -5,19 +5,23 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.metamx.common.Granularity;
|
||||
import com.metamx.druid.QueryGranularity;
|
||||
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
|
||||
|
@ -26,6 +30,7 @@ public class TaskSerdeTest
|
|||
public void testIndexTaskSerde() throws Exception
|
||||
{
|
||||
final Task task = new IndexTask(
|
||||
null,
|
||||
"foo",
|
||||
new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P2D"))),
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
|
@ -54,6 +59,7 @@ public class TaskSerdeTest
|
|||
public void testIndexGeneratorTaskSerde() throws Exception
|
||||
{
|
||||
final Task task = new IndexGeneratorTask(
|
||||
null,
|
||||
"foo",
|
||||
new Interval("2010-01-01/P1D"),
|
||||
null,
|
||||
|
@ -68,6 +74,8 @@ public class TaskSerdeTest
|
|||
|
||||
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());
|
||||
|
@ -80,17 +88,23 @@ public class TaskSerdeTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testAppendTaskSerde() throws Exception
|
||||
public void testMergeTaskSerde() throws Exception
|
||||
{
|
||||
final Task task = new AppendTask(
|
||||
final Task task = new MergeTask(
|
||||
null,
|
||||
"foo",
|
||||
ImmutableList.<DataSegment>of(
|
||||
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
|
||||
),
|
||||
ImmutableList.<AggregatorFactory>of(
|
||||
new CountAggregatorFactory("cnt")
|
||||
)
|
||||
);
|
||||
|
||||
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());
|
||||
|
@ -100,20 +114,165 @@ public class TaskSerdeTest
|
|||
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
|
||||
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
|
||||
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
|
||||
Assert.assertEquals(((MergeTask) task).getSegments(), ((MergeTask) task2).getSegments());
|
||||
Assert.assertEquals(
|
||||
((MergeTask) task).getAggregators().get(0).getName(),
|
||||
((MergeTask) task2).getAggregators().get(0).getName()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeleteTaskSerde() throws Exception
|
||||
public void testKillTaskSerde() throws Exception
|
||||
{
|
||||
final Task task = new DeleteTask(
|
||||
final Task task = new KillTask(
|
||||
null,
|
||||
"foo",
|
||||
new Interval("2010-01-01/P1D")
|
||||
);
|
||||
|
||||
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.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
|
||||
|
||||
Assert.assertEquals(task.getId(), task2.getId());
|
||||
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
|
||||
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
|
||||
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testVersionConverterTaskSerde() throws Exception
|
||||
{
|
||||
final Task task = VersionConverterTask.create(
|
||||
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
|
||||
);
|
||||
|
||||
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.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
|
||||
|
||||
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(((VersionConverterTask) task).getSegment(), ((VersionConverterTask) task).getSegment());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testVersionConverterSubTaskSerde() throws Exception
|
||||
{
|
||||
final Task task = new VersionConverterTask.SubTask(
|
||||
"myGroupId",
|
||||
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
|
||||
);
|
||||
|
||||
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.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
|
||||
Assert.assertEquals("myGroupId", task.getGroupId());
|
||||
|
||||
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(
|
||||
((VersionConverterTask.SubTask) task).getSegment(),
|
||||
((VersionConverterTask.SubTask) task).getSegment()
|
||||
);
|
||||
}
|
||||
|
||||
@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
|
||||
{
|
||||
final Task task = new DeleteTask(
|
||||
null,
|
||||
"foo",
|
||||
new Interval("2010-01-01/P1D")
|
||||
);
|
||||
|
||||
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.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
|
||||
|
||||
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(task.getImplicitLockInterval().get(), task2.getImplicitLockInterval().get());
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testDeleteTaskFromJson() throws Exception
|
||||
{
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
final Task task = jsonMapper.readValue(
|
||||
"{\"type\":\"delete\",\"dataSource\":\"foo\",\"interval\":\"2010-01-01/P1D\"}",
|
||||
Task.class
|
||||
);
|
||||
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.assertNotNull(task.getId());
|
||||
Assert.assertEquals("foo", task.getDataSource());
|
||||
Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
|
||||
|
||||
Assert.assertEquals(task.getId(), task2.getId());
|
||||
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
|
||||
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
|
||||
|
@ -121,10 +280,39 @@ public class TaskSerdeTest
|
|||
Assert.assertEquals(task.getImplicitLockInterval().get(), task2.getImplicitLockInterval().get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppendTaskSerde() throws Exception
|
||||
{
|
||||
final Task task = new AppendTask(
|
||||
null,
|
||||
"foo",
|
||||
ImmutableList.of(
|
||||
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
|
||||
)
|
||||
);
|
||||
|
||||
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.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
|
||||
|
||||
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(task.getImplicitLockInterval().get(), task2.getImplicitLockInterval().get());
|
||||
Assert.assertEquals(((AppendTask) task).getSegments(), ((AppendTask) task2).getSegments());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHadoopIndexTaskSerde() throws Exception
|
||||
{
|
||||
final HadoopIndexTask task = new HadoopIndexTask(
|
||||
null,
|
||||
new HadoopDruidIndexerConfig(
|
||||
null,
|
||||
"foo",
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
|
||||
|
@ -184,6 +186,7 @@ public class TaskLifecycleTest
|
|||
public void testIndexTask() throws Exception
|
||||
{
|
||||
final Task indexTask = new IndexTask(
|
||||
null,
|
||||
"foo",
|
||||
new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P2D"))),
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
|
@ -226,6 +229,7 @@ public class TaskLifecycleTest
|
|||
public void testIndexTaskFailure() throws Exception
|
||||
{
|
||||
final Task indexTask = new IndexTask(
|
||||
null,
|
||||
"foo",
|
||||
new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P1D"))),
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
|
@ -249,7 +253,7 @@ public class TaskLifecycleTest
|
|||
{
|
||||
// This test doesn't actually do anything right now. We should actually put things into the Mocked coordinator
|
||||
// Such that this test can test things...
|
||||
final Task killTask = new KillTask("foo", new Interval("2010-01-02/P2D"));
|
||||
final Task killTask = new KillTask(null, "foo", new Interval("2010-01-02/P2D"));
|
||||
|
||||
final TaskStatus status = runTask(killTask);
|
||||
Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
|
||||
|
@ -282,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()
|
||||
|
@ -307,7 +309,7 @@ public class TaskLifecycleTest
|
|||
DataSegment.builder()
|
||||
.dataSource("foo")
|
||||
.interval(interval1)
|
||||
.version(lock1.get().getVersion())
|
||||
.version(lock1.getVersion())
|
||||
.build()
|
||||
)
|
||||
)
|
||||
|
@ -318,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()
|
||||
|
@ -328,7 +330,7 @@ public class TaskLifecycleTest
|
|||
DataSegment.builder()
|
||||
.dataSource("foo")
|
||||
.interval(interval2)
|
||||
.version(lock2.get().getVersion())
|
||||
.version(lock2.getVersion())
|
||||
.build()
|
||||
)
|
||||
)
|
||||
|
@ -392,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"))
|
||||
{
|
||||
|
@ -426,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"))
|
||||
{
|
||||
|
@ -506,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()
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -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();
|
||||
}
|
|
@ -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;
|
||||
|
||||
/**
|
||||
*/
|
|
@ -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.");
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -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(
|
|
@ -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);
|
||||
}
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.Lists;
|
|||
import com.metamx.druid.query.filter.AndDimFilter;
|
||||
import com.metamx.druid.query.filter.DimFilter;
|
||||
import com.metamx.druid.query.filter.ExtractionDimFilter;
|
||||
import com.metamx.druid.query.filter.JavaScriptDimFilter;
|
||||
import com.metamx.druid.query.filter.NotDimFilter;
|
||||
import com.metamx.druid.query.filter.OrDimFilter;
|
||||
import com.metamx.druid.query.filter.RegexDimFilter;
|
||||
|
@ -84,6 +85,10 @@ public class Filters
|
|||
final SearchQueryDimFilter searchQueryFilter = (SearchQueryDimFilter) dimFilter;
|
||||
|
||||
filter = new SearchQueryFilter(searchQueryFilter.getDimension(), searchQueryFilter.getQuery());
|
||||
} else if (dimFilter instanceof JavaScriptDimFilter) {
|
||||
final JavaScriptDimFilter javaScriptDimFilter = (JavaScriptDimFilter) dimFilter;
|
||||
|
||||
filter = new JavaScriptFilter(javaScriptDimFilter.getDimension(), javaScriptDimFilter.getFunction());
|
||||
}
|
||||
|
||||
return filter;
|
||||
|
|
|
@ -0,0 +1,127 @@
|
|||
package com.metamx.druid.index.brita;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||
import org.mozilla.javascript.Context;
|
||||
import org.mozilla.javascript.Function;
|
||||
import org.mozilla.javascript.ScriptableObject;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
|
||||
public class JavaScriptFilter implements Filter
|
||||
{
|
||||
private final JavaScriptPredicate predicate;
|
||||
private final String dimension;
|
||||
|
||||
public JavaScriptFilter(String dimension, final String script)
|
||||
{
|
||||
this.dimension = dimension;
|
||||
this.predicate = new JavaScriptPredicate(script);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableConciseSet goConcise(final BitmapIndexSelector selector)
|
||||
{
|
||||
final Context cx = Context.enter();
|
||||
try {
|
||||
ImmutableConciseSet conciseSet = ImmutableConciseSet.union(
|
||||
FunctionalIterable.create(selector.getDimensionValues(dimension))
|
||||
.filter(new Predicate<String>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(@Nullable String input)
|
||||
{
|
||||
return predicate.applyInContext(cx, input);
|
||||
}
|
||||
})
|
||||
.transform(
|
||||
new com.google.common.base.Function<String, ImmutableConciseSet>()
|
||||
{
|
||||
@Override
|
||||
public ImmutableConciseSet apply(@Nullable String input)
|
||||
{
|
||||
return selector.getConciseInvertedIndex(dimension, input);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
return conciseSet;
|
||||
} finally {
|
||||
Context.exit();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeMatcher(ValueMatcherFactory factory)
|
||||
{
|
||||
// suboptimal, since we need create one context per call to predicate.apply()
|
||||
return factory.makeValueMatcher(dimension, predicate);
|
||||
}
|
||||
|
||||
static class JavaScriptPredicate implements Predicate<String> {
|
||||
final ScriptableObject scope;
|
||||
final Function fnApply;
|
||||
final String script;
|
||||
|
||||
public JavaScriptPredicate(final String script) {
|
||||
Preconditions.checkNotNull(script, "script must not be null");
|
||||
this.script = script;
|
||||
|
||||
final Context cx = Context.enter();
|
||||
try {
|
||||
cx.setOptimizationLevel(9);
|
||||
scope = cx.initStandardObjects();
|
||||
|
||||
fnApply = cx.compileFunction(scope, script, "script", 1, null);
|
||||
} finally {
|
||||
Context.exit();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean apply(final String input)
|
||||
{
|
||||
// one and only one context per thread
|
||||
final Context cx = Context.enter();
|
||||
try {
|
||||
return applyInContext(cx, input);
|
||||
} finally {
|
||||
Context.exit();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public boolean applyInContext(Context cx, String input)
|
||||
{
|
||||
return Context.toBoolean(fnApply.call(cx, scope, scope, new String[]{input}));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
JavaScriptPredicate that = (JavaScriptPredicate) o;
|
||||
|
||||
if (!script.equals(that.script)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return script.hashCode();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue