From 21613bc73be1cc7bf1b3564e9f1a3bc15152cde9 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 16 Jan 2013 17:31:01 -0800 Subject: [PATCH 01/63] initial commit to hard delete segments --- .../metamx/druid/merge/ClientKillQuery.java | 41 +++++ .../druid/merger/common/TaskToolbox.java | 9 ++ .../druid/merger/common/task/KillTask.java | 52 +++++++ .../metamx/druid/merger/common/task/Task.java | 4 +- .../http/IndexerCoordinatorNode.java | 27 +++- .../druid/merger/worker/http/WorkerNode.java | 15 +- .../coordinator/RemoteTaskRunnerTest.java | 2 +- .../druid/db/DatabaseSegmentManager.java | 40 +++++ .../com/metamx/druid/http/InfoResource.java | 6 +- .../com/metamx/druid/http/MasterMain.java | 4 +- .../com/metamx/druid/http/MasterResource.java | 11 +- .../metamx/druid/loading/S3SegmentKiller.java | 144 ++++++++++++++++++ .../metamx/druid/loading/SegmentKiller.java | 14 ++ .../com/metamx/druid/master/DruidMaster.java | 47 +++++- .../src/main/resources/static/css/enable.css | 3 + server/src/main/resources/static/enable.html | 75 +++++++++ server/src/main/resources/static/index.html | 25 ++- .../main/resources/static/js/enable-0.0.1.js | 97 ++++++++++++ .../main/resources/static/js/kill-0.0.1.js | 58 +++++++ server/src/main/resources/static/kill.html | 61 ++++++++ 20 files changed, 713 insertions(+), 22 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java create mode 100644 server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java create mode 100644 server/src/main/java/com/metamx/druid/loading/SegmentKiller.java create mode 100644 server/src/main/resources/static/css/enable.css create mode 100644 server/src/main/resources/static/enable.html create mode 100644 server/src/main/resources/static/js/enable-0.0.1.js create mode 100644 server/src/main/resources/static/js/kill-0.0.1.js create mode 100644 server/src/main/resources/static/kill.html diff --git a/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java new file mode 100644 index 00000000000..ae39aeb512b --- /dev/null +++ b/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java @@ -0,0 +1,41 @@ +package com.metamx.druid.merge; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.joda.time.Interval; + +/** + */ +public class ClientKillQuery +{ + private final String dataSource; + private final Interval interval; + + @JsonCreator + public ClientKillQuery( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval + ) + { + this.dataSource = dataSource; + this.interval = interval; + } + + @JsonProperty + public String getType() + { + return "kill"; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index 74a546cf696..26336b45c25 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.metamx.druid.loading.S3SegmentPuller; import com.metamx.druid.loading.S3SegmentGetterConfig; import com.metamx.druid.loading.S3ZippedSegmentPuller; +import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.loading.SegmentPuller; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; @@ -43,6 +44,7 @@ public class TaskToolbox private final ServiceEmitter emitter; private final RestS3Service s3Client; private final SegmentPusher segmentPusher; + private final SegmentKiller segmentKiller; private final ObjectMapper objectMapper; public TaskToolbox( @@ -50,6 +52,7 @@ public class TaskToolbox ServiceEmitter emitter, RestS3Service s3Client, SegmentPusher segmentPusher, + SegmentKiller segmentKiller, ObjectMapper objectMapper ) { @@ -57,6 +60,7 @@ public class TaskToolbox this.emitter = emitter; this.s3Client = s3Client; this.segmentPusher = segmentPusher; + this.segmentKiller = segmentKiller; this.objectMapper = objectMapper; } @@ -80,6 +84,11 @@ public class TaskToolbox return segmentPusher; } + public SegmentKiller getSegmentKiller() + { + return segmentKiller; + } + public ObjectMapper getObjectMapper() { return objectMapper; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java new file mode 100644 index 00000000000..5b2d415eb47 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -0,0 +1,52 @@ +package com.metamx.druid.merger.common.task; + +import com.google.common.collect.Lists; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.coordinator.TaskContext; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +/** + */ +public class KillTask extends AbstractTask +{ + private static final Logger log = new Logger(KillTask.class); + + @JsonCreator + public KillTask( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval + ) + { + super( + String.format( + "kill_%s_%s_%s_%s", + dataSource, + interval.getStart(), + interval.getEnd(), + new DateTime().toString() + ), + dataSource, + interval + ); + } + + @Override + public Type getType() + { + return Task.Type.KILL; + } + + @Override + public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + { + // Kill segments + toolbox.getSegmentKiller().kill(getDataSource(), getInterval()); + return TaskStatus.success(getId(), Lists.newArrayList()); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index b2059210b58..961afac96ce 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -37,6 +37,7 @@ import org.joda.time.Interval; @JsonSubTypes(value = { @JsonSubTypes.Type(name = "append", value = AppendTask.class), @JsonSubTypes.Type(name = "delete", value = DeleteTask.class), + @JsonSubTypes.Type(name = "kill", value = KillTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), @JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class), @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class) @@ -49,7 +50,8 @@ public interface Task MERGE, APPEND, DELETE, - TEST + TEST, + KILL } public String getId(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index bc48c7c71de..d77208c09e2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -47,6 +47,8 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.loading.S3SegmentKiller; +import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -98,6 +100,7 @@ import org.mortbay.jetty.servlet.DefaultServlet; import org.mortbay.jetty.servlet.FilterHolder; import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; +import org.skife.jdbi.v2.DBI; import java.net.URL; import java.util.Arrays; @@ -125,6 +128,8 @@ public class IndexerCoordinatorNode extends RegisteringNode private List monitors = null; private ServiceEmitter emitter = null; + private DbConnectorConfig dbConnectorConfig = null; + private DBI dbi = null; private IndexerCoordinatorConfig config = null; private TaskToolbox taskToolbox = null; private MergerDBCoordinator mergerDBCoordinator = null; @@ -193,6 +198,7 @@ public class IndexerCoordinatorNode extends RegisteringNode initializeEmitter(); initializeMonitors(); + initializeDB(); initializeIndexerCoordinatorConfig(); initializeMergeDBCoordinator(); initializeTaskToolbox(); @@ -370,6 +376,16 @@ public class IndexerCoordinatorNode extends RegisteringNode } } + private void initializeDB() + { + if (dbConnectorConfig == null) { + dbConnectorConfig = configFactory.build(DbConnectorConfig.class); + } + if (dbi == null) { + dbi = new DbConnector(dbConnectorConfig).getDBI(); + } + } + private void initializeIndexerCoordinatorConfig() { if (config == null) { @@ -391,18 +407,23 @@ public class IndexerCoordinatorNode extends RegisteringNode configFactory.build(S3SegmentPusherConfig.class), jsonMapper ); - taskToolbox = new TaskToolbox(config, emitter, s3Client, segmentPusher, jsonMapper); + final SegmentKiller segmentKiller = new S3SegmentKiller( + s3Client, + dbi, + dbConnectorConfig, + jsonMapper + ); + taskToolbox = new TaskToolbox(config, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper); } } public void initializeMergeDBCoordinator() { if (mergerDBCoordinator == null) { - final DbConnectorConfig dbConnectorConfig = configFactory.build(DbConnectorConfig.class); mergerDBCoordinator = new MergerDBCoordinator( jsonMapper, dbConnectorConfig, - new DbConnector(dbConnectorConfig).getDBI() + dbi ); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 0799a8de37c..65400f209d9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -29,11 +29,15 @@ 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.db.DbConnector; +import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.loading.S3SegmentKiller; +import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -69,6 +73,7 @@ import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.DefaultServlet; import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; +import org.skife.jdbi.v2.DBI; import java.io.IOException; import java.util.Arrays; @@ -293,7 +298,15 @@ public class WorkerNode extends RegisteringNode configFactory.build(S3SegmentPusherConfig.class), jsonMapper ); - taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, segmentPusher, jsonMapper); + final DbConnectorConfig dbConnectorConfig = configFactory.build(DbConnectorConfig.class); + DBI dbi = new DbConnector(dbConnectorConfig).getDBI(); + final SegmentKiller segmentKiller = new S3SegmentKiller( + s3Client, + dbi, + dbConnectorConfig, + jsonMapper + ); + taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper); } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index eb10731abd9..e4996965fbd 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -322,7 +322,7 @@ public class RemoteTaskRunnerTest { return null; } - }, null, null, null, jsonMapper + }, null, null, null, null, jsonMapper ), Executors.newSingleThreadExecutor() ); diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index ef620cd1e38..e49728baea4 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; +import com.metamx.common.MapUtils; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; @@ -45,6 +46,7 @@ import org.skife.jdbi.v2.tweak.HandleCallback; import javax.annotation.Nullable; import java.sql.SQLException; +import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; @@ -343,6 +345,44 @@ public class DatabaseSegmentManager return dataSources.get().values(); } + public Collection getAllDatasourceNames() + { + synchronized (lock) { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) throws Exception + { + return handle.createQuery( + String.format("SELECT DISTINCT(datasource) FROM %s", config.getSegmentTable()) + ) + .fold( + Lists.newArrayList(), + new Folder3, Map>() + { + @Override + public ArrayList fold( + ArrayList druidDataSources, + Map stringObjectMap, + FoldController foldController, + StatementContext statementContext + ) throws SQLException + { + druidDataSources.add( + MapUtils.getString(stringObjectMap, "datasource") + ); + return druidDataSources; + } + } + ); + + } + } + ); + } + } + public void poll() { try { diff --git a/server/src/main/java/com/metamx/druid/http/InfoResource.java b/server/src/main/java/com/metamx/druid/http/InfoResource.java index 800e3a93e46..090e311ad31 100644 --- a/server/src/main/java/com/metamx/druid/http/InfoResource.java +++ b/server/src/main/java/com/metamx/druid/http/InfoResource.java @@ -548,10 +548,14 @@ public class InfoResource @Path("/db/datasources") @Produces("application/json") public Response getDatabaseDataSources( - @QueryParam("full") String full + @QueryParam("full") String full, + @QueryParam("includeDisabled") String includeDisabled ) { Response.ResponseBuilder builder = Response.status(Response.Status.OK); + if (includeDisabled != null) { + return builder.entity(databaseSegmentManager.getAllDatasourceNames()).build(); + } if (full != null) { return builder.entity(databaseSegmentManager.getInventory()).build(); } diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index 51df502fa64..f65e6e2b57c 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -190,7 +190,9 @@ public class MasterMain emitter, scheduledExecutorFactory, new ConcurrentHashMap(), - serviceProvider + serviceProvider, + httpClient, + new ToStringResponseHandler(Charsets.UTF_8) ); lifecycle.addManagedInstance(master); diff --git a/server/src/main/java/com/metamx/druid/http/MasterResource.java b/server/src/main/java/com/metamx/druid/http/MasterResource.java index ca992815eb6..c73e74e528c 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterResource.java +++ b/server/src/main/java/com/metamx/druid/http/MasterResource.java @@ -21,13 +21,13 @@ package com.metamx.druid.http; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.LoadPeonCallback; +import com.metamx.druid.merge.ClientKillQuery; import javax.inject.Inject; import javax.ws.rs.Consumes; import javax.ws.rs.GET; import javax.ws.rs.POST; import javax.ws.rs.Path; -import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.core.Response; import java.util.List; @@ -111,6 +111,15 @@ public class MasterResource return resp; } + @POST + @Path("/kill") + @Consumes("application/json") + public Response killSegments(ClientKillQuery killQuery) + { + master.killSegments(killQuery); + return Response.ok().build(); + } + @GET @Path("/loadstatus") @Produces("application/json") diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java new file mode 100644 index 00000000000..346bb036ff3 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java @@ -0,0 +1,144 @@ +package com.metamx.druid.loading; + +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.db.DbConnectorConfig; +import org.codehaus.jackson.map.ObjectMapper; +import org.jets3t.service.ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.joda.time.Interval; +import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.FoldController; +import org.skife.jdbi.v2.Folder3; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.StatementContext; +import org.skife.jdbi.v2.tweak.HandleCallback; + +import java.sql.SQLException; +import java.util.List; +import java.util.Map; + +/** + */ +public class S3SegmentKiller implements SegmentKiller +{ + private static final Logger log = new Logger(S3SegmentKiller.class); + + private final RestS3Service s3Client; + private final DBI dbi; + private final DbConnectorConfig config; + private final ObjectMapper jsonMapper; + + @Inject + public S3SegmentKiller( + RestS3Service s3Client, + DBI dbi, + DbConnectorConfig config, + ObjectMapper jsonMapper + ) + { + this.s3Client = s3Client; + this.dbi = dbi; + this.config = config; + this.jsonMapper = jsonMapper; + } + + + @Override + public List kill(final String datasource, final Interval interval) throws ServiceException + { + List matchingSegments = dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) throws Exception + { + return handle.createQuery( + String.format( + "SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and end <= :end and used = 0", + config.getSegmentTable() + ) + ) + .bind("dataSource", datasource) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .fold( + Lists.newArrayList(), + new Folder3, Map>() + { + @Override + public List fold( + List accumulator, + Map stringObjectMap, + FoldController foldController, + StatementContext statementContext + ) throws SQLException + { + try { + DataSegment segment = jsonMapper.readValue( + (String) stringObjectMap.get("payload"), + DataSegment.class + ); + + accumulator.add(segment); + + return accumulator; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + } + } + ); + + log.info("Found %,d segments for %s for interval %s.", matchingSegments.size(), datasource, interval); + for (final DataSegment segment : matchingSegments) { + // Remove database entry + log.info("Removing DB entry for %s", segment.getIdentifier()); + dbi.withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception + { + handle.createStatement( + String.format("DELETE from %s WHERE id = :segmentID", config.getSegmentTable()) + ).bind("segmentID", segment.getIdentifier()) + .execute(); + + return null; + } + } + ); + + // Remove from S3 + + Map loadSpec = segment.getLoadSpec(); + String s3Bucket = MapUtils.getString(loadSpec, "bucket"); + String s3Path = MapUtils.getString(loadSpec, "key"); + String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; + + if (!s3Client.isObjectInBucket(s3Bucket, s3Path)) { + throw new ISE("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3Path); + } + if (!s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { + throw new ISE("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3DescriptorPath); + } + + log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); + log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); + s3Client.deleteObject(s3Bucket, s3Path); + s3Client.deleteObject(s3Bucket, s3DescriptorPath); + } + + return matchingSegments; + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java new file mode 100644 index 00000000000..56a14d2e933 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java @@ -0,0 +1,14 @@ +package com.metamx.druid.loading; + +import com.metamx.druid.client.DataSegment; +import org.jets3t.service.ServiceException; +import org.joda.time.Interval; + +import java.util.List; + +/** + */ +public interface SegmentKiller +{ + public List kill(String datasource, Interval interval) throws ServiceException; +} diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 80172c432fa..d5d4a8bd978 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -21,7 +21,7 @@ package com.metamx.druid.master; import com.google.common.base.Function; import com.google.common.base.Predicate; -import com.google.common.collect.Collections2; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -36,7 +36,6 @@ import com.metamx.common.guava.Comparators; import com.metamx.common.guava.FunctionalIterable; 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.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; @@ -44,9 +43,12 @@ import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; +import com.metamx.druid.merge.ClientKillQuery; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.HttpResponseHandler; import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; import com.netflix.curator.x.discovery.ServiceProvider; @@ -55,8 +57,8 @@ import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.Duration; import javax.annotation.Nullable; +import java.net.URL; import java.util.Arrays; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; @@ -89,6 +91,8 @@ public class DruidMaster private final PhoneBookPeon masterPeon; private final Map loadManagementPeons; private final ServiceProvider serviceProvider; + private final HttpClient httpClient; + private final HttpResponseHandler responseHandler; private final ObjectMapper jsonMapper; @@ -103,7 +107,9 @@ public class DruidMaster ServiceEmitter emitter, ScheduledExecutorFactory scheduledExecutorFactory, Map loadManagementPeons, - ServiceProvider serviceProvider + ServiceProvider serviceProvider, + HttpClient httpClient, + HttpResponseHandler responseHandler ) { this.config = config; @@ -124,6 +130,9 @@ public class DruidMaster this.loadManagementPeons = loadManagementPeons; this.serviceProvider = serviceProvider; + + this.httpClient = httpClient; + this.responseHandler = responseHandler; } public boolean isClusterMaster() @@ -199,6 +208,27 @@ public class DruidMaster databaseSegmentManager.enableDatasource(ds); } + public void killSegments(ClientKillQuery killQuery) + { + try { + httpClient.post( + new URL( + String.format( + "http://%s:%s/mmx/merger/v1/index", + serviceProvider.getInstance().getAddress(), + serviceProvider.getInstance().getPort() + ) + ) + ) + .setContent("application/json", jsonMapper.writeValueAsBytes(killQuery)) + .go(responseHandler) + .get(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + public void moveSegment(String from, String to, String segmentName, final LoadPeonCallback callback) { final DruidServer fromServer = serverInventoryManager.getInventoryValue(from); @@ -688,7 +718,14 @@ public class DruidMaster super( ImmutableList.of( new DruidMasterSegmentInfoLoader(DruidMaster.this), - new DruidMasterSegmentMerger(jsonMapper, serviceProvider), + new DruidMasterSegmentMerger( + new HttpMergerClient( + httpClient, + responseHandler, + jsonMapper, + serviceProvider + ) + ), new DruidMasterHelper() { @Override diff --git a/server/src/main/resources/static/css/enable.css b/server/src/main/resources/static/css/enable.css new file mode 100644 index 00000000000..4c1a4cdb9b1 --- /dev/null +++ b/server/src/main/resources/static/css/enable.css @@ -0,0 +1,3 @@ +#select_datasource { + margin: 20px 0 20px 0; +} \ No newline at end of file diff --git a/server/src/main/resources/static/enable.html b/server/src/main/resources/static/enable.html new file mode 100644 index 00000000000..265ae7780df --- /dev/null +++ b/server/src/main/resources/static/enable.html @@ -0,0 +1,75 @@ + + + + + + Druid Master Console - Enable/Disable Datasources + + + + + + + + + + + + + + +
+ +
Enable/Disable Datasources
+ +
+

Enabled Datasources:

+
    +
+
+ +
+

Disabled Datasources:

+
    +
+
+ +
+ Select Data Source: + +
+ +
+ +
+

Are you sure you want to enable the selected datasource?

+
+ +
+

Are you sure you want to disable the selected datasource?

+
+ +
+
+
+ + \ No newline at end of file diff --git a/server/src/main/resources/static/index.html b/server/src/main/resources/static/index.html index bd36d0c5c53..8a6a68a7faa 100644 --- a/server/src/main/resources/static/index.html +++ b/server/src/main/resources/static/index.html @@ -20,20 +20,29 @@ - Druid Master Console - - - + Druid Master Console + + + - + \ No newline at end of file diff --git a/server/src/main/resources/static/js/enable-0.0.1.js b/server/src/main/resources/static/js/enable-0.0.1.js new file mode 100644 index 00000000000..f25b1a53a54 --- /dev/null +++ b/server/src/main/resources/static/js/enable-0.0.1.js @@ -0,0 +1,97 @@ +$(document).ready(function() { + $("button").button(); + + $("#error_dialog").dialog({ + autoOpen: false, + modal:true, + resizeable: false, + buttons: { + Ok : function() { + $(this).dialog("close"); + } + } + }); + + $("#enable_dialog").dialog({ + autoOpen: false, + modal:true, + resizeable: false, + buttons: { + Yes : function() { + var selected = $('#datasources option:selected').text(); + $.ajax({ + type: 'POST', + url:'/info/datasources/' + selected, + data: JSON.stringify(selected), + contentType:"application/json; charset=utf-8", + dataType:"json", + error: function(xhr, status, error) { + $("#enable_dialog").dialog("close"); + $("#error_dialog").html(xhr.responseText); + $("#error_dialog").dialog("open"); + }, + success: function(data, status, xhr) { + $("#enable_dialog").dialog("close"); + } + }); + }, + Cancel: function() { + $(this).dialog("close"); + } + } + }); + + $("#disable_dialog").dialog({ + autoOpen: false, + modal:true, + resizeable: false, + buttons: { + Yes : function() { + var selected = $('#datasources option:selected').text(); + $.ajax({ + type: 'DELETE', + url:'/info/datasources/' + selected, + data: JSON.stringify(selected), + contentType:"application/json; charset=utf-8", + dataType:"json", + error: function(xhr, status, error) { + $("#disable_dialog").dialog("close"); + $("#error_dialog").html(xhr.responseText); + $("#error_dialog").dialog("open"); + }, + success: function(data, status, xhr) { + $("#disable_dialog").dialog("close"); + } + }); + }, + Cancel: function() { + $(this).dialog("close"); + } + } + }); + + $.getJSON("/info/db/datasources", function(enabled_datasources) { + $.each(enabled_datasources, function(index, datasource) { + $('#enabled_datasources').append($('
  • ' + datasource + '
  • ')); + }); + + $.getJSON("/info/db/datasources?includeDisabled", function(db_datasources) { + var disabled_datasources = _.difference(db_datasources, enabled_datasources); + $.each(disabled_datasources, function(index, datasource) { + $('#disabled_datasources').append($('
  • ' + datasource + '
  • ')); + }); + $.each(db_datasources, function(index, datasource) { + $('#datasources').append($('').attr("value", datasource).text(datasource)); + }); + }); + }); + + + $("#enable").click(function() { + $("#enable_dialog").dialog("open"); + }); + + $('#disable').click(function (){ + $("#disable_dialog").dialog("open") + }); +}); \ No newline at end of file diff --git a/server/src/main/resources/static/js/kill-0.0.1.js b/server/src/main/resources/static/js/kill-0.0.1.js new file mode 100644 index 00000000000..495c6c47f2d --- /dev/null +++ b/server/src/main/resources/static/js/kill-0.0.1.js @@ -0,0 +1,58 @@ +$(document).ready(function() { + $("button").button(); + + $("#error_dialog").dialog({ + autoOpen: false, + modal:true, + resizeable: false, + buttons: { + Ok : function() { + $(this).dialog("close"); + } + } + }); + + $("#confirm_dialog").dialog({ + autoOpen: false, + modal:true, + resizeable: false, + buttons: { + Yes : function() { + var selected = $('#datasources option:selected').text(); + var interval = $('#interval').val(); + var toSend = { + "dataSource" : selected, + "interval" : interval + } + $.ajax({ + type: 'POST', + url:'/master/kill', + data: JSON.stringify(toSend), + contentType:"application/json; charset=utf-8", + dataType:"json", + error: function(xhr, status, error) { + $("#confirm_dialog").dialog("close"); + $("#error_dialog").html(xhr.responseText); + $("#error_dialog").dialog("open"); + }, + success: function(data, status, xhr) { + $("#confirm_dialog").dialog("close"); + } + }); + }, + Cancel: function() { + $(this).dialog("close"); + } + } + }); + + $.getJSON("/info/db/datasources?includeDisabled", function(data) { + $.each(data, function(index, datasource) { + $('#datasources').append($('').attr("value", datasource).text(datasource)); + }); + }); + + $("#confirm").click(function() { + $("#confirm_dialog").dialog("open"); + }); +}); \ No newline at end of file diff --git a/server/src/main/resources/static/kill.html b/server/src/main/resources/static/kill.html new file mode 100644 index 00000000000..8741fd25f41 --- /dev/null +++ b/server/src/main/resources/static/kill.html @@ -0,0 +1,61 @@ + + + + + + Druid Master Console - Enable/Disable Datasources + + + + + + + + + + + + + +
    + +
    Permanently Delete Segments
    + +
    + Select Data Source: + +
    + +

    Interval:

    + + +
    + +
    +

    Are you sure you want delete segments for this datasource and range? There is no going back!

    +
    + +
    +
    +
    + + \ No newline at end of file From 272d737517baecf281353894d8780e60685fe831 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Tue, 22 Jan 2013 16:21:38 -0800 Subject: [PATCH 02/63] cleaning up some interactions with RTR and workers --- .../druid/client/ServerInventoryManager.java | 4 +- .../{coordinator => common}/TaskCallback.java | 2 +- .../druid/merger/common/TaskStatus.java | 62 ++++++++-- .../druid/merger/common/task/DeleteTask.java | 9 +- .../task/IndexDeterminePartitionsTask.java | 72 +++++------ .../common/task/IndexGeneratorTask.java | 28 +++-- .../druid/merger/common/task/IndexTask.java | 5 +- .../druid/merger/common/task/KillTask.java | 9 +- .../druid/merger/common/task/MergeTask.java | 12 +- .../metamx/druid/merger/common/task/Task.java | 7 +- .../merger/coordinator/LocalTaskRunner.java | 3 +- .../coordinator/MergerDBCoordinator.java | 20 ++++ .../merger/coordinator/RemoteTaskRunner.java | 37 +++--- .../druid/merger/coordinator/TaskQueue.java | 62 ++++++---- .../druid/merger/coordinator/TaskRunner.java | 1 + .../druid/merger/coordinator/TaskWrapper.java | 1 + .../merger/coordinator/exec/TaskConsumer.java | 113 ++++++++++++------ .../druid/merger/worker/TaskMonitor.java | 10 +- .../coordinator/RemoteTaskRunnerTest.java | 8 +- .../merger/coordinator/TaskQueueTest.java | 71 ++++++----- .../metamx/druid/db/DatabaseRuleManager.java | 6 +- .../metamx/druid/loading/S3SegmentKiller.java | 34 +----- .../com/metamx/druid/master/DruidMaster.java | 2 +- .../metamx/druid/master/DruidMasterTest.java | 2 + 24 files changed, 356 insertions(+), 224 deletions(-) rename merger/src/main/java/com/metamx/druid/merger/{coordinator => common}/TaskCallback.java (95%) diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryManager.java b/client/src/main/java/com/metamx/druid/client/ServerInventoryManager.java index bd32a62791d..891d41aaec9 100644 --- a/client/src/main/java/com/metamx/druid/client/ServerInventoryManager.java +++ b/client/src/main/java/com/metamx/druid/client/ServerInventoryManager.java @@ -19,7 +19,6 @@ package com.metamx.druid.client; -import com.google.common.collect.Maps; import com.metamx.common.Pair; import com.metamx.common.logger.Logger; import com.metamx.phonebook.PhoneBook; @@ -27,12 +26,13 @@ import com.metamx.phonebook.PhoneBookPeon; import java.util.Iterator; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; /** */ public class ServerInventoryManager extends InventoryManager { - private static final Map removedSegments = Maps.newHashMap(); + private static final Map removedSegments = new ConcurrentHashMap(); public ServerInventoryManager( ServerInventoryManagerConfig config, diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskCallback.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java similarity index 95% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskCallback.java rename to merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java index 549d0e7c4a7..a134ad2455e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskCallback.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java @@ -17,7 +17,7 @@ * 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.common.TaskStatus; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java index 39b38a18a93..a492f3015f7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java @@ -40,6 +40,13 @@ public class TaskStatus CONTINUED } + public static enum Action + { + NONE, + ANNOUNCE_SEGMENTS, + DELETE_SEGMENTS + } + public static TaskStatus running(String taskId) { return new TaskStatus( @@ -47,29 +54,44 @@ public class TaskStatus Status.RUNNING, Collections.emptyList(), Collections.emptyList(), - -1 + -1, + Action.NONE ); } public static TaskStatus failure(String taskId) { - return new TaskStatus(taskId, Status.FAILED, Collections.emptyList(), Collections.emptyList(), -1); + return new TaskStatus( + taskId, + Status.FAILED, + Collections.emptyList(), + Collections.emptyList(), + -1, + Action.NONE + ); } public static TaskStatus success(String taskId, List segments) { - return new TaskStatus(taskId, Status.SUCCESS, ImmutableList.copyOf(segments), Collections.emptyList(), -1); + return new TaskStatus( + taskId, + Status.SUCCESS, + ImmutableList.copyOf(segments), + Collections.emptyList(), + -1, + Action.NONE + ); } public static TaskStatus continued(String taskId, List nextTasks) { - Preconditions.checkArgument(nextTasks.size() > 0, "nextTasks.size() > 0"); return new TaskStatus( taskId, Status.CONTINUED, Collections.emptyList(), ImmutableList.copyOf(nextTasks), - -1 + -1, + Action.NONE ); } @@ -78,6 +100,7 @@ public class TaskStatus private final List nextTasks; private final Status status; private final long duration; + private final Action action; @JsonCreator private TaskStatus( @@ -85,7 +108,8 @@ public class TaskStatus @JsonProperty("status") Status status, @JsonProperty("segments") List segments, @JsonProperty("nextTasks") List nextTasks, - @JsonProperty("duration") long duration + @JsonProperty("duration") long duration, + @JsonProperty("action") Action action ) { this.id = id; @@ -93,6 +117,7 @@ public class TaskStatus this.nextTasks = nextTasks; this.status = status; this.duration = duration; + this.action = action; } @JsonProperty("id") @@ -125,6 +150,12 @@ public class TaskStatus return duration; } + @JsonProperty("action") + public Action getAction() + { + return action; + } + /** * Signals that a task is not yet complete, and is still runnable on a worker. Exactly one of isRunnable, * isContinued, isSuccess, or isFailure will be true at any one time. @@ -171,7 +202,23 @@ public class TaskStatus public TaskStatus withDuration(long _duration) { - return new TaskStatus(id, status, segments, nextTasks, _duration); + return new TaskStatus(id, status, segments, nextTasks, _duration, action); + } + + public TaskStatus withSegments(List _segments) + { + return new TaskStatus(id, status, _segments, nextTasks, duration, action); + } + + public TaskStatus withNextTasks(List _nextTasks) + { + Preconditions.checkArgument(_nextTasks.size() > 0, "nextTasks.size() > 0"); + return new TaskStatus(id, status, segments, _nextTasks, duration, action); + } + + public TaskStatus withAction(Action _action) + { + return new TaskStatus(id, status, segments, nextTasks, duration, _action); } @Override @@ -183,6 +230,7 @@ public class TaskStatus .add("nextTasks", nextTasks) .add("status", status) .add("duration", duration) + .add("action", action) .toString(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 5f08298cd2d..82c85a67f40 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -28,19 +28,17 @@ import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IncrementalIndexAdapter; import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.index.v1.IndexableAdapter; -import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.shard.NoneShardSpec; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; import org.joda.time.Interval; import java.io.File; -import java.util.ArrayList; public class DeleteTask extends AbstractTask { @@ -72,7 +70,7 @@ public class DeleteTask extends AbstractTask } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { // Strategy: Create an empty segment covering the interval to be deleted final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); @@ -100,6 +98,7 @@ public class DeleteTask extends AbstractTask segment.getVersion() ); - return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)); + return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)) + .withAction(TaskStatus.Action.ANNOUNCE_SEGMENTS); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 55edfdbc3bc..fce5ebffb3b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -20,7 +20,6 @@ package com.metamx.druid.merger.common.task; import com.google.common.base.Function; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -29,6 +28,7 @@ import com.google.common.collect.TreeMultiset; import com.google.common.primitives.Ints; import com.metamx.common.logger.Logger; import com.metamx.druid.input.InputRow; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -42,16 +42,18 @@ import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.util.List; import java.util.Map; import java.util.Set; public class IndexDeterminePartitionsTask extends AbstractTask { - @JsonProperty private final FirehoseFactory firehoseFactory; - @JsonProperty private final Schema schema; - @JsonProperty private final long targetPartitionSize; + @JsonProperty + private final FirehoseFactory firehoseFactory; + @JsonProperty + private final Schema schema; + @JsonProperty + private final long targetPartitionSize; private static final Logger log = new Logger(IndexTask.class); @@ -88,7 +90,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { log.info("Running with targetPartitionSize[%d]", targetPartitionSize); @@ -108,24 +110,24 @@ public class IndexDeterminePartitionsTask extends AbstractTask final Firehose firehose = firehoseFactory.connect(); try { - while(firehose.hasMore()) { + while (firehose.hasMore()) { final InputRow inputRow = firehose.nextRow(); - if(getInterval().contains(inputRow.getTimestampFromEpoch())) { + if (getInterval().contains(inputRow.getTimestampFromEpoch())) { // Extract dimensions from event for (final String dim : inputRow.getDimensions()) { final List dimValues = inputRow.getDimension(dim); - if(!unusableDimensions.contains(dim)) { + if (!unusableDimensions.contains(dim)) { - if(dimValues.size() == 1) { + if (dimValues.size() == 1) { // Track this value TreeMultiset dimensionValueMultiset = dimensionValueMultisets.get(dim); - if(dimensionValueMultiset == null) { + if (dimensionValueMultiset == null) { dimensionValueMultiset = TreeMultiset.create(); dimensionValueMultisets.put(dim, dimensionValueMultiset); } @@ -146,7 +148,8 @@ public class IndexDeterminePartitionsTask extends AbstractTask } } - } finally { + } + finally { firehose.close(); } @@ -166,7 +169,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask } }; - if(dimensionValueMultisets.isEmpty()) { + if (dimensionValueMultisets.isEmpty()) { // No suitable partition dimension. We'll make one big segment and hope for the best. log.info("No suitable partition dimension found"); shardSpecs.add(new NoneShardSpec()); @@ -188,9 +191,9 @@ public class IndexDeterminePartitionsTask extends AbstractTask // Iterate over unique partition dimension values in sorted order String currentPartitionStart = null; int currentPartitionSize = 0; - for(final String partitionDimValue : partitionDimValues.elementSet()) { + for (final String partitionDimValue : partitionDimValues.elementSet()) { currentPartitionSize += partitionDimValues.count(partitionDimValue); - if(currentPartitionSize >= targetPartitionSize) { + if (currentPartitionSize >= targetPartitionSize) { final ShardSpec shardSpec = new SingleDimensionShardSpec( partitionDim, currentPartitionStart, @@ -229,25 +232,26 @@ public class IndexDeterminePartitionsTask extends AbstractTask return TaskStatus.continued( getId(), Lists.transform( - shardSpecs, new Function() - { - @Override - public Task apply(ShardSpec shardSpec) - { - return new IndexGeneratorTask( - getGroupId(), - getInterval(), - firehoseFactory, - new Schema( - schema.getDataSource(), - schema.getAggregators(), - schema.getIndexGranularity(), - shardSpec - ) - ); - } - } + shardSpecs, + new Function() + { + @Override + public Task apply(ShardSpec shardSpec) + { + return new IndexGeneratorTask( + getGroupId(), + getInterval(), + firehoseFactory, + new Schema( + schema.getDataSource(), + schema.getAggregators(), + schema.getIndexGranularity(), + shardSpec + ) + ); + } + } ) - ); + ).withAction(TaskStatus.Action.ANNOUNCE_SEGMENTS); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index b89142ef19a..aedcca1b665 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -25,6 +25,8 @@ import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.input.InputRow; +import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.index.YeOldePlumberSchool; @@ -34,7 +36,6 @@ import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.Schema; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.realtime.Sink; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; @@ -89,7 +90,7 @@ public class IndexGeneratorTask extends AbstractTask } @Override - public TaskStatus run(final TaskContext context, final TaskToolbox toolbox) throws Exception + public TaskStatus run(final TaskContext context, final TaskToolbox toolbox, TaskCallback callback) throws Exception { // Set up temporary directory for indexing final File tmpDir = new File( @@ -131,10 +132,10 @@ public class IndexGeneratorTask extends AbstractTask ).findPlumber(schema, metrics); try { - while(firehose.hasMore()) { + while (firehose.hasMore()) { final InputRow inputRow = firehose.nextRow(); - if(shouldIndex(inputRow)) { + if (shouldIndex(inputRow)) { final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch()); if (sink == null) { throw new NullPointerException( @@ -148,14 +149,15 @@ public class IndexGeneratorTask extends AbstractTask int numRows = sink.add(inputRow); metrics.incrementProcessed(); - if(numRows >= toolbox.getConfig().getRowFlushBoundary()) { + if (numRows >= toolbox.getConfig().getRowFlushBoundary()) { plumber.persist(firehose.commit()); } } else { metrics.incrementThrownAway(); } } - } finally { + } + finally { firehose.close(); } @@ -174,23 +176,27 @@ public class IndexGeneratorTask extends AbstractTask ); // Done - return TaskStatus.success(getId(), ImmutableList.copyOf(pushedSegments)); + return TaskStatus.success(getId(), ImmutableList.copyOf(pushedSegments)) + .withAction(TaskStatus.Action.ANNOUNCE_SEGMENTS); } /** * Should we index this inputRow? Decision is based on our interval and shardSpec. + * * @param inputRow the row to check + * * @return true or false */ - private boolean shouldIndex(InputRow inputRow) { - if(!getInterval().contains(inputRow.getTimestampFromEpoch())) { + private boolean shouldIndex(InputRow inputRow) + { + if (!getInterval().contains(inputRow.getTimestampFromEpoch())) { return false; } final Map eventDimensions = Maps.newHashMapWithExpectedSize(inputRow.getDimensions().size()); - for(final String dim : inputRow.getDimensions()) { + for (final String dim : inputRow.getDimensions()) { final List dimValues = inputRow.getDimension(dim); - if(dimValues.size() == 1) { + if (dimValues.size() == 1) { eventDimensions.put(dim, Iterables.getOnlyElement(dimValues)); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 6074765ddbf..c8391d7bfce 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -25,6 +25,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.indexer.granularity.GranularitySpec; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -126,11 +127,11 @@ public class IndexTask extends AbstractTask @Override public TaskStatus preflight(TaskContext context) throws Exception { - return TaskStatus.continued(getId(), toSubtasks()); + return TaskStatus.continued(getId(), toSubtasks()).withAction(TaskStatus.Action.ANNOUNCE_SEGMENTS); } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { throw new IllegalStateException("IndexTasks should not be run!"); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index 5b2d415eb47..06612d978df 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -3,6 +3,7 @@ package com.metamx.druid.merger.common.task; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -11,6 +12,8 @@ import org.codehaus.jackson.annotate.JsonProperty; import org.joda.time.DateTime; import org.joda.time.Interval; +import java.util.List; + /** */ public class KillTask extends AbstractTask @@ -43,10 +46,10 @@ public class KillTask extends AbstractTask } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { // Kill segments - toolbox.getSegmentKiller().kill(getDataSource(), getInterval()); - return TaskStatus.success(getId(), Lists.newArrayList()); + List segmentsToKill = toolbox.getSegmentKiller().kill(getDataSource(), getInterval()); + return TaskStatus.success(getId(), segmentsToKill).withAction(TaskStatus.Action.DELETE_SEGMENTS); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 7388058bad0..d1be1f18ce7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -35,6 +35,7 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.SegmentPuller; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -115,7 +116,7 @@ public abstract class MergeTask extends AbstractTask } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { final ServiceEmitter emitter = toolbox.getEmitter(); final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); @@ -171,7 +172,8 @@ public abstract class MergeTask extends AbstractTask emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)); + return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)) + .withAction(TaskStatus.Action.ANNOUNCE_SEGMENTS); } catch (Exception e) { log.error( @@ -285,12 +287,12 @@ public abstract class MergeTask 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(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 961afac96ce..84c66512266 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -21,13 +21,10 @@ package com.metamx.druid.merger.common.task; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.coordinator.TaskContext; -import com.metamx.druid.merger.common.task.IndexDeterminePartitionsTask; -import com.metamx.druid.merger.common.task.IndexGeneratorTask; -import com.metamx.druid.merger.common.task.IndexTask; import org.codehaus.jackson.annotate.JsonSubTypes; import org.codehaus.jackson.annotate.JsonTypeInfo; -import org.joda.time.DateTime; import org.joda.time.Interval; /** @@ -86,5 +83,5 @@ public interface Task * @return Some kind of finished status (isRunnable must be false). * @throws Exception */ - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception; + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception; } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index fc03504792f..5dd4f4d2204 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -22,6 +22,7 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Throwables; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.task.Task; @@ -70,7 +71,7 @@ public class LocalTaskRunner implements TaskRunner TaskStatus status; try { - status = task.run(context, toolbox); + status = task.run(context, toolbox, null); } catch (InterruptedException e) { log.error(e, "Interrupted while running task[%s]", task); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java index bc53ef0d4f7..828404d3961 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java @@ -191,4 +191,24 @@ public class MergerDBCoordinator } } } + + public void deleteSegment(final DataSegment segment) + { + dbi.withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception + { + handle.createStatement( + String.format("DELETE from %s WHERE id = :id", dbConnectorConfig.getSegmentTable()) + ).bind("id", segment.getIdentifier()) + .execute(); + + return null; + } + } + ); + + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 5537a6b6420..37bb3e9ca08 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -33,6 +33,7 @@ import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.PeriodGranularity; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskHolder; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; @@ -427,27 +428,27 @@ public class RemoteTaskRunner implements TaskRunner statusLock.notify(); + final TaskWrapper taskWrapper = tasks.get(taskId); + if (taskWrapper == null) { + log.warn( + "WTF?! Worker[%s] announcing a status for a task I didn't know about: %s", + worker.getHost(), + taskId + ); + } else { + final TaskCallback callback = taskWrapper.getCallback(); + + // Cleanup + if (callback != null) { + callback.notify(taskStatus); + } + } + if (taskStatus.isComplete()) { // Worker is done with this task workerWrapper.setLastCompletedTaskTime(new DateTime()); - final TaskWrapper taskWrapper = tasks.get(taskId); - - if (taskWrapper == null) { - log.warn( - "WTF?! Worker[%s] completed a task I didn't know about: %s", - worker.getHost(), - taskId - ); - } else { - final TaskCallback callback = taskWrapper.getCallback(); - - // Cleanup - if (callback != null) { - callback.notify(taskStatus); - } - tasks.remove(taskId); - cf.delete().guaranteed().inBackground().forPath(event.getData().getPath()); - } + tasks.remove(taskId); + cf.delete().guaranteed().inBackground().forPath(event.getData().getPath()); } } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java index f674b8744cd..79d54b5ef20 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java @@ -52,18 +52,18 @@ import java.util.concurrent.locks.ReentrantLock; /** * Interface between task producers and task consumers. - * + *

    * The queue accepts tasks from producers using {@link #add} and delivers tasks to consumers using either * {@link #take} or {@link #poll}. Ordering is mostly-FIFO, with deviations when the natural next task would conflict * with a currently-running task. In that case, tasks are skipped until a runnable one is found. - * + *

    * To manage locking, the queue keeps track of currently-running tasks as {@link TaskGroup} objects. The idea is that * only one TaskGroup can be running on a particular dataSource + interval, and that TaskGroup has a single version * string that all tasks in the group must use to publish segments. Tasks in the same TaskGroup may run concurrently. - * + *

    * For persistence, the queue saves new tasks from {@link #add} and task status updates from {@link #done} using a * {@link TaskStorage} object. - * + *

    * To support leader election of our containing system, the queue can be stopped (in which case it will not accept * any new tasks, or hand out any more tasks, until started again). */ @@ -127,13 +127,13 @@ public class TaskQueue } }; - for(final Pair taskAndVersion : byVersionOrdering.sortedCopy(runningTasks)) { + for (final Pair taskAndVersion : byVersionOrdering.sortedCopy(runningTasks)) { final Task task = taskAndVersion.lhs; final String preferredVersion = taskAndVersion.rhs; queue.add(task); - if(preferredVersion != null) { + if (preferredVersion != null) { final Optional version = tryLock(task, Optional.of(preferredVersion)); log.info( @@ -173,7 +173,8 @@ public class TaskQueue running.clear(); active = false; - } finally { + } + finally { giant.unlock(); } } @@ -182,6 +183,7 @@ public class TaskQueue * Adds some work to the queue and the underlying task storage facility with a generic "running" status. * * @param task task to add + * * @return true */ public boolean add(Task task) @@ -205,6 +207,7 @@ public class TaskQueue /** * Locks and returns next doable work from the queue. Blocks if there is no doable work. + * * @return runnable task */ public VersionedTaskWrapper take() throws InterruptedException @@ -214,19 +217,21 @@ public class TaskQueue try { VersionedTaskWrapper taskWrapper; - while((taskWrapper = poll()) == null) { + while ((taskWrapper = poll()) == null) { log.info("Waiting for work..."); workMayBeAvailable.await(); } return taskWrapper; - } finally { + } + finally { giant.unlock(); } } /** * Locks and removes next doable work from the queue. Returns null if there is no doable work. + * * @return runnable task or null */ public VersionedTaskWrapper poll() @@ -235,9 +240,9 @@ public class TaskQueue try { log.info("Checking for doable work"); - for(final Task task : queue) { + for (final Task task : queue) { final Optional maybeVersion = tryLock(task); - if(maybeVersion.isPresent()) { + if (maybeVersion.isPresent()) { Preconditions.checkState(active, "wtf? Found task when inactive"); taskStorage.setVersion(task.getId(), maybeVersion.get()); queue.remove(task); @@ -259,6 +264,7 @@ public class TaskQueue * running. * * @param task task to unlock + * * @throws IllegalStateException if task is not currently locked */ private void unlock(final Task task) @@ -284,7 +290,7 @@ public class TaskQueue ); final TaskGroup taskGroup; - if(maybeTaskGroup.size() == 1) { + if (maybeTaskGroup.size() == 1) { taskGroup = maybeTaskGroup.get(0); } else { throw new IllegalStateException(String.format("Task must be running: %s", task.getId())); @@ -294,12 +300,12 @@ public class TaskQueue log.info("Removing task[%s] from TaskGroup[%s]", task.getId(), taskGroup.getGroupId()); taskGroup.getTaskSet().remove(task); - if(taskGroup.getTaskSet().size() == 0) { + if (taskGroup.getTaskSet().size() == 0) { log.info("TaskGroup complete: %s", taskGroup); running.get(dataSource).remove(taskGroup.getInterval()); } - if(running.get(dataSource).size() == 0) { + if (running.get(dataSource).size() == 0) { running.remove(dataSource); } @@ -314,8 +320,9 @@ public class TaskQueue * Unlock some task and update its status in the task storage facility. If "status" is a continuation status (i.e. * it has nextTasks) this will add the next tasks to the queue with a generic running status. * - * @param task task to unlock + * @param task task to unlock * @param status task completion status; must not be runnable + * * @throws IllegalStateException if task is not currently running, or if status is runnable */ public void done(final Task task, final TaskStatus status) @@ -338,11 +345,12 @@ public class TaskQueue // Add next tasks, if any try { - for(final Task nextTask : status.getNextTasks()) { + for (final Task nextTask : status.getNextTasks()) { add(nextTask); tryLock(nextTask); } - } catch(Exception e) { + } + catch (Exception e) { log.error(e, "Failed to continue task: %s", task.getId()); actualStatus = TaskStatus.failure(task.getId()); } @@ -369,7 +377,7 @@ public class TaskQueue try { final Optional statusOptional = taskStorage.getStatus(taskid); - if(statusOptional.isPresent()) { + if (statusOptional.isPresent()) { // See if we can collapse this down return Optional.of(collapseStatus(statusOptional.get())); } else { @@ -383,17 +391,16 @@ public class TaskQueue private TaskStatus collapseStatus(TaskStatus status) { - if (status.isContinued()) { int nSubtasks = 0; int nSuccesses = 0; List segments = Lists.newArrayList(); - for(final Task subtask : status.getNextTasks()) { + for (final Task subtask : status.getNextTasks()) { final TaskStatus subtaskStatus = collapseStatus(taskStorage.getStatus(subtask.getId()).get()); - nSubtasks ++; + nSubtasks++; if (subtaskStatus.isFailure()) { return TaskStatus.failure(status.getId()); @@ -405,7 +412,7 @@ public class TaskQueue } if (nSubtasks == nSuccesses) { - return TaskStatus.success(status.getId(), segments); + return TaskStatus.success(status.getId(), segments).withAction(status.getAction()); } } @@ -419,6 +426,7 @@ public class TaskQueue * Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task. * * @param task task to attempt to lock + * * @return lock version if lock was acquired, absent otherwise */ private Optional tryLock(final Task task) @@ -429,8 +437,9 @@ public class TaskQueue /** * Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task. * - * @param task task to attempt to lock + * @param task task to attempt to lock * @param preferredVersion use this version if possible (no guarantees, though!) + * * @return lock version if lock was acquired, absent otherwise */ private Optional tryLock(final Task task, final Optional preferredVersion) @@ -474,7 +483,7 @@ public class TaskQueue final String version; - if(preferredVersion.isPresent()) { + if (preferredVersion.isPresent()) { // We have a preferred version. Since this is a private method, we'll trust our caller to not break our // ordering assumptions and just use it. version = preferredVersion.get(); @@ -503,7 +512,8 @@ public class TaskQueue return Optional.of(taskGroupToUse.getVersion()); - } finally { + } + finally { giant.unlock(); } @@ -518,7 +528,7 @@ public class TaskQueue try { final NavigableMap dsRunning = running.get(dataSource); - if(dsRunning == null) { + if (dsRunning == null) { // No locks at all return Collections.emptyList(); } else { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java index 5362741ac92..2cc9ba43600 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.task.Task; /** diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java index c757bb2dc33..22e7d4152b4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.task.Task; /** diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index 78326d3a3cc..f24c562a045 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -21,13 +21,14 @@ package com.metamx.druid.merger.coordinator.exec; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableSet; +import com.metamx.common.ISE; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.MergerDBCoordinator; -import com.metamx.druid.merger.coordinator.TaskCallback; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskRunner; @@ -36,6 +37,8 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import java.util.List; + public class TaskConsumer implements Runnable { private final TaskQueue queue; @@ -107,7 +110,7 @@ public class TaskConsumer implements Runnable .emit(); // Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now. - if(!shutdown) { + if (!shutdown) { queue.done(task, TaskStatus.failure(task.getId())); } } @@ -141,7 +144,8 @@ public class TaskConsumer implements Runnable try { preflightStatus = task.preflight(context); log.info("Preflight done for task: %s", task.getId()); - } catch(Exception e) { + } + catch (Exception e) { preflightStatus = TaskStatus.failure(task.getId()); log.error(e, "Exception thrown during preflight for task: %s", task.getId()); } @@ -168,47 +172,25 @@ public class TaskConsumer implements Runnable // If we're not supposed to be running anymore, don't do anything. Somewhat racey if the flag gets set after // we check and before we commit the database transaction, but better than nothing. - if(shutdown) { + if (shutdown) { log.info("Abandoning task due to shutdown: %s", task.getId()); return; } - // Publish returned segments - // FIXME: Publish in transaction try { - for (DataSegment segment : status.getSegments()) { - if (!task.getDataSource().equals(segment.getDataSource())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid dataSource: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - - if (!task.getInterval().contains(segment.getInterval())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid interval: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - - if (!context.getVersion().equals(segment.getVersion())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid version: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - - log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), task.getId()); - mergerDBCoordinator.announceHistoricalSegment(segment); + switch (status.getAction()) { + case ANNOUNCE_SEGMENTS: + // Publish returned segments + // FIXME: Publish in transaction + publishSegments(task, context, status.getSegments()); + break; + case DELETE_SEGMENTS: + deleteSegments(task, context, status.getSegments()); + break; + case NONE: + break; + default: + throw new ISE("Unknown Action[%s]", status.getAction().getClass()); } } catch (Exception e) { @@ -257,4 +239,57 @@ public class TaskConsumer implements Runnable } ); } + + private void deleteSegments(Task task, TaskContext context, List segments) throws Exception + { + for (DataSegment segment : segments) { + verifySegment(task, context, segment); + + log.info("Deleting segment[%s] for task[%s]", segment.getIdentifier(), task.getId()); + mergerDBCoordinator.deleteSegment(segment); + } + } + + private void publishSegments(Task task, TaskContext context, List segments) throws Exception + { + for (DataSegment segment : segments) { + verifySegment(task, context, segment); + + log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), task.getId()); + mergerDBCoordinator.announceHistoricalSegment(segment); + } + } + + private void verifySegment(Task task, TaskContext context, DataSegment segment) + { + if (!task.getDataSource().equals(segment.getDataSource())) { + throw new IllegalStateException( + String.format( + "Segment for task[%s] has invalid dataSource: %s", + task.getId(), + segment.getIdentifier() + ) + ); + } + + if (!task.getInterval().contains(segment.getInterval())) { + throw new IllegalStateException( + String.format( + "Segment for task[%s] has invalid interval: %s", + task.getId(), + segment.getIdentifier() + ) + ); + } + + if (!context.getVersion().equals(segment.getVersion())) { + throw new IllegalStateException( + String.format( + "Segment for task[%s] has invalid version: %s", + task.getId(), + segment.getIdentifier() + ) + ); + } + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index 5fc49788fcd..506b33e1416 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -21,6 +21,7 @@ package com.metamx.druid.merger.worker; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskHolder; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; @@ -110,7 +111,14 @@ public class TaskMonitor try { workerCuratorCoordinator.unannounceTask(task.getId()); workerCuratorCoordinator.announceStatus(TaskStatus.running(task.getId())); - taskStatus = task.run(taskContext, toolbox); + taskStatus = task.run(taskContext, toolbox, new TaskCallback() + { + @Override + public void notify(TaskStatus status) + { + workerCuratorCoordinator.updateStatus(status); + } + }); } catch (Exception e) { log.makeAlert(e, "Failed to run task") diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index c64731c243c..d74150aa1fe 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -7,6 +7,7 @@ import com.metamx.common.ISE; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; @@ -190,8 +191,7 @@ public class RemoteTaskRunnerTest String.format("%s/worker1/task1", statusPath), jsonMapper.writeValueAsBytes( TaskStatus.success( - "task1", - Lists.newArrayList() + "task1" ) ) ); @@ -500,9 +500,9 @@ public class RemoteTaskRunnerTest } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { - return TaskStatus.success("task1", Lists.newArrayList()); + return TaskStatus.success("task1"); } } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 47913404eed..5eb460a3b4c 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.task.AbstractTask; @@ -76,18 +77,23 @@ public class TaskQueueTest Throwable thrown; - for(Task task : tasks) { + for (Task task : tasks) { tq.add(task); } // get task status for in-progress task - Assert.assertEquals("T2 status (before finishing)", TaskStatus.Status.RUNNING, tq.getStatus(tasks[2].getId()).get().getStatusCode()); + Assert.assertEquals( + "T2 status (before finishing)", + TaskStatus.Status.RUNNING, + tq.getStatus(tasks[2].getId()).get().getStatusCode() + ); // Can't add tasks with the same id thrown = null; try { tq.add(newTask("T5", "G5", "baz", new Interval("2013-02-01/PT1H"))); - } catch(IllegalStateException e) { + } + catch (IllegalStateException e) { thrown = e; } @@ -97,7 +103,7 @@ public class TaskQueueTest final List taken = Lists.newArrayList(); while (true) { final VersionedTaskWrapper taskWrapper = tq.poll(); - if(taskWrapper != null) { + if (taskWrapper != null) { taken.add(taskWrapper.getTask()); } else { break; @@ -114,16 +120,21 @@ public class TaskQueueTest ); // mark one done - tq.done(tasks[2], tasks[2].run(null, null)); + tq.done(tasks[2], tasks[2].run(null, null, null)); // get its status back - Assert.assertEquals("T2 status (after finishing)", TaskStatus.Status.SUCCESS, tq.getStatus(tasks[2].getId()).get().getStatusCode()); + Assert.assertEquals( + "T2 status (after finishing)", + TaskStatus.Status.SUCCESS, + tq.getStatus(tasks[2].getId()).get().getStatusCode() + ); // Can't do a task twice thrown = null; try { - tq.done(tasks[2], tasks[2].run(null, null)); - } catch(IllegalStateException e) { + tq.done(tasks[2], tasks[2].run(null, null, null)); + } + catch (IllegalStateException e) { thrown = e; } @@ -133,7 +144,7 @@ public class TaskQueueTest taken.clear(); while (true) { final VersionedTaskWrapper taskWrapper = tq.poll(); - if(taskWrapper != null) { + if (taskWrapper != null) { taken.add(taskWrapper.getTask()); } else { break; @@ -162,9 +173,9 @@ public class TaskQueueTest final Task t1 = newContinuedTask("T1", "G1", "bar", new Interval("2013/P1Y"), Lists.newArrayList(t0)); tq.add(t1); - Assert.assertTrue("T0 isPresent (#1)", !tq.getStatus("T0").isPresent()); - Assert.assertTrue("T1 isPresent (#1)", tq.getStatus("T1").isPresent()); - Assert.assertTrue("T1 isRunnable (#1)", tq.getStatus("T1").get().isRunnable()); + Assert.assertTrue("T0 isPresent (#1)", !tq.getStatus("T0").isPresent()); + Assert.assertTrue("T1 isPresent (#1)", tq.getStatus("T1").isPresent()); + Assert.assertTrue("T1 isRunnable (#1)", tq.getStatus("T1").get().isRunnable()); Assert.assertTrue("T1 isComplete (#1)", !tq.getStatus("T1").get().isComplete()); // should be able to get t1 out @@ -172,28 +183,28 @@ public class TaskQueueTest Assert.assertNull("poll #2", tq.poll()); // report T1 done. Should cause T0 to be created - tq.done(t1, t1.run(null, null)); + tq.done(t1, t1.run(null, null, null)); - Assert.assertTrue("T0 isPresent (#2)", tq.getStatus("T0").isPresent()); - Assert.assertTrue("T0 isRunnable (#2)", tq.getStatus("T0").get().isRunnable()); + Assert.assertTrue("T0 isPresent (#2)", tq.getStatus("T0").isPresent()); + Assert.assertTrue("T0 isRunnable (#2)", tq.getStatus("T0").get().isRunnable()); Assert.assertTrue("T0 isComplete (#2)", !tq.getStatus("T0").get().isComplete()); - Assert.assertTrue("T1 isPresent (#2)", tq.getStatus("T1").isPresent()); + Assert.assertTrue("T1 isPresent (#2)", tq.getStatus("T1").isPresent()); Assert.assertTrue("T1 isRunnable (#2)", !tq.getStatus("T1").get().isRunnable()); - Assert.assertTrue("T1 isComplete (#2)", tq.getStatus("T1").get().isComplete()); + Assert.assertTrue("T1 isComplete (#2)", tq.getStatus("T1").get().isComplete()); // should be able to get t0 out Assert.assertEquals("poll #3", "T0", tq.poll().getTask().getId()); Assert.assertNull("poll #4", tq.poll()); // report T0 done. Should cause T0, T1 to be marked complete - tq.done(t0, t0.run(null, null)); + tq.done(t0, t0.run(null, null, null)); - Assert.assertTrue("T0 isPresent (#3)", tq.getStatus("T0").isPresent()); + Assert.assertTrue("T0 isPresent (#3)", tq.getStatus("T0").isPresent()); Assert.assertTrue("T0 isRunnable (#3)", !tq.getStatus("T0").get().isRunnable()); - Assert.assertTrue("T0 isComplete (#3)", tq.getStatus("T0").get().isComplete()); - Assert.assertTrue("T1 isPresent (#3)", tq.getStatus("T1").isPresent()); + Assert.assertTrue("T0 isComplete (#3)", tq.getStatus("T0").get().isComplete()); + Assert.assertTrue("T1 isPresent (#3)", tq.getStatus("T1").isPresent()); Assert.assertTrue("T1 isRunnable (#3)", !tq.getStatus("T1").get().isRunnable()); - Assert.assertTrue("T1 isComplete (#3)", tq.getStatus("T1").get().isComplete()); + Assert.assertTrue("T1 isComplete (#3)", tq.getStatus("T1").get().isComplete()); // should be no more events available for polling Assert.assertNull("poll #5", tq.poll()); @@ -227,7 +238,7 @@ public class TaskQueueTest Thread.sleep(5); // Finish t0 - tq.done(t0, t0.run(null, null)); + tq.done(t0, t0.run(null, null, null)); // take max number of tasks final Set taken = Sets.newHashSet(); @@ -238,7 +249,7 @@ public class TaskQueueTest final VersionedTaskWrapper taskWrapper = tq.poll(); - if(taskWrapper != null) { + if (taskWrapper != null) { Assert.assertEquals( String.format("%s version", taskWrapper.getTask().getId()), wt0.getVersion(), @@ -254,11 +265,11 @@ public class TaskQueueTest Assert.assertEquals("taken", Sets.newHashSet("T1", "T3"), taken); // Finish t1 - tq.done(t1, t1.run(null, null)); + tq.done(t1, t1.run(null, null, null)); Assert.assertNull("null poll #2", tq.poll()); // Finish t3 - tq.done(t3, t3.run(null, null)); + tq.done(t3, t3.run(null, null, null)); // We should be able to get t2 now final VersionedTaskWrapper wt2 = tq.poll(); @@ -268,7 +279,7 @@ public class TaskQueueTest Assert.assertNull("null poll #3", tq.poll()); // Finish t2 - tq.done(t2, t2.run(null, null)); + tq.done(t2, t2.run(null, null, null)); // We should be able to get t4 // And it should be in group G0, but that group should have a different version than last time @@ -281,7 +292,7 @@ public class TaskQueueTest Assert.assertNotSame("wt4 version", wt2.getVersion(), wt4.getVersion()); // Kind of done testing at this point, but let's finish t4 anyway - tq.done(t4, t4.run(null, null)); + tq.done(t4, t4.run(null, null, null)); Assert.assertNull("null poll #4", tq.poll()); } @@ -314,7 +325,7 @@ public class TaskQueueTest return new AbstractTask(id, groupId, dataSource, interval) { @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { return TaskStatus.success( id, @@ -358,7 +369,7 @@ public class TaskQueueTest } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { return TaskStatus.continued(id, nextTasks); } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index 2840eb037a8..6dc83bf3264 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -189,7 +189,11 @@ public class DatabaseRuleManager public Map> withHandle(Handle handle) throws Exception { return handle.createQuery( - String.format("SELECT dataSource, payload FROM %s", config.getRuleTable()) + // Return latest version rule by dataSource + String.format( + "SELECT %1$s.dataSource, %1$s.payload FROM %1$s INNER JOIN(SELECT dataSource, max(version) as version, payload FROM %1$s GROUP BY dataSource) ds ON %1$s.datasource = ds.datasource and %1$s.version = ds.version", + config.getRuleTable() + ) ).fold( Maps.>newHashMap(), new Folder3>, Map>() diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java index 346bb036ff3..801e16a1602 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java @@ -101,42 +101,20 @@ public class S3SegmentKiller implements SegmentKiller log.info("Found %,d segments for %s for interval %s.", matchingSegments.size(), datasource, interval); for (final DataSegment segment : matchingSegments) { - // Remove database entry - log.info("Removing DB entry for %s", segment.getIdentifier()); - dbi.withHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) throws Exception - { - handle.createStatement( - String.format("DELETE from %s WHERE id = :segmentID", config.getSegmentTable()) - ).bind("segmentID", segment.getIdentifier()) - .execute(); - - return null; - } - } - ); - // Remove from S3 - Map loadSpec = segment.getLoadSpec(); String s3Bucket = MapUtils.getString(loadSpec, "bucket"); String s3Path = MapUtils.getString(loadSpec, "key"); String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; - if (!s3Client.isObjectInBucket(s3Bucket, s3Path)) { - throw new ISE("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3Path); + if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { + log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); + s3Client.deleteObject(s3Bucket, s3Path); } - if (!s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { - throw new ISE("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3DescriptorPath); + if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { + log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); + s3Client.deleteObject(s3Bucket, s3DescriptorPath); } - - log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); - log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); - s3Client.deleteObject(s3Bucket, s3Path); - s3Client.deleteObject(s3Bucket, s3DescriptorPath); } return matchingSegments; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index d5d4a8bd978..e792f313f51 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -594,7 +594,6 @@ public class DruidMaster DruidMasterRuntimeParams.newBuilder() .withStartTime(startTime) .withDatasources(databaseSegmentManager.getInventory()) - .withLoadManagementPeons(loadManagementPeons) .withMillisToWaitBeforeDeleting(config.getMillisToWaitBeforeDeleting()) .withEmitter(emitter) .withMergeBytesLimit(config.getMergeBytesLimit()) @@ -696,6 +695,7 @@ public class DruidMaster decrementRemovedSegmentsLifetime(); return params.buildFromExisting() + .withLoadManagementPeons(loadManagementPeons) .withDruidCluster(cluster) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java index c54a9a66564..e9fc200a2d6 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -134,6 +134,8 @@ public class DruidMasterTest new NoopServiceEmitter(), scheduledExecutorFactory, loadManagementPeons, + null, + null, null ); } From 7f410f201d4bd0dfbf571b42202897e55117b51f Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Thu, 24 Jan 2013 17:47:10 -0800 Subject: [PATCH 03/63] updating amazon sdk version --- indexer/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexer/pom.xml b/indexer/pom.xml index 433aea2dc13..caa8c0c52bf 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -51,7 +51,7 @@ com.amazonaws aws-java-sdk - 1.2.15 + 1.3.27 javax.mail From a14200d7797104e425da160b88808b0dfcad5382 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 25 Jan 2013 10:40:15 -0800 Subject: [PATCH 04/63] Index service support for early returns and choice of commit semantics. Task: - Add TaskCallback to run method (for early returns) TaskStatus: - Remove CONTINUED status - Add segmentsNuked (placeholder for future deletion support) - Add more builder methods - Add validations to constructor TaskStorage: - Add TaskStorageQueryAdapter, a concrete class that wraps TaskStorages and provides various read-only convenience methods - Add getTask method for benefit of TaskStorageQueryAdapter TaskQueue: - Rename "done" to "notify" - notify is responsible for deciding if we should commit - Add optional commitRunnable to "notify", which gets called when it's time to commit - Allow nextTasks and commits to run early (statusCode RUNNING) - Move getStatus, collapseStatus functionality to TaskStorageQueryAdapter --- .../{coordinator => common}/TaskCallback.java | 2 +- .../druid/merger/common/TaskStatus.java | 137 ++++++-- .../druid/merger/common/task/DeleteTask.java | 6 +- .../task/IndexDeterminePartitionsTask.java | 10 +- .../common/task/IndexGeneratorTask.java | 6 +- .../druid/merger/common/task/IndexTask.java | 5 +- .../druid/merger/common/task/MergeTask.java | 5 +- .../metamx/druid/merger/common/task/Task.java | 9 +- .../merger/coordinator/DbTaskStorage.java | 30 ++ .../merger/coordinator/LocalTaskRunner.java | 9 +- .../merger/coordinator/LocalTaskStorage.java | 11 + .../merger/coordinator/RemoteTaskRunner.java | 1 + .../druid/merger/coordinator/TaskGroup.java | 73 ++-- .../druid/merger/coordinator/TaskQueue.java | 324 +++++++++--------- .../druid/merger/coordinator/TaskRunner.java | 4 +- .../druid/merger/coordinator/TaskStorage.java | 8 + .../coordinator/TaskStorageQueryAdapter.java | 145 ++++++++ .../druid/merger/coordinator/TaskWrapper.java | 1 + .../coordinator/commit/CommitStyle.java | 32 ++ .../commit/ImmediateCommitStyle.java | 33 ++ .../coordinator/commit/TaskCommitStyle.java | 32 ++ .../merger/coordinator/exec/TaskConsumer.java | 177 +++++----- .../http/IndexerCoordinatorNode.java | 2 + .../http/IndexerCoordinatorResource.java | 6 +- .../http/IndexerCoordinatorServletModule.java | 5 + .../druid/merger/worker/TaskMonitor.java | 12 +- .../druid/merger/worker/http/WorkerNode.java | 2 +- .../coordinator/RemoteTaskRunnerTest.java | 8 +- .../merger/coordinator/TaskQueueTest.java | 229 ++++++++++--- 29 files changed, 933 insertions(+), 391 deletions(-) rename merger/src/main/java/com/metamx/druid/merger/{coordinator => common}/TaskCallback.java (95%) create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/commit/CommitStyle.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/commit/ImmediateCommitStyle.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/commit/TaskCommitStyle.java diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskCallback.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java similarity index 95% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskCallback.java rename to merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java index 549d0e7c4a7..a134ad2455e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskCallback.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java @@ -17,7 +17,7 @@ * 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.common.TaskStatus; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java index 39b38a18a93..6f26cb5d3b0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java @@ -22,22 +22,31 @@ package com.metamx.druid.merger.common; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.task.Task; import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonIgnore; import org.codehaus.jackson.annotate.JsonProperty; -import java.util.Collections; import java.util.List; +import java.util.Set; +/** + * Represents the status of a task. The task may be ongoing ({@link #isComplete()} false) or it may be + * complete ({@link #isComplete()} true). Ongoing tasks may request segments to be created, but only + * complete tasks may request segments to be nuked or spawn other tasks. Failed tasks may not request + * anything. + * + * TaskStatus objects are immutable. + */ public class TaskStatus { public static enum Status { RUNNING, SUCCESS, - FAILED, - CONTINUED + FAILED } public static TaskStatus running(String taskId) @@ -45,37 +54,46 @@ public class TaskStatus return new TaskStatus( taskId, Status.RUNNING, - Collections.emptyList(), - Collections.emptyList(), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableList.of(), + -1 + ); + } + + public static TaskStatus success(String taskId) + { + return success(taskId, ImmutableSet.of()); + } + + public static TaskStatus success(String taskId, Set segments) + { + return new TaskStatus( + taskId, + Status.SUCCESS, + segments, + ImmutableSet.of(), + ImmutableList.of(), -1 ); } public static TaskStatus failure(String taskId) { - return new TaskStatus(taskId, Status.FAILED, Collections.emptyList(), Collections.emptyList(), -1); - } - - public static TaskStatus success(String taskId, List segments) - { - return new TaskStatus(taskId, Status.SUCCESS, ImmutableList.copyOf(segments), Collections.emptyList(), -1); - } - - public static TaskStatus continued(String taskId, List nextTasks) - { - Preconditions.checkArgument(nextTasks.size() > 0, "nextTasks.size() > 0"); return new TaskStatus( taskId, - Status.CONTINUED, - Collections.emptyList(), - ImmutableList.copyOf(nextTasks), + Status.FAILED, + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableList.of(), -1 ); } private final String id; - private final List segments; - private final List nextTasks; + private final ImmutableSet segments; + private final ImmutableSet segmentsNuked; + private final ImmutableList nextTasks; private final Status status; private final long duration; @@ -83,16 +101,42 @@ public class TaskStatus private TaskStatus( @JsonProperty("id") String id, @JsonProperty("status") Status status, - @JsonProperty("segments") List segments, + @JsonProperty("segments") Set segments, + @JsonProperty("segmentsNuked") Set segmentsNuked, @JsonProperty("nextTasks") List nextTasks, @JsonProperty("duration") long duration ) { this.id = id; - this.segments = segments; - this.nextTasks = nextTasks; + this.segments = ImmutableSet.copyOf(segments); + this.segmentsNuked = ImmutableSet.copyOf(segmentsNuked); + this.nextTasks = ImmutableList.copyOf(nextTasks); this.status = status; this.duration = duration; + + // Check class invariants. + Preconditions.checkNotNull(id, "id"); + Preconditions.checkNotNull(status, "status"); + + if (this.segments.size() > 0) { + Preconditions.checkArgument( + status == Status.RUNNING || status == Status.SUCCESS, + "segments not allowed for %s tasks", + status + ); + } + + if (this.segmentsNuked.size() > 0) { + Preconditions.checkArgument(status == Status.SUCCESS, "segmentsNuked not allowed for %s tasks", status); + } + + if (this.nextTasks.size() > 0) { + Preconditions.checkArgument( + status == Status.SUCCESS || status == Status.RUNNING, + "nextTasks not allowed for %s tasks", + status + ); + } } @JsonProperty("id") @@ -108,11 +152,17 @@ public class TaskStatus } @JsonProperty("segments") - public List getSegments() + public Set getSegments() { return segments; } + @JsonProperty("segmentsNuked") + public Set getSegmentsNuked() + { + return segmentsNuked; + } + @JsonProperty("nextTasks") public List getNextTasks() { @@ -127,51 +177,62 @@ public class TaskStatus /** * Signals that a task is not yet complete, and is still runnable on a worker. Exactly one of isRunnable, - * isContinued, isSuccess, or isFailure will be true at any one time. + * isSuccess, or isFailure will be true at any one time. */ + @JsonIgnore public boolean isRunnable() { return status == Status.RUNNING; } - /** - * Returned by tasks when they complete successfully without spawning subtasks. Exactly one of isRunnable, - * isContinued, isSuccess, or isFailure will be true at any one time. - */ - public boolean isContinued() - { - return status == Status.CONTINUED; - } - /** * Inverse of {@link #isRunnable}. */ + @JsonIgnore public boolean isComplete() { return !isRunnable(); } /** - * Returned by tasks when they spawn subtasks. Exactly one of isRunnable, isContinued, isSuccess, or isFailure will + * Returned by tasks when they spawn subtasks. Exactly one of isRunnable, isSuccess, or isFailure will * be true at any one time. */ + @JsonIgnore public boolean isSuccess() { return status == Status.SUCCESS; } /** - * Returned by tasks when they complete unsuccessfully. Exactly one of isRunnable, isContinued, isSuccess, or + * Returned by tasks when they complete unsuccessfully. Exactly one of isRunnable, isSuccess, or * isFailure will be true at any one time. */ + @JsonIgnore public boolean isFailure() { return status == Status.FAILED; } + public TaskStatus withSegments(Set _segments) + { + return new TaskStatus(id, status, _segments, segmentsNuked, nextTasks, duration); + } + + + public TaskStatus withSegmentsNuked(Set _segmentsNuked) + { + return new TaskStatus(id, status, segments, _segmentsNuked, nextTasks, duration); + } + + public TaskStatus withNextTasks(List _nextTasks) + { + return new TaskStatus(id, status, segments, segmentsNuked, _nextTasks, duration); + } + public TaskStatus withDuration(long _duration) { - return new TaskStatus(id, status, segments, nextTasks, _duration); + return new TaskStatus(id, status, segments, segmentsNuked, nextTasks, _duration); } @Override diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 5f08298cd2d..08877797704 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.common.task; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import com.metamx.druid.QueryGranularity; @@ -29,6 +30,7 @@ import com.metamx.druid.index.v1.IncrementalIndexAdapter; import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.index.v1.IndexableAdapter; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -72,7 +74,7 @@ public class DeleteTask extends AbstractTask } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { // Strategy: Create an empty segment covering the interval to be deleted final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); @@ -100,6 +102,6 @@ public class DeleteTask extends AbstractTask segment.getVersion() ); - return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)); + return TaskStatus.success(getId(), ImmutableSet.of(uploadedSegment)); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 55edfdbc3bc..44398b9bc2f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -29,6 +29,7 @@ import com.google.common.collect.TreeMultiset; import com.google.common.primitives.Ints; import com.metamx.common.logger.Logger; import com.metamx.druid.input.InputRow; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -88,15 +89,15 @@ public class IndexDeterminePartitionsTask extends AbstractTask } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { log.info("Running with targetPartitionSize[%d]", targetPartitionSize); // This is similar to what DeterminePartitionsJob does in the hadoop indexer, but we don't require // a preconfigured partition dimension (we'll just pick the one with highest cardinality). - // XXX - Space-efficiency (stores all unique dimension values, although at least not all combinations) - // XXX - Time-efficiency (runs all this on one single node instead of through map/reduce) + // NOTE: Space-efficiency (stores all unique dimension values, although at least not all combinations) + // NOTE: Time-efficiency (runs all this on one single node instead of through map/reduce) // Blacklist dimensions that have multiple values per row final Set unusableDimensions = Sets.newHashSet(); @@ -226,8 +227,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask } } - return TaskStatus.continued( - getId(), + return TaskStatus.success(getId()).withNextTasks( Lists.transform( shardSpecs, new Function() { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index b89142ef19a..9b70cbf5952 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -20,11 +20,13 @@ package com.metamx.druid.merger.common.task; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.input.InputRow; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.index.YeOldePlumberSchool; @@ -89,7 +91,7 @@ public class IndexGeneratorTask extends AbstractTask } @Override - public TaskStatus run(final TaskContext context, final TaskToolbox toolbox) throws Exception + public TaskStatus run(final TaskContext context, final TaskToolbox toolbox, TaskCallback callback) throws Exception { // Set up temporary directory for indexing final File tmpDir = new File( @@ -174,7 +176,7 @@ public class IndexGeneratorTask extends AbstractTask ); // Done - return TaskStatus.success(getId(), ImmutableList.copyOf(pushedSegments)); + return TaskStatus.success(getId(), ImmutableSet.copyOf(pushedSegments)); } /** diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 6074765ddbf..4d5e5624ccb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -25,6 +25,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.indexer.granularity.GranularitySpec; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -126,11 +127,11 @@ public class IndexTask extends AbstractTask @Override public TaskStatus preflight(TaskContext context) throws Exception { - return TaskStatus.continued(getId(), toSubtasks()); + return TaskStatus.success(getId()).withNextTasks(toSubtasks()); } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { throw new IllegalStateException("IndexTasks should not be run!"); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 7388058bad0..34419009c88 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -35,6 +35,7 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.SegmentPuller; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -115,7 +116,7 @@ public abstract class MergeTask extends AbstractTask } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { final ServiceEmitter emitter = toolbox.getEmitter(); final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); @@ -171,7 +172,7 @@ public abstract class MergeTask extends AbstractTask emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)); + return TaskStatus.success(getId(), ImmutableSet.of(uploadedSegment)); } catch (Exception e) { log.error( diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index b2059210b58..d69d000b01c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -21,13 +21,10 @@ package com.metamx.druid.merger.common.task; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.coordinator.TaskContext; -import com.metamx.druid.merger.common.task.IndexDeterminePartitionsTask; -import com.metamx.druid.merger.common.task.IndexGeneratorTask; -import com.metamx.druid.merger.common.task.IndexTask; import org.codehaus.jackson.annotate.JsonSubTypes; import org.codehaus.jackson.annotate.JsonTypeInfo; -import org.joda.time.DateTime; import org.joda.time.Interval; /** @@ -81,8 +78,10 @@ public interface Task * * @param context Context for this task, gathered under indexer lock * @param toolbox Toolbox for this task + * @param callback Callback for "early returns". Statuses returned to this callback must not be + * complete (isRunnable must be true). * @return Some kind of finished status (isRunnable must be false). * @throws Exception */ - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception; + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception; } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java index c1dbce4fc61..d1a71039274 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java @@ -161,6 +161,36 @@ public class DbTaskStorage implements TaskStorage ); } + @Override + public Optional getTask(final String taskid) + { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public Optional withHandle(Handle handle) throws Exception + { + final List> dbTasks = + handle.createQuery( + String.format( + "SELECT payload FROM %s WHERE id = :id", + dbConnectorConfig.getTaskTable() + ) + ) + .bind("id", taskid) + .list(); + + if(dbTasks.size() == 0) { + return Optional.absent(); + } else { + final Map dbStatus = Iterables.getOnlyElement(dbTasks); + return Optional.of(jsonMapper.readValue(dbStatus.get("payload").toString(), Task.class)); + } + } + } + ); + } + @Override public Optional getStatus(final String taskid) { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index fc03504792f..edc3a6ae314 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -22,6 +22,7 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Throwables; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.task.Task; @@ -70,7 +71,7 @@ public class LocalTaskRunner implements TaskRunner TaskStatus status; try { - status = task.run(context, toolbox); + status = task.run(context, toolbox, callback); } catch (InterruptedException e) { log.error(e, "Interrupted while running task[%s]", task); @@ -97,9 +98,9 @@ public class LocalTaskRunner implements TaskRunner try { callback.notify(status.withDuration(System.currentTimeMillis() - startTime)); - } catch(Throwable t) { - log.error(t, "Uncaught Throwable during callback for task[%s]", task); - throw Throwables.propagate(t); + } catch(Exception e) { + log.error(e, "Uncaught Exception during callback for task[%s]", task); + throw Throwables.propagate(e); } } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java index 16ecd06073f..a4055a637e3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java @@ -59,6 +59,17 @@ public class LocalTaskStorage implements TaskStorage tasks.put(task.getId(), new TaskStuff(task, status)); } + @Override + public Optional getTask(String taskid) + { + Preconditions.checkNotNull(taskid, "taskid"); + if(tasks.containsKey(taskid)) { + return Optional.of(tasks.get(taskid).task); + } else { + return Optional.absent(); + } + } + @Override public void setStatus(String taskid, TaskStatus status) { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 5537a6b6420..ce24300c442 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -33,6 +33,7 @@ import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.PeriodGranularity; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskHolder; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java index 85b2ef62a04..0287f93a1bc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java @@ -19,16 +19,15 @@ package com.metamx.druid.merger.coordinator; -import com.google.common.base.Function; import com.google.common.base.Objects; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.commit.CommitStyle; +import com.metamx.druid.merger.coordinator.commit.ImmediateCommitStyle; import org.joda.time.Interval; -import java.util.Set; -import java.util.TreeSet; +import java.util.Map; /** * Represents a transaction as well as the lock it holds. Not immutable: the task set can change. @@ -39,16 +38,7 @@ public class TaskGroup private final String dataSource; private final Interval interval; private final String version; - private final Set taskSet = new TreeSet( - new Ordering() - { - @Override - public int compare(Task task, Task task1) - { - return task.getId().compareTo(task1.getId()); - } - }.nullsFirst() - ); + private final Map taskMap = Maps.newHashMap(); public TaskGroup(String groupId, String dataSource, Interval interval, String version) { @@ -78,9 +68,40 @@ public class TaskGroup return version; } - public Set getTaskSet() + public CommitStyle getCommitStyle() { - return taskSet; + // TODO -- should be configurable + return new ImmediateCommitStyle(); + } + + public int size() { + return taskMap.size(); + } + + public boolean add(final Task task) { + Preconditions.checkArgument( + task.getGroupId().equals(groupId), + "Task group id[%s] != TaskGroup group id[%s]", + task.getGroupId(), + groupId + ); + + // Act sort of like a Set + if(taskMap.containsKey(task.getId())) { + return false; + } else { + taskMap.put(task.getId(), task); + return true; + } + } + + public boolean contains(final String taskId) { + return taskMap.containsKey(taskId); + } + + public Task remove(final String taskId) + { + return taskMap.remove(taskId); } @Override @@ -91,21 +112,7 @@ public class TaskGroup .add("dataSource", dataSource) .add("interval", interval) .add("version", version) - .add( - "taskSet", - Lists.newArrayList( - Iterables.transform( - taskSet, new Function() - { - @Override - public Object apply(Task task) - { - return task.getId(); - } - } - ) - ) - ) + .add("tasks", taskMap.keySet()) .toString(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java index f674b8744cd..e43602a8226 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java @@ -23,20 +23,19 @@ import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import com.google.common.collect.HashMultimap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; -import com.google.common.primitives.Booleans; -import com.metamx.common.Pair; import com.metamx.common.guava.Comparators; import com.metamx.common.guava.FunctionalIterable; 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.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; +import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -61,7 +60,7 @@ import java.util.concurrent.locks.ReentrantLock; * only one TaskGroup can be running on a particular dataSource + interval, and that TaskGroup has a single version * string that all tasks in the group must use to publish segments. Tasks in the same TaskGroup may run concurrently. * - * For persistence, the queue saves new tasks from {@link #add} and task status updates from {@link #done} using a + * For persistence, the queue saves new tasks from {@link #add} and task status updates from {@link #notify} using a * {@link TaskStorage} object. * * To support leader election of our containing system, the queue can be stopped (in which case it will not accept @@ -71,6 +70,8 @@ public class TaskQueue { private final List queue = Lists.newLinkedList(); private final Map> running = Maps.newHashMap(); + private final Multimap seenNextTasks = HashMultimap.create(); + private final TaskStorage taskStorage; private final ReentrantLock giant = new ReentrantLock(); @@ -78,7 +79,7 @@ public class TaskQueue private volatile boolean active = false; - private static final Logger log = new Logger(TaskQueue.class); + private static final EmittingLogger log = new EmittingLogger(TaskQueue.class); public TaskQueue(TaskStorage taskStorage) { @@ -100,36 +101,24 @@ public class TaskQueue Preconditions.checkState(queue.isEmpty(), "queue must be empty!"); Preconditions.checkState(running.isEmpty(), "running list must be empty!"); - // XXX - We might want a TaskStorage API that does this, but including the Pair type in the interface seems clumsy. - final List> runningTasks = Lists.transform( - taskStorage.getRunningTasks(), - new Function>() - { - @Override - public Pair apply(Task task) - { - return Pair.of(task, taskStorage.getVersion(task.getId()).orNull()); - } - } - ); + final TaskStorageQueryAdapter taskStorageQueryAdapter = new TaskStorageQueryAdapter(taskStorage); + final List runningTasks = taskStorageQueryAdapter.getRunningTaskVersions(); // Sort by version, with nulls last - final Ordering> byVersionOrdering = new Ordering>() + final Ordering byVersionOrdering = new Ordering() { final private Ordering baseOrdering = Ordering.natural().nullsLast(); @Override - public int compare( - Pair left, Pair right - ) + public int compare(VersionedTaskWrapper left, VersionedTaskWrapper right) { - return baseOrdering.compare(left.rhs, right.rhs); + return baseOrdering.compare(left.getVersion(), right.getVersion()); } }; - for(final Pair taskAndVersion : byVersionOrdering.sortedCopy(runningTasks)) { - final Task task = taskAndVersion.lhs; - final String preferredVersion = taskAndVersion.rhs; + for(final VersionedTaskWrapper taskAndVersion : byVersionOrdering.sortedCopy(runningTasks)) { + final Task task = taskAndVersion.getTask(); + final String preferredVersion = taskAndVersion.getVersion(); queue.add(task); @@ -169,6 +158,8 @@ public class TaskQueue try { + log.info("Naptime!"); + queue.clear(); running.clear(); active = false; @@ -184,21 +175,22 @@ public class TaskQueue * @param task task to add * @return true */ - public boolean add(Task task) + public boolean add(final Task task) { giant.lock(); try { Preconditions.checkState(active, "Queue is not active!"); + // If this throws, we don't want to insert the task into our queue. + // (This is how we detect duplicates) taskStorage.insert(task, TaskStatus.running(task.getId())); queue.add(task); workMayBeAvailable.signalAll(); return true; - } - finally { + } finally { giant.unlock(); } } @@ -254,6 +246,107 @@ public class TaskQueue } } + public void notify(final Task task, final TaskStatus status) + { + notify(task, status, null); + } + + /** + * Notify this queue that some task has an updated status. If this update is valid, the status will be persisted in + * the task storage facility, and any nextTasks present in the status will be created. If the status is a completed + * status, the task will be unlocked and no further updates will be accepted. If this task has failed, the task group + * it is part of will be terminated. + * + * Finally, if this task is not supposed to be running, this method will simply do nothing. + * + * @param task task to update + * @param status new task status + * @param commitRunnable operation to perform if this task is ready to commit + * @throws NullPointerException if task or status is null + * @throws IllegalArgumentException if the task ID does not match the status ID + * @throws IllegalStateException if this queue is currently shut down + */ + public void notify(final Task task, final TaskStatus status, final Runnable commitRunnable) + { + giant.lock(); + + try { + Preconditions.checkNotNull(task, "task"); + Preconditions.checkNotNull(status, "status"); + Preconditions.checkState(active, "Queue is not active!"); + Preconditions.checkArgument( + task.getId().equals(status.getId()), + "Mismatching task ids[%s/%s]", + task.getId(), + status.getId() + ); + + final TaskGroup taskGroup; + + final Optional maybeTaskGroup = findTaskGroupForTask(task); + if(!maybeTaskGroup.isPresent()) { + log.info("Ignoring notification for dead task: %s", task.getId()); + return; + } else { + taskGroup = maybeTaskGroup.get(); + } + + // Update status in DB + // TODO: We can either do this first, in which case we run the risk of having a task marked done in the DB but + // TODO: not committed here; or we can do it last, in which case we run the risk of having a task marked running + // TODO: in the DB but committed here. Currently, we err on the former side because we don't want a ticking time + // TODO: bomb in the DB (a task marked running that we have forgotten about, which will potentially be re- + // TODO: started much later when a coordinator bootstraps). + // TODO: + // TODO: Eventually we should have this status update enter a retry queue instead of throwing an exception + // TODO: if it fails. + taskStorage.setStatus(task.getId(), status); + + // Should we commit? + if(taskGroup.getCommitStyle().shouldCommit(task, status)) { + log.info("Committing %s status for task: %s", status.getStatusCode(), task.getId()); + + // Add next tasks + try { + if(commitRunnable != null) { + log.info("Running commitRunnable for task: %s", task.getId()); + commitRunnable.run(); + } + + // We want to allow tasks to submit RUNNING statuses with the same nextTasks over and over. + // So, we need to remember which ones we've already spawned and not do them again. + for(final Task nextTask : status.getNextTasks()) { + if(!seenNextTasks.containsEntry(task.getId(), nextTask.getId())) { + add(nextTask); + tryLock(nextTask); + seenNextTasks.put(task.getId(), nextTask.getId()); + } else { + log.info("Already added followup task %s to original task: %s", nextTask.getId(), task.getId()); + } + } + } catch(Exception e) { + log.makeAlert(e, "Failed to commit task") + .addData("task", task.getId()) + .addData("statusCode", status.getStatusCode()) + .emit(); + + // TODO -- If this fails, it should enter a retry queue instead of throwing an exception + taskStorage.setStatus(task.getId(), TaskStatus.failure(task.getId()).withDuration(status.getDuration())); + } + } else { + log.info("Not committing %s status for task: %s", status.getStatusCode(), task); + } + + if(status.isComplete()) { + unlock(task); + seenNextTasks.removeAll(task.getId()); + log.info("Task done: %s", task); + } + } finally { + giant.unlock(); + } + } + /** * Unlock some work. Does not update the task storage facility. Throws an exception if this work is not currently * running. @@ -267,34 +360,21 @@ public class TaskQueue try { final String dataSource = task.getDataSource(); - final Interval interval = task.getInterval(); - - final List maybeTaskGroup = Lists.newArrayList( - FunctionalIterable.create(findLocks(dataSource, interval)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskGroup taskGroup) - { - return taskGroup.getTaskSet().contains(task); - } - } - ) - ); final TaskGroup taskGroup; - if(maybeTaskGroup.size() == 1) { - taskGroup = maybeTaskGroup.get(0); + final Optional maybeTaskGroup = findTaskGroupForTask(task); + + if(maybeTaskGroup.isPresent()) { + taskGroup = maybeTaskGroup.get(); } else { throw new IllegalStateException(String.format("Task must be running: %s", task.getId())); } // Remove task from live list log.info("Removing task[%s] from TaskGroup[%s]", task.getId(), taskGroup.getGroupId()); - taskGroup.getTaskSet().remove(task); + taskGroup.remove(task.getId()); - if(taskGroup.getTaskSet().size() == 0) { + if(taskGroup.size() == 0) { log.info("TaskGroup complete: %s", taskGroup); running.get(dataSource).remove(taskGroup.getInterval()); } @@ -310,111 +390,6 @@ public class TaskQueue } } - /** - * Unlock some task and update its status in the task storage facility. If "status" is a continuation status (i.e. - * it has nextTasks) this will add the next tasks to the queue with a generic running status. - * - * @param task task to unlock - * @param status task completion status; must not be runnable - * @throws IllegalStateException if task is not currently running, or if status is runnable - */ - public void done(final Task task, final TaskStatus status) - { - giant.lock(); - - try { - Preconditions.checkState(active, "Queue is not active!"); - Preconditions.checkState(!status.isRunnable(), "status must no longer be runnable"); - - Preconditions.checkState( - task.getId().equals(status.getId()), - "Mismatching task ids[%s/%s]", - task.getId(), - status.getId() - ); - - // Might change on continuation failure - TaskStatus actualStatus = status; - - // Add next tasks, if any - try { - for(final Task nextTask : status.getNextTasks()) { - add(nextTask); - tryLock(nextTask); - } - } catch(Exception e) { - log.error(e, "Failed to continue task: %s", task.getId()); - actualStatus = TaskStatus.failure(task.getId()); - } - - unlock(task); - - // Update status in DB - taskStorage.setStatus(task.getId(), actualStatus); - - log.info("Task done: %s", task); - } - finally { - giant.unlock(); - } - } - - /** - * Returns task status for a particular task ID. May collapse "continued" statuses down to "success" or "failure" - * if appropriate. - */ - public Optional getStatus(final String taskid) - { - giant.lock(); - - try { - final Optional statusOptional = taskStorage.getStatus(taskid); - if(statusOptional.isPresent()) { - // See if we can collapse this down - return Optional.of(collapseStatus(statusOptional.get())); - } else { - return statusOptional; - } - } - finally { - giant.unlock(); - } - } - - private TaskStatus collapseStatus(TaskStatus status) - { - - if (status.isContinued()) { - - int nSubtasks = 0; - int nSuccesses = 0; - List segments = Lists.newArrayList(); - - for(final Task subtask : status.getNextTasks()) { - - final TaskStatus subtaskStatus = collapseStatus(taskStorage.getStatus(subtask.getId()).get()); - nSubtasks ++; - - if (subtaskStatus.isFailure()) { - return TaskStatus.failure(status.getId()); - } else if (subtaskStatus.isSuccess()) { - nSuccesses++; - segments.addAll(subtaskStatus.getSegments()); - } - - } - - if (nSubtasks == nSuccesses) { - return TaskStatus.success(status.getId(), segments); - } - - } - - // unable to collapse it down - return status; - - } - /** * Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task. * @@ -442,7 +417,7 @@ public class TaskQueue final String dataSource = task.getDataSource(); final Interval interval = task.getInterval(); - final List foundLocks = findLocks(dataSource, interval); + final List foundLocks = findTaskGroupsForInterval(dataSource, interval); final TaskGroup taskGroupToUse; if (foundLocks.size() > 1) { @@ -495,7 +470,7 @@ public class TaskQueue } // Add to existing TaskGroup, if necessary - if (taskGroupToUse.getTaskSet().add(task)) { + if (taskGroupToUse.add(task)) { log.info("Added task[%s] to TaskGroup[%s]", task.getId(), taskGroupToUse.getGroupId()); } else { log.info("Task[%s] already present in TaskGroup[%s]", task.getId(), taskGroupToUse.getGroupId()); @@ -509,10 +484,47 @@ public class TaskQueue } + /** + * Return the currently-running task group for some task. If the task has no currently-running task group, this will + * return an absentee Optional. + * + * @param task task for which to locate group + */ + private Optional findTaskGroupForTask(final Task task) + { + giant.lock(); + + try { + final List maybeTaskGroup = Lists.newArrayList( + FunctionalIterable.create(findTaskGroupsForInterval(task.getDataSource(), task.getInterval())) + .filter( + new Predicate() + { + @Override + public boolean apply(TaskGroup taskGroup) + { + return taskGroup.contains(task.getId()); + } + } + ) + ); + + if(maybeTaskGroup.size() == 1) { + return Optional.of(maybeTaskGroup.get(0)); + } else if(maybeTaskGroup.size() == 0) { + return Optional.absent(); + } else { + throw new IllegalStateException(String.format("WTF?! Task %s is in multiple task groups!", task.getId())); + } + } finally { + giant.unlock(); + } + } + /** * Return all locks that overlap some search interval. */ - private List findLocks(final String dataSource, final Interval interval) + private List findTaskGroupsForInterval(final String dataSource, final Interval interval) { giant.lock(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java index 5362741ac92..826a3b639cb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.task.Task; /** @@ -28,7 +29,8 @@ import com.metamx.druid.merger.common.task.Task; public interface TaskRunner { /** - * Run a task with a particular context and call a callback. The callback should be called exactly once. + * Run a task with a particular context and call a callback. The callback may be called multiple times with RUNNING + * status, but should be called exactly once with a non-RUNNING status (e.g. SUCCESS, FAILED, CONTINUED...). * * @param task task to run * @param context task context to run under diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java index 13308db86a2..258fd22444c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java @@ -44,6 +44,14 @@ public interface TaskStorage */ public void setVersion(String taskid, String version); + /** + * Returns task as stored in the storage facility. If the task ID does not exist, this will return an + * absentee Optional. + * + * TODO -- This method probably wants to be combined with {@link #getStatus}. + */ + public Optional getTask(String taskid); + /** * Returns task status as stored in the storage facility. If the task ID does not exist, this will return * an absentee Optional. diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java new file mode 100644 index 00000000000..2c9dbc6181b --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java @@ -0,0 +1,145 @@ +/* + * 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.merger.coordinator; + +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.task.Task; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Wraps a {@link TaskStorage}, providing a useful collection of read-only methods. + */ +public class TaskStorageQueryAdapter +{ + private final TaskStorage storage; + + public TaskStorageQueryAdapter(TaskStorage storage) + { + this.storage = storage; + } + + public Optional getStatus(final String taskid) + { + return storage.getStatus(taskid); + } + + /** + * Returns all recursive task statuses for a particular task from the same task group. Includes that task, + * plus any tasks it spawned, and so on. Excludes spawned tasks that ended up in a different task group. + */ + public Map> getGroupRecursiveStatuses(final String taskid) + { + final Optional taskOptional = storage.getTask(taskid); + final Optional statusOptional = storage.getStatus(taskid); + final ImmutableMap.Builder> resultBuilder = ImmutableMap.builder(); + + resultBuilder.put(taskid, statusOptional); + + if(taskOptional.isPresent() && statusOptional.isPresent()) { + for(final Task nextTask : statusOptional.get().getNextTasks()) { + if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) { + resultBuilder.putAll(getGroupRecursiveStatuses(nextTask.getId())); + } + } + } + + return resultBuilder.build(); + } + + /** + * Like {@link #getGroupRecursiveStatuses}, but flattens all recursive statuses for the same task group into a + * single, merged status. + */ + public Optional getGroupMergedStatus(final String taskid) + { + final Map> statuses = getGroupRecursiveStatuses(taskid); + + int nSuccesses = 0; + int nFailures = 0; + int nTotal = 0; + + final Set segments = Sets.newHashSet(); + final Set segmentsNuked = Sets.newHashSet(); + final List nextTasks = Lists.newArrayList(); + + for(final Optional statusOption : statuses.values()) { + nTotal ++; + + if(statusOption.isPresent()) { + final TaskStatus status = statusOption.get(); + + segments.addAll(status.getSegments()); + segmentsNuked.addAll(status.getSegmentsNuked()); + nextTasks.addAll(status.getNextTasks()); + + if(status.isSuccess()) { + nSuccesses ++; + } else if(status.isFailure()) { + nFailures ++; + } + } + } + + final Optional status; + + if(nTotal == 0) { + status = Optional.absent(); + } else if(nSuccesses == nTotal) { + status = Optional.of(TaskStatus.success(taskid) + .withSegments(segments) + .withSegmentsNuked(segmentsNuked) + .withNextTasks(nextTasks)); + } else if(nFailures > 0) { + status = Optional.of(TaskStatus.failure(taskid)); + } else { + status = Optional.of(TaskStatus.running(taskid)); + } + + return status; + } + + /** + * Returns running tasks along with their preferred versions. If no version is present for a task, the + * version field of the returned {@link VersionedTaskWrapper} will be null. + */ + public List getRunningTaskVersions() + { + return Lists.transform( + storage.getRunningTasks(), + new Function() + { + @Override + public VersionedTaskWrapper apply(Task task) + { + return new VersionedTaskWrapper(task, storage.getVersion(task.getId()).orNull()); + } + } + ); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java index c757bb2dc33..22e7d4152b4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.task.Task; /** diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/CommitStyle.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/CommitStyle.java new file mode 100644 index 00000000000..4f9c2e4a989 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/CommitStyle.java @@ -0,0 +1,32 @@ +/* + * 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.merger.coordinator.commit; + +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.TaskGroup; + +/** + * Determines whether or not metadata from a task status update should be committed. + */ +public interface CommitStyle +{ + public boolean shouldCommit(Task task, TaskStatus taskStatus); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/ImmediateCommitStyle.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/ImmediateCommitStyle.java new file mode 100644 index 00000000000..56d29533cd3 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/ImmediateCommitStyle.java @@ -0,0 +1,33 @@ +/* + * 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.merger.coordinator.commit; + +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.TaskGroup; + +public class ImmediateCommitStyle implements CommitStyle +{ + @Override + public boolean shouldCommit(Task task, TaskStatus taskStatus) + { + return true; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/TaskCommitStyle.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/TaskCommitStyle.java new file mode 100644 index 00000000000..28c7d6fc3ca --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/TaskCommitStyle.java @@ -0,0 +1,32 @@ +/* + * 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.merger.coordinator.commit; + +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.task.Task; + +public class TaskCommitStyle implements CommitStyle +{ + @Override + public boolean shouldCommit(Task task, TaskStatus taskStatus) + { + return taskStatus.isSuccess(); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index 78326d3a3cc..02d93351a31 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -27,7 +27,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.MergerDBCoordinator; -import com.metamx.druid.merger.coordinator.TaskCallback; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskRunner; @@ -108,7 +108,7 @@ public class TaskConsumer implements Runnable // Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now. if(!shutdown) { - queue.done(task, TaskStatus.failure(task.getId())); + queue.notify(task, TaskStatus.failure(task.getId())); } } } @@ -148,111 +148,116 @@ public class TaskConsumer implements Runnable if (!preflightStatus.isRunnable()) { log.info("Task finished during preflight: %s", task.getId()); - queue.done(task, preflightStatus); + queue.notify(task, preflightStatus); return; } // Hand off work to TaskRunner + // TODO -- Should something in the TaskCallback enforce that each returned status is logically after the previous? + // TODO -- Probably yes. But make sure it works in the face of RTR retries. runner.run( task, context, new TaskCallback() { @Override public void notify(final TaskStatus statusFromRunner) { - - // task is done - log.info("TaskRunner finished task: %s", task); - - // we might need to change this due to exceptions - TaskStatus status = statusFromRunner; - - // If we're not supposed to be running anymore, don't do anything. Somewhat racey if the flag gets set after - // we check and before we commit the database transaction, but better than nothing. - if(shutdown) { - log.info("Abandoning task due to shutdown: %s", task.getId()); - return; - } - - // Publish returned segments - // FIXME: Publish in transaction try { - for (DataSegment segment : status.getSegments()) { - if (!task.getDataSource().equals(segment.getDataSource())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid dataSource: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } + log.info("Received %s status for task: %s", statusFromRunner.getStatusCode(), task); - if (!task.getInterval().contains(segment.getInterval())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid interval: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - - if (!context.getVersion().equals(segment.getVersion())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid version: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - - log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), task.getId()); - mergerDBCoordinator.announceHistoricalSegment(segment); + // If we're not supposed to be running anymore, don't do anything. Somewhat racey if the flag gets set after + // we check and before we commit the database transaction, but better than nothing. + if(shutdown) { + log.info("Abandoning task due to shutdown: %s", task.getId()); + return; } - } - catch (Exception e) { - log.error(e, "Exception while publishing segments for task: %s", task); - status = TaskStatus.failure(task.getId()).withDuration(status.getDuration()); - } - try { - queue.done(task, status); - } - catch (Exception e) { - log.error(e, "Exception while marking task done: %s", task); - throw Throwables.propagate(e); - } + queue.notify(task, statusFromRunner, new Runnable() + { + @Override + public void run() + { + try { + // Publish returned segments + // TODO -- Publish in transaction + if(statusFromRunner.getSegments().size() > 0) { + for (DataSegment segment : statusFromRunner.getSegments()) { + if (!task.getDataSource().equals(segment.getDataSource())) { + throw new IllegalStateException( + String.format( + "Segment for task[%s] has invalid dataSource: %s", + task.getId(), + segment.getIdentifier() + ) + ); + } - // emit event and log - int bytes = 0; - for (DataSegment segment : status.getSegments()) { - bytes += segment.getSize(); - } + if (!task.getInterval().contains(segment.getInterval())) { + throw new IllegalStateException( + String.format( + "Segment for task[%s] has invalid interval: %s", + task.getId(), + segment.getIdentifier() + ) + ); + } - builder.setUser3(status.getStatusCode().toString()); + if (!context.getVersion().equals(segment.getVersion())) { + throw new IllegalStateException( + String.format( + "Segment for task[%s] has invalid version: %s", + task.getId(), + segment.getIdentifier() + ) + ); + } - emitter.emit(builder.build("indexer/time/run/millis", status.getDuration())); - emitter.emit(builder.build("indexer/segment/count", status.getSegments().size())); - emitter.emit(builder.build("indexer/segment/bytes", bytes)); + log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), task.getId()); + mergerDBCoordinator.announceHistoricalSegment(segment); + } + } + } catch(Exception e) { + log.error(e, "Exception while publishing segments for task: %s", task); + throw Throwables.propagate(e); + } + } + }); - if (status.isFailure()) { - log.makeAlert("Failed to index") + // Emit event and log, if the task is done + if(statusFromRunner.isComplete()) { + int segmentBytes = 0; + for (DataSegment segment : statusFromRunner.getSegments()) { + segmentBytes += segment.getSize(); + } + + builder.setUser3(statusFromRunner.getStatusCode().toString()); + + emitter.emit(builder.build("indexer/time/run/millis", statusFromRunner.getDuration())); + emitter.emit(builder.build("indexer/segment/count", statusFromRunner.getSegments().size())); + emitter.emit(builder.build("indexer/segment/bytes", segmentBytes)); + + if (statusFromRunner.isFailure()) { + log.makeAlert("Failed to index") + .addData("task", task.getId()) + .addData("type", task.getType().toString()) + .addData("dataSource", task.getDataSource()) + .addData("interval", task.getInterval()) + .emit(); + } + + log.info( + "Task %s: %s (%d segments) (%d run duration)", + statusFromRunner.getStatusCode(), + task, + statusFromRunner.getSegments().size(), + statusFromRunner.getDuration() + ); + } + } catch(Exception e) { + log.makeAlert(e, "Failed to handle task callback") .addData("task", task.getId()) - .addData("type", task.getType().toString()) - .addData("dataSource", task.getDataSource()) - .addData("interval", task.getInterval()) + .addData("statusCode", statusFromRunner.getStatusCode()) .emit(); } - - log.info( - "Task %s: %s (%d segments) (%d run duration)", - status.getStatusCode(), - task, - status.getSegments().size(), - status.getDuration() - ); - } } ); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 3dae4046764..3e068c7b199 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -64,6 +64,7 @@ import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskRunner; import com.metamx.druid.merger.coordinator.TaskRunnerFactory; import com.metamx.druid.merger.coordinator.TaskStorage; +import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; @@ -235,6 +236,7 @@ public class IndexerCoordinatorNode extends RegisteringNode config, emitter, taskQueue, + new TaskStorageQueryAdapter(taskStorage), workerSetupManager ) ); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index e4acd93514f..7465b1c231e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -27,6 +27,7 @@ import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.MergeTask; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskQueue; +import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; @@ -50,6 +51,7 @@ public class IndexerCoordinatorResource private final IndexerCoordinatorConfig config; private final ServiceEmitter emitter; private final TaskQueue tasks; + private final TaskStorageQueryAdapter taskStorageQueryAdapter; private final WorkerSetupManager workerSetupManager; @Inject @@ -57,6 +59,7 @@ public class IndexerCoordinatorResource IndexerCoordinatorConfig config, ServiceEmitter emitter, TaskQueue tasks, + TaskStorageQueryAdapter taskStorageQueryAdapter, WorkerSetupManager workerSetupManager ) throws Exception @@ -64,6 +67,7 @@ public class IndexerCoordinatorResource this.config = config; this.emitter = emitter; this.tasks = tasks; + this.taskStorageQueryAdapter = taskStorageQueryAdapter; this.workerSetupManager = workerSetupManager; } @@ -108,7 +112,7 @@ public class IndexerCoordinatorResource @Produces("application/json") public Response doStatus(@PathParam("taskid") String taskid) { - final Optional status = tasks.getStatus(taskid); + final Optional status = taskStorageQueryAdapter.getGroupMergedStatus(taskid); if (!status.isPresent()) { return Response.status(Response.Status.NOT_FOUND).build(); } else { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java index 4cc1df9fa6f..51c44baab15 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java @@ -21,6 +21,7 @@ package com.metamx.druid.merger.coordinator.http; import com.google.inject.Provides; import com.metamx.druid.merger.coordinator.TaskQueue; +import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; @@ -39,6 +40,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule private final IndexerCoordinatorConfig indexerCoordinatorConfig; private final ServiceEmitter emitter; private final TaskQueue tasks; + private final TaskStorageQueryAdapter taskStorageQueryAdapter; private final WorkerSetupManager workerSetupManager; public IndexerCoordinatorServletModule( @@ -46,6 +48,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule IndexerCoordinatorConfig indexerCoordinatorConfig, ServiceEmitter emitter, TaskQueue tasks, + TaskStorageQueryAdapter taskStorageQueryAdapter, WorkerSetupManager workerSetupManager ) { @@ -53,6 +56,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule this.indexerCoordinatorConfig = indexerCoordinatorConfig; this.emitter = emitter; this.tasks = tasks; + this.taskStorageQueryAdapter = taskStorageQueryAdapter; this.workerSetupManager = workerSetupManager; } @@ -64,6 +68,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule bind(IndexerCoordinatorConfig.class).toInstance(indexerCoordinatorConfig); bind(ServiceEmitter.class).toInstance(emitter); bind(TaskQueue.class).toInstance(tasks); + bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter); bind(WorkerSetupManager.class).toInstance(workerSetupManager); serve("/*").with(GuiceContainer.class); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index 5fc49788fcd..9c815dd1f27 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -21,6 +21,7 @@ package com.metamx.druid.merger.worker; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskHolder; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; @@ -110,7 +111,16 @@ public class TaskMonitor try { workerCuratorCoordinator.unannounceTask(task.getId()); workerCuratorCoordinator.announceStatus(TaskStatus.running(task.getId())); - taskStatus = task.run(taskContext, toolbox); + taskStatus = task.run( + taskContext, toolbox, new TaskCallback() + { + @Override + public void notify(TaskStatus status) + { + workerCuratorCoordinator.announceStatus(status); + } + } + ); } catch (Exception e) { log.makeAlert(e, "Failed to run task") diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 0799a8de37c..c803b7294f1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -96,7 +96,7 @@ public class WorkerNode extends RegisteringNode private List monitors = null; private ServiceEmitter emitter = null; - private IndexerCoordinatorConfig coordinatorConfig = null; // FIXME needed for task toolbox, but shouldn't be + private IndexerCoordinatorConfig coordinatorConfig = null; // TODO needed for task toolbox, but shouldn't be private WorkerConfig workerConfig = null; private TaskToolbox taskToolbox = null; private CuratorFramework curatorFramework = null; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index 1e31efa121c..914b8fca74a 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -1,12 +1,14 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.ISE; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; @@ -191,7 +193,7 @@ public class RemoteTaskRunnerTest jsonMapper.writeValueAsBytes( TaskStatus.success( "task1", - Lists.newArrayList() + ImmutableSet.of() ) ) ); @@ -500,9 +502,9 @@ public class RemoteTaskRunnerTest } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { - return TaskStatus.success("task1", Lists.newArrayList()); + return TaskStatus.success("task1", ImmutableSet.of()); } } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 47913404eed..8c196001125 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -20,9 +20,11 @@ package com.metamx.druid.merger.coordinator; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.task.AbstractTask; @@ -40,20 +42,16 @@ public class TaskQueueTest @Test public void testEmptyQueue() throws Exception { - final TaskQueue tq = newTaskQueue(); + final TaskStorage ts = new LocalTaskStorage(); + final TaskQueue tq = newTaskQueueWithStorage(ts); // get task status for nonexistent task - Assert.assertFalse("getStatus", tq.getStatus("foo").isPresent()); + Assert.assertFalse("getStatus", ts.getStatus("foo").isPresent()); // poll on empty queue Assert.assertNull("poll", tq.poll()); } - private static TaskQueue newTaskQueue() - { - return newTaskQueueWithStorage(new LocalTaskStorage()); - } - public static TaskQueue newTaskQueueWithStorage(TaskStorage storage) { final TaskQueue tq = new TaskQueue(storage); @@ -64,7 +62,9 @@ public class TaskQueueTest @Test public void testAddRemove() throws Exception { - final TaskQueue tq = newTaskQueue(); + final TaskStorage ts = new LocalTaskStorage(); + final TaskQueue tq = newTaskQueueWithStorage(ts); + final Task[] tasks = { newTask("T0", "G0", "bar", new Interval("2011/P1Y")), newTask("T1", "G1", "bar", new Interval("2011-03-01/P1D")), @@ -81,7 +81,7 @@ public class TaskQueueTest } // get task status for in-progress task - Assert.assertEquals("T2 status (before finishing)", TaskStatus.Status.RUNNING, tq.getStatus(tasks[2].getId()).get().getStatusCode()); + Assert.assertEquals("T2 status (before finishing)", TaskStatus.Status.RUNNING, ts.getStatus(tasks[2].getId()).get().getStatusCode()); // Can't add tasks with the same id thrown = null; @@ -114,20 +114,23 @@ public class TaskQueueTest ); // mark one done - tq.done(tasks[2], tasks[2].run(null, null)); + final TestCommitRunnable commit1 = newCommitRunnable(); + tq.notify(tasks[2], tasks[2].run(null, null, null), commit1); // get its status back - Assert.assertEquals("T2 status (after finishing)", TaskStatus.Status.SUCCESS, tq.getStatus(tasks[2].getId()).get().getStatusCode()); + Assert.assertEquals( + "T2 status (after finishing)", + TaskStatus.Status.SUCCESS, + ts.getStatus(tasks[2].getId()).get().getStatusCode() + ); + + Assert.assertEquals("Commit #1 wasRun", commit1.wasRun(), true); // Can't do a task twice - thrown = null; - try { - tq.done(tasks[2], tasks[2].run(null, null)); - } catch(IllegalStateException e) { - thrown = e; - } + final TestCommitRunnable commit2 = newCommitRunnable(); + tq.notify(tasks[2], tasks[2].run(null, null, null), commit2); - Assert.assertNotNull("Exception on twice-done task", thrown); + Assert.assertEquals("Commit #2 wasRun", commit2.wasRun(), false); // we should be able to get one more task now taken.clear(); @@ -156,44 +159,45 @@ public class TaskQueueTest @Test public void testContinues() throws Exception { - final TaskQueue tq = newTaskQueue(); + final TaskStorage ts = new LocalTaskStorage(); + final TaskQueue tq = newTaskQueueWithStorage(ts); final Task t0 = newTask("T0", "G0", "bar", new Interval("2011/P1Y")); final Task t1 = newContinuedTask("T1", "G1", "bar", new Interval("2013/P1Y"), Lists.newArrayList(t0)); tq.add(t1); - Assert.assertTrue("T0 isPresent (#1)", !tq.getStatus("T0").isPresent()); - Assert.assertTrue("T1 isPresent (#1)", tq.getStatus("T1").isPresent()); - Assert.assertTrue("T1 isRunnable (#1)", tq.getStatus("T1").get().isRunnable()); - Assert.assertTrue("T1 isComplete (#1)", !tq.getStatus("T1").get().isComplete()); + Assert.assertTrue("T0 isPresent (#1)", !ts.getStatus("T0").isPresent()); + Assert.assertTrue("T1 isPresent (#1)", ts.getStatus("T1").isPresent()); + Assert.assertTrue("T1 isRunnable (#1)", ts.getStatus("T1").get().isRunnable()); + Assert.assertTrue("T1 isComplete (#1)", !ts.getStatus("T1").get().isComplete()); // should be able to get t1 out Assert.assertEquals("poll #1", "T1", tq.poll().getTask().getId()); Assert.assertNull("poll #2", tq.poll()); // report T1 done. Should cause T0 to be created - tq.done(t1, t1.run(null, null)); + tq.notify(t1, t1.run(null, null, null)); - Assert.assertTrue("T0 isPresent (#2)", tq.getStatus("T0").isPresent()); - Assert.assertTrue("T0 isRunnable (#2)", tq.getStatus("T0").get().isRunnable()); - Assert.assertTrue("T0 isComplete (#2)", !tq.getStatus("T0").get().isComplete()); - Assert.assertTrue("T1 isPresent (#2)", tq.getStatus("T1").isPresent()); - Assert.assertTrue("T1 isRunnable (#2)", !tq.getStatus("T1").get().isRunnable()); - Assert.assertTrue("T1 isComplete (#2)", tq.getStatus("T1").get().isComplete()); + Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent()); + Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable()); + Assert.assertTrue("T0 isComplete (#2)", !ts.getStatus("T0").get().isComplete()); + Assert.assertTrue("T1 isPresent (#2)", ts.getStatus("T1").isPresent()); + Assert.assertTrue("T1 isRunnable (#2)", !ts.getStatus("T1").get().isRunnable()); + Assert.assertTrue("T1 isComplete (#2)", ts.getStatus("T1").get().isComplete()); // should be able to get t0 out Assert.assertEquals("poll #3", "T0", tq.poll().getTask().getId()); Assert.assertNull("poll #4", tq.poll()); // report T0 done. Should cause T0, T1 to be marked complete - tq.done(t0, t0.run(null, null)); + tq.notify(t0, t0.run(null, null, null)); - Assert.assertTrue("T0 isPresent (#3)", tq.getStatus("T0").isPresent()); - Assert.assertTrue("T0 isRunnable (#3)", !tq.getStatus("T0").get().isRunnable()); - Assert.assertTrue("T0 isComplete (#3)", tq.getStatus("T0").get().isComplete()); - Assert.assertTrue("T1 isPresent (#3)", tq.getStatus("T1").isPresent()); - Assert.assertTrue("T1 isRunnable (#3)", !tq.getStatus("T1").get().isRunnable()); - Assert.assertTrue("T1 isComplete (#3)", tq.getStatus("T1").get().isComplete()); + Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent()); + Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable()); + Assert.assertTrue("T0 isComplete (#3)", ts.getStatus("T0").get().isComplete()); + Assert.assertTrue("T1 isPresent (#3)", ts.getStatus("T1").isPresent()); + Assert.assertTrue("T1 isRunnable (#3)", !ts.getStatus("T1").get().isRunnable()); + Assert.assertTrue("T1 isComplete (#3)", ts.getStatus("T1").get().isComplete()); // should be no more events available for polling Assert.assertNull("poll #5", tq.poll()); @@ -202,7 +206,8 @@ public class TaskQueueTest @Test public void testConcurrency() throws Exception { - final TaskQueue tq = newTaskQueue(); + final TaskStorage ts = new LocalTaskStorage(); + final TaskQueue tq = newTaskQueueWithStorage(ts); // Imagine a larger task that splits itself up into pieces final Task t1 = newTask("T1", "G0", "bar", new Interval("2011-01-01/P1D")); @@ -227,7 +232,7 @@ public class TaskQueueTest Thread.sleep(5); // Finish t0 - tq.done(t0, t0.run(null, null)); + tq.notify(t0, t0.run(null, null, null)); // take max number of tasks final Set taken = Sets.newHashSet(); @@ -254,11 +259,11 @@ public class TaskQueueTest Assert.assertEquals("taken", Sets.newHashSet("T1", "T3"), taken); // Finish t1 - tq.done(t1, t1.run(null, null)); + tq.notify(t1, t1.run(null, null, null)); Assert.assertNull("null poll #2", tq.poll()); // Finish t3 - tq.done(t3, t3.run(null, null)); + tq.notify(t3, t3.run(null, null, null)); // We should be able to get t2 now final VersionedTaskWrapper wt2 = tq.poll(); @@ -268,7 +273,7 @@ public class TaskQueueTest Assert.assertNull("null poll #3", tq.poll()); // Finish t2 - tq.done(t2, t2.run(null, null)); + tq.notify(t2, t2.run(null, null, null)); // We should be able to get t4 // And it should be in group G0, but that group should have a different version than last time @@ -281,7 +286,7 @@ public class TaskQueueTest Assert.assertNotSame("wt4 version", wt2.getVersion(), wt4.getVersion()); // Kind of done testing at this point, but let's finish t4 anyway - tq.done(t4, t4.run(null, null)); + tq.notify(t4, t4.run(null, null, null)); Assert.assertNull("null poll #4", tq.poll()); } @@ -299,7 +304,7 @@ public class TaskQueueTest Assert.assertEquals("vt1 id", "T1", vt1.getTask().getId()); Assert.assertEquals("vt1 version", "1234", vt1.getVersion()); - tq.done(vt1.getTask(), TaskStatus.success("T1", ImmutableList.of())); + tq.notify(vt1.getTask(), TaskStatus.success("T1", ImmutableSet.of())); // re-bootstrap tq.stop(); @@ -309,16 +314,123 @@ public class TaskQueueTest Assert.assertNull("null poll", tq.poll()); } + @Test + public void testRealtimeish() throws Exception + { + final TaskStorage ts = new LocalTaskStorage(); + final TaskQueue tq = newTaskQueueWithStorage(ts); + + class StructThingy + { + boolean pushed = false; + boolean pass1 = false; + boolean pass2 = false; + } + + final StructThingy structThingy = new StructThingy(); + + // Test a task that acts sort of like the realtime task, to make sure this case works. + final Task rtTask = new AbstractTask("id1", "ds", new Interval("2010-01-01T00:00:00Z/PT1H")) + { + @Override + public Type getType() + { + return Type.TEST; + } + + @Override + public TaskStatus run( + TaskContext context, TaskToolbox toolbox, TaskCallback callback + ) throws Exception + { + final Set segments = ImmutableSet.of( + DataSegment.builder() + .dataSource("ds") + .interval(new Interval("2010-01-01T00:00:00Z/PT1H")) + .version(context.getVersion()) + .build() + ); + + final List nextTasks = ImmutableList.of( + newTask( + "id2", + "id2", + "ds", + new Interval( + "2010-01-01T01:00:00Z/PT1H" + ) + ) + ); + + final TaskStatus status1 = TaskStatus.running("id1").withNextTasks(nextTasks); + final TaskStatus status2 = TaskStatus.running("id1").withNextTasks(nextTasks).withSegments(segments); + final TaskStatus status3 = TaskStatus.success("id1").withNextTasks(nextTasks).withSegments(segments); + + // Create a new realtime task! + callback.notify(status1); + if(ts.getStatus("id2").get().getStatusCode() == TaskStatus.Status.RUNNING) { + // test immediate creation of nextTask + structThingy.pass1 = true; + } + + // Hand off a segment! + callback.notify(status2); + if(structThingy.pushed) { + // test immediate handoff of segment + structThingy.pass2 = true; + } + + // Return success! + return status3; + } + }; + + tq.add(rtTask); + + final VersionedTaskWrapper vt = tq.poll(); + final TaskCallback callback = new TaskCallback() + { + @Override + public void notify(final TaskStatus status) + { + final Runnable commitRunnable = new Runnable() + { + @Override + public void run() + { + if(status.getNextTasks().size() > 0) { + structThingy.pushed = true; + } + } + }; + + tq.notify(vt.getTask(), status, commitRunnable); + } + }; + + callback.notify(vt.getTask().run(new TaskContext(vt.getVersion(), null), null, callback)); + + // OK, finally ready to test stuff. + Assert.assertTrue("pass1", structThingy.pass1); + Assert.assertTrue("pass2", structThingy.pass2); + Assert.assertTrue("id1 isSuccess", ts.getStatus("id1").get().isSuccess()); + Assert.assertTrue( + "id1 isSuccess (merged)", + new TaskStorageQueryAdapter(ts).getGroupMergedStatus("id1").get().isSuccess() + ); + Assert.assertTrue("id2 isRunnable", ts.getStatus("id2").get().isRunnable()); + } + private static Task newTask(final String id, final String groupId, final String dataSource, final Interval interval) { return new AbstractTask(id, groupId, dataSource, interval) { @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { return TaskStatus.success( id, - Lists.newArrayList( + ImmutableSet.of( new DataSegment( dataSource, interval, @@ -358,10 +470,31 @@ public class TaskQueueTest } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { - return TaskStatus.continued(id, nextTasks); + return TaskStatus.success(id).withNextTasks(nextTasks); } }; } + + private static TestCommitRunnable newCommitRunnable() + { + return new TestCommitRunnable(); + } + + private static class TestCommitRunnable implements Runnable + { + private boolean _wasRun = false; + + @Override + public void run() + { + _wasRun = true; + } + + public boolean wasRun() + { + return _wasRun; + } + } } From f3b04d3f5f21a5566acb35b881c6d730daea6a06 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 25 Jan 2013 11:33:50 -0800 Subject: [PATCH 05/63] S3SegmentKiller: Add TODO note --- .../src/main/java/com/metamx/druid/loading/S3SegmentKiller.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java index 801e16a1602..add1d45824e 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java @@ -52,6 +52,8 @@ public class S3SegmentKiller implements SegmentKiller @Override public List kill(final String datasource, final Interval interval) throws ServiceException { + // TODO -- Awkward for workers to use the DB! + List matchingSegments = dbi.withHandle( new HandleCallback>() { From 82d77a8b72432e9e7b5c128cb1e0e8561c69b586 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 25 Jan 2013 13:15:49 -0800 Subject: [PATCH 06/63] Index service: Fix kill task --- .../druid/merger/common/task/KillTask.java | 2 +- .../merger/coordinator/LocalTaskRunner.java | 8 +-- .../merger/coordinator/exec/TaskConsumer.java | 50 +++++++++++-------- 3 files changed, 36 insertions(+), 24 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index c361920f594..8c636c27a55 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -56,6 +56,6 @@ public class KillTask extends AbstractTask .kill(getDataSource(), getInterval()) ); - return TaskStatus.success(getId(), segmentsToKill); + return TaskStatus.success(getId()).withSegmentsNuked(segmentsToKill); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index edc3a6ae314..ddb07438f9a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -39,7 +39,7 @@ public class LocalTaskRunner implements TaskRunner private final TaskToolbox toolbox; private final ExecutorService exec; - private static final Logger log = new Logger(TaskQueue.class); + private static final Logger log = new Logger(LocalTaskRunner.class); public LocalTaskRunner( TaskToolbox toolbox, @@ -66,11 +66,11 @@ public class LocalTaskRunner implements TaskRunner public void run() { final long startTime = System.currentTimeMillis(); - final File taskDir = toolbox.getConfig().getTaskDir(task); TaskStatus status; try { + log.info("Running task: %s", task.getId()); status = task.run(context, toolbox, callback); } catch (InterruptedException e) { @@ -87,13 +87,15 @@ public class LocalTaskRunner implements TaskRunner } try { + final File taskDir = toolbox.getConfig().getTaskDir(task); + if (taskDir.exists()) { log.info("Removing task directory: %s", taskDir); FileUtils.deleteDirectory(taskDir); } } catch (Exception e) { - log.error(e, "Failed to delete task directory[%s]", taskDir.toString()); + log.error(e, "Failed to delete task directory: %s", task.getId()); } try { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index bded69c2f7d..8996ad8c0f7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -21,14 +21,13 @@ package com.metamx.druid.merger.coordinator.exec; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableSet; -import com.metamx.common.ISE; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.MergerDBCoordinator; -import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskRunner; @@ -37,7 +36,6 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import java.util.List; import java.util.Set; public class TaskConsumer implements Runnable @@ -96,7 +94,7 @@ public class TaskConsumer implements Runnable } catch (InterruptedException e) { log.info(e, "Interrupted while waiting for new work"); - throw Throwables.propagate(e); + throw e; } try { @@ -117,10 +115,10 @@ public class TaskConsumer implements Runnable } } } - catch (Throwable t) { + catch (Exception e) { // exit thread - log.error(t, "Uncaught Throwable while consuming tasks"); - throw Throwables.propagate(t); + log.error(e, "Uncaught exception while consuming tasks"); + throw Throwables.propagate(e); } } @@ -242,7 +240,18 @@ public class TaskConsumer implements Runnable private void deleteSegments(Task task, TaskContext context, Set segments) throws Exception { for (DataSegment segment : segments) { - verifySegment(task, context, segment); + verifyDataSourceAndInterval(task, context, segment); + + // Verify version (must be less than our context version) + if (segment.getVersion().compareTo(context.getVersion()) >= 0) { + throw new IllegalStateException( + String.format( + "Segment-to-nuke for task[%s] has invalid version: %s", + task.getId(), + segment.getIdentifier() + ) + ); + } log.info("Deleting segment[%s] for task[%s]", segment.getIdentifier(), task.getId()); mergerDBCoordinator.deleteSegment(segment); @@ -252,14 +261,25 @@ public class TaskConsumer implements Runnable private void publishSegments(Task task, TaskContext context, Set segments) throws Exception { for (DataSegment segment : segments) { - verifySegment(task, context, segment); + verifyDataSourceAndInterval(task, context, segment); + + // Verify version (must be equal to our context version) + if (!context.getVersion().equals(segment.getVersion())) { + throw new IllegalStateException( + String.format( + "Segment for task[%s] has invalid version: %s", + task.getId(), + segment.getIdentifier() + ) + ); + } log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), task.getId()); mergerDBCoordinator.announceHistoricalSegment(segment); } } - private void verifySegment(Task task, TaskContext context, DataSegment segment) + private void verifyDataSourceAndInterval(Task task, TaskContext context, DataSegment segment) { if (!task.getDataSource().equals(segment.getDataSource())) { throw new IllegalStateException( @@ -280,15 +300,5 @@ public class TaskConsumer implements Runnable ) ); } - - if (!context.getVersion().equals(segment.getVersion())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid version: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } } } From bb68091cef9d49eefcb3934bb939a1d557dbe422 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 25 Jan 2013 13:15:55 -0800 Subject: [PATCH 07/63] Index service: TaskConsumer tests --- .../merger/coordinator/TaskConsumerTest.java | 150 ++++++++++++++++++ 1 file changed, 150 insertions(+) create mode 100644 merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java new file mode 100644 index 00000000000..4d2d5947d75 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java @@ -0,0 +1,150 @@ +package com.metamx.druid.merger.coordinator; + +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskCallback; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.task.AbstractTask; +import com.metamx.druid.merger.coordinator.exec.TaskConsumer; +import com.metamx.emitter.core.Event; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceEventBuilder; +import junit.framework.Assert; +import org.joda.time.Interval; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Executors; + +public class TaskConsumerTest +{ + @Test + public void testSimple() + { + final TaskStorage ts = new LocalTaskStorage(); + final TaskQueue tq = new TaskQueue(ts); + final TaskRunner tr = new LocalTaskRunner( + new TaskToolbox(null, null, null, null, null, null), + Executors.newSingleThreadExecutor() + ); + + final MockMergerDBCoordinator mdc = newMockMDC(); + final TaskConsumer tc = new TaskConsumer(tq, tr, mdc, newMockEmitter()); + + tq.start(); + tc.start(); + + try { + tq.add(new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + { + @Override + public Type getType() + { + return Type.TEST; + } + + @Override + public TaskStatus run( + TaskContext context, TaskToolbox toolbox, TaskCallback callback + ) throws Exception + { + return TaskStatus.success(getId()).withSegments( + ImmutableSet.of( + DataSegment.builder() + .dataSource("ds") + .interval(new Interval("2012-01-01/P1D")) + .version(context.getVersion()) + .build() + ) + ); + } + }); + + while (ts.getStatus("id1").get().isRunnable()) { + Thread.sleep(100); + } + + final TaskStatus status = ts.getStatus("id1").get(); + Assert.assertTrue("nextTasks", status.getNextTasks().isEmpty()); + Assert.assertEquals("segments.size", 1, status.getSegments().size()); + Assert.assertEquals("segmentsNuked.size", 0, status.getSegmentsNuked().size()); + Assert.assertEquals("segments published", status.getSegments(), mdc.getPublished()); + Assert.assertEquals("segments nuked", status.getSegmentsNuked(), mdc.getNuked()); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + finally { + tc.stop(); + tq.stop(); + } + } + + private static class MockMergerDBCoordinator extends MergerDBCoordinator + { + final private Set published = Sets.newHashSet(); + final private Set nuked = Sets.newHashSet(); + + private MockMergerDBCoordinator() + { + super(null, null, null); + } + + @Override + public List getSegmentsForInterval(String dataSource, Interval interval) throws IOException + { + return ImmutableList.of(); + } + + @Override + public void announceHistoricalSegment(DataSegment segment) throws Exception + { + published.add(segment); + } + + @Override + public void deleteSegment(DataSegment segment) + { + nuked.add(segment); + } + + public Set getPublished() + { + return ImmutableSet.copyOf(published); + } + + public Set getNuked() + { + return ImmutableSet.copyOf(nuked); + } + } + + private MockMergerDBCoordinator newMockMDC() + { + return new MockMergerDBCoordinator(); + } + + private ServiceEmitter newMockEmitter() + { + return new ServiceEmitter(null, null, null) + { + @Override + public void emit(Event event) + { + + } + + @Override + public void emit(ServiceEventBuilder builder) + { + + } + }; + } +} From 0f4746450bf079593858e01fa7894cc61dbfde5a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 25 Jan 2013 13:40:20 -0800 Subject: [PATCH 08/63] RemoteTaskRunner: null check on workerData --- .../merger/coordinator/RemoteTaskRunner.java | 26 ++++++++++++------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 37bb3e9ca08..067356ceddc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -44,6 +44,7 @@ import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; +import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; @@ -286,21 +287,28 @@ public class RemoteTaskRunner implements TaskRunner try { log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskWrapper.getTask().getId()); - TaskStatus taskStatus = jsonMapper.readValue( - workerWrapper.getStatusCache() - .getCurrentData( - JOINER.join(config.getStatusPath(), worker.getHost(), taskWrapper.getTask().getId()) - ) - .getData(), - TaskStatus.class - ); + final ChildData workerData = workerWrapper.getStatusCache() + .getCurrentData( + JOINER.join( + config.getStatusPath(), + worker.getHost(), + taskWrapper.getTask().getId() + ) + ); + + if (workerData != null && workerData.getData() != null) { + final TaskStatus taskStatus = jsonMapper.readValue( + workerData.getData(), + TaskStatus.class + ); - if (taskStatus.isComplete()) { TaskCallback callback = taskWrapper.getCallback(); if (callback != null) { callback.notify(taskStatus); } new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()).run(); + } else { + log.warn("Worker data was null for worker: %s", worker.getHost()); } } catch (Exception e) { From efdff7b9f143d340bd27ac35a28658209fda47d5 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 25 Jan 2013 14:33:05 -0800 Subject: [PATCH 09/63] fix unit test bug --- .../druid/merger/coordinator/RemoteTaskRunner.java | 10 ++++++++++ .../merger/coordinator/RemoteTaskRunnerTest.java | 12 +++++++++++- 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 067356ceddc..fa6a7185058 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -264,6 +264,16 @@ public class RemoteTaskRunner implements TaskRunner return zkWorkers.size(); } + public boolean isTaskRunning(String taskId) + { + for (WorkerWrapper workerWrapper : zkWorkers.values()) { + if (workerWrapper.getRunningTasks().contains(taskId)) { + return true; + } + } + return false; + } + @Override public void run(Task task, TaskContext context, TaskCallback callback) { diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index 6f90029e6b1..4f4bc9fdfed 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -196,8 +196,13 @@ public class RemoteTaskRunnerTest ); // Really don't like this way of waiting for the task to appear - while (remoteTaskRunner.getNumWorkers() == 0) { + int count = 0; + while (!remoteTaskRunner.isTaskRunning("task1")) { Thread.sleep(500); + if (count > 10) { + throw new ISE("WTF?! Task still not announced in ZK?"); + } + count++; } final MutableBoolean callbackCalled = new MutableBoolean(false); @@ -364,8 +369,13 @@ public class RemoteTaskRunnerTest String.format("%s/worker1", announcementsPath), jsonMapper.writeValueAsBytes(worker1) ); + int count = 0; while (remoteTaskRunner.getNumWorkers() == 0) { Thread.sleep(500); + if (count > 10) { + throw new ISE("WTF?! Still can't find worker!"); + } + count++; } } From 1cff766e2a3c0709c8f0631748219a651c6e455b Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 25 Jan 2013 16:01:17 -0800 Subject: [PATCH 10/63] RemoteTaskRunner: Only cleanup paths during bootstrapping if task is complete --- .../metamx/druid/merger/coordinator/RemoteTaskRunner.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index fa6a7185058..a36bdb5a852 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -316,7 +316,10 @@ public class RemoteTaskRunner implements TaskRunner if (callback != null) { callback.notify(taskStatus); } - new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()).run(); + + if (taskStatus.isComplete()) { + new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()).run(); + } } else { log.warn("Worker data was null for worker: %s", worker.getHost()); } From 74057600f9d0fd7db69264815d44f903848471c4 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Sun, 27 Jan 2013 09:57:37 -0800 Subject: [PATCH 11/63] fix worker node still using db connection --- .../druid/merger/common/task/KillTask.java | 8 +- .../coordinator/MergerDBCoordinator.java | 59 ++++++++++++- .../merger/coordinator/RemoteTaskRunner.java | 12 +-- .../druid/merger/coordinator/TaskContext.java | 11 ++- .../merger/coordinator/exec/TaskConsumer.java | 27 ++++-- .../http/IndexerCoordinatorNode.java | 7 +- .../druid/merger/worker/http/WorkerNode.java | 16 +--- .../coordinator/RemoteTaskRunnerTest.java | 25 ++++-- .../merger/coordinator/TaskConsumerTest.java | 56 +++++++------ .../merger/coordinator/TaskQueueTest.java | 2 +- .../metamx/druid/loading/S3SegmentKiller.java | 83 +------------------ .../metamx/druid/loading/SegmentKiller.java | 4 +- 12 files changed, 159 insertions(+), 151 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index 8c636c27a55..eb5992925f0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -51,11 +51,9 @@ public class KillTask extends AbstractTask public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { // Kill segments - Set segmentsToKill = ImmutableSet.copyOf( - toolbox.getSegmentKiller() - .kill(getDataSource(), getInterval()) - ); + toolbox.getSegmentKiller() + .kill(context.getUnusedSegments()); - return TaskStatus.success(getId()).withSegmentsNuked(segmentsToKill); + return TaskStatus.success(getId()).withSegmentsNuked(context.getUnusedSegments()); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java index 828404d3961..6f72bff2da2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java @@ -20,6 +20,7 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Function; +import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.metamx.common.logger.Logger; @@ -31,12 +32,16 @@ import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; import org.joda.time.Interval; import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.FoldController; +import org.skife.jdbi.v2.Folder3; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.ResultIterator; +import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.tweak.HandleCallback; import javax.annotation.Nullable; import java.io.IOException; +import java.sql.SQLException; import java.util.List; import java.util.Map; @@ -63,7 +68,7 @@ public class MergerDBCoordinator this.dbi = dbi; } - public List getSegmentsForInterval(final String dataSource, final Interval interval) throws IOException + public List getUsedSegmentsForInterval(final String dataSource, final Interval interval) throws IOException { synchronized (lock) { @@ -209,6 +214,58 @@ public class MergerDBCoordinator } } ); + } + public List getUnusedSegmentsForInterval(final String dataSource, final Interval interval) + { + List matchingSegments = dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) throws Exception + { + return handle.createQuery( + String.format( + "SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and end <= :end and used = 0", + dbConnectorConfig.getSegmentTable() + ) + ) + .bind("dataSource", dataSource) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .fold( + Lists.newArrayList(), + new Folder3, Map>() + { + @Override + public List fold( + List accumulator, + Map stringObjectMap, + FoldController foldController, + StatementContext statementContext + ) throws SQLException + { + try { + DataSegment segment = jsonMapper.readValue( + (String) stringObjectMap.get("payload"), + DataSegment.class + ); + + accumulator.add(segment); + + return accumulator; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + } + } + ); + + log.info("Found %,d segments for %s for interval %s.", matchingSegments.size(), dataSource, interval); + return matchingSegments; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index a36bdb5a852..2e53c699e58 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -463,13 +463,13 @@ public class RemoteTaskRunner implements TaskRunner if (callback != null) { callback.notify(taskStatus); } - } - if (taskStatus.isComplete()) { - // Worker is done with this task - workerWrapper.setLastCompletedTaskTime(new DateTime()); - tasks.remove(taskId); - cf.delete().guaranteed().inBackground().forPath(event.getData().getPath()); + if (taskStatus.isComplete()) { + // Worker is done with this task + workerWrapper.setLastCompletedTaskTime(new DateTime()); + tasks.remove(taskId); + cf.delete().guaranteed().inBackground().forPath(event.getData().getPath()); + } } } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java index fcbb2450cbc..1d279fe0511 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java @@ -32,15 +32,18 @@ public class TaskContext { final String version; final Set currentSegments; + final Set unusedSegments; @JsonCreator public TaskContext( @JsonProperty("version") String version, - @JsonProperty("currentSegments") Set currentSegments + @JsonProperty("currentSegments") Set currentSegments, + @JsonProperty("unusedSegments") Set unusedSegments ) { this.version = version; this.currentSegments = currentSegments; + this.unusedSegments = unusedSegments; } @JsonProperty @@ -54,4 +57,10 @@ public class TaskContext { return currentSegments; } + + @JsonProperty + public Set getUnusedSegments() + { + return unusedSegments; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index 8996ad8c0f7..74c1b4fb46e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -109,7 +109,7 @@ public class TaskConsumer implements Runnable .emit(); // Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now. - if(!shutdown) { + if (!shutdown) { queue.notify(task, TaskStatus.failure(task.getId())); } } @@ -127,7 +127,13 @@ public class TaskConsumer implements Runnable final TaskContext context = new TaskContext( version, ImmutableSet.copyOf( - mergerDBCoordinator.getSegmentsForInterval( + mergerDBCoordinator.getUsedSegmentsForInterval( + task.getDataSource(), + task.getInterval() + ) + ), + ImmutableSet.copyOf( + mergerDBCoordinator.getUnusedSegmentsForInterval( task.getDataSource(), task.getInterval() ) @@ -169,23 +175,24 @@ public class TaskConsumer implements Runnable // If we're not supposed to be running anymore, don't do anything. Somewhat racey if the flag gets set after // we check and before we commit the database transaction, but better than nothing. - if(shutdown) { + if (shutdown) { log.info("Abandoning task due to shutdown: %s", task.getId()); return; } - queue.notify(task, statusFromRunner, new Runnable() + queue.notify( + task, statusFromRunner, new Runnable() { @Override public void run() { try { - if(statusFromRunner.getSegments().size() > 0) { + if (statusFromRunner.getSegments().size() > 0) { // TODO -- Publish in transaction publishSegments(task, context, statusFromRunner.getSegments()); } - if(statusFromRunner.getSegmentsNuked().size() > 0) { + if (statusFromRunner.getSegmentsNuked().size() > 0) { deleteSegments(task, context, statusFromRunner.getSegmentsNuked()); } } @@ -194,10 +201,11 @@ public class TaskConsumer implements Runnable throw Throwables.propagate(e); } } - }); + } + ); // Emit event and log, if the task is done - if(statusFromRunner.isComplete()) { + if (statusFromRunner.isComplete()) { int segmentBytes = 0; for (DataSegment segment : statusFromRunner.getSegments()) { segmentBytes += segment.getSize(); @@ -226,7 +234,8 @@ public class TaskConsumer implements Runnable statusFromRunner.getDuration() ); } - } catch(Exception e) { + } + catch (Exception e) { log.makeAlert(e, "Failed to handle task callback") .addData("task", task.getId()) .addData("statusCode", statusFromRunner.getStatusCode()) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index fb8b161d46b..3d03d7e10b8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -424,10 +424,7 @@ public class IndexerCoordinatorNode extends RegisteringNode jsonMapper ); final SegmentKiller segmentKiller = new S3SegmentKiller( - s3Client, - dbi, - dbConnectorConfig, - jsonMapper + s3Client ); taskToolbox = new TaskToolbox(config, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper); } @@ -487,8 +484,6 @@ public class IndexerCoordinatorNode extends RegisteringNode public void initializeWorkerSetupManager() { if (workerSetupManager == null) { - final DbConnectorConfig dbConnectorConfig = configFactory.build(DbConnectorConfig.class); - final DBI dbi = new DbConnector(dbConnectorConfig).getDBI(); final WorkerSetupManagerConfig workerSetupManagerConfig = configFactory.build(WorkerSetupManagerConfig.class); DbConnector.createConfigTable(dbi, workerSetupManagerConfig.getConfigTable()); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 050069003a7..2ee0a1ecaff 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -29,15 +29,16 @@ 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.db.DbConnector; -import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.S3SegmentKiller; +import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.loading.S3SegmentPusherConfig; import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -46,9 +47,6 @@ 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.config.WorkerConfig; -import com.metamx.druid.loading.S3SegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -73,7 +71,6 @@ import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.DefaultServlet; import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; -import org.skife.jdbi.v2.DBI; import java.io.IOException; import java.util.Arrays; @@ -298,13 +295,8 @@ public class WorkerNode extends RegisteringNode configFactory.build(S3SegmentPusherConfig.class), jsonMapper ); - final DbConnectorConfig dbConnectorConfig = configFactory.build(DbConnectorConfig.class); - DBI dbi = new DbConnector(dbConnectorConfig).getDBI(); final SegmentKiller segmentKiller = new S3SegmentKiller( - s3Client, - dbi, - dbConnectorConfig, - jsonMapper + s3Client ); taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper); } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index 4f4bc9fdfed..b8620eb42df 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -1,7 +1,6 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.ISE; @@ -135,7 +134,7 @@ public class RemoteTaskRunnerTest { remoteTaskRunner.run( task1, - new TaskContext(new DateTime().toString(), Sets.newHashSet()), + new TaskContext(new DateTime().toString(), Sets.newHashSet(), Sets.newHashSet()), null ); } @@ -143,9 +142,25 @@ public class RemoteTaskRunnerTest @Test public void testAlreadyExecutedTask() throws Exception { - remoteTaskRunner.run(task1, new TaskContext(new DateTime().toString(), Sets.newHashSet()), null); + remoteTaskRunner.run( + task1, + new TaskContext( + new DateTime().toString(), + Sets.newHashSet(), + Sets.newHashSet() + ), + null + ); try { - remoteTaskRunner.run(task1, new TaskContext(new DateTime().toString(), Sets.newHashSet()), null); + remoteTaskRunner.run( + task1, + new TaskContext( + new DateTime().toString(), + Sets.newHashSet(), + Sets.newHashSet() + ), + null + ); fail("ISE expected"); } catch (ISE expected) { @@ -175,7 +190,7 @@ public class RemoteTaskRunnerTest ) ), Lists.newArrayList() ), - new TaskContext(new DateTime().toString(), Sets.newHashSet()), + new TaskContext(new DateTime().toString(), Sets.newHashSet(), Sets.newHashSet()), null ); } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java index 4d2d5947d75..c811d007b57 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java @@ -41,30 +41,32 @@ public class TaskConsumerTest tc.start(); try { - tq.add(new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) - { - @Override - public Type getType() - { - return Type.TEST; - } + tq.add( + new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + { + @Override + public Type getType() + { + return Type.TEST; + } - @Override - public TaskStatus run( - TaskContext context, TaskToolbox toolbox, TaskCallback callback - ) throws Exception - { - return TaskStatus.success(getId()).withSegments( - ImmutableSet.of( - DataSegment.builder() - .dataSource("ds") - .interval(new Interval("2012-01-01/P1D")) - .version(context.getVersion()) - .build() - ) - ); - } - }); + @Override + public TaskStatus run( + TaskContext context, TaskToolbox toolbox, TaskCallback callback + ) throws Exception + { + return TaskStatus.success(getId()).withSegments( + ImmutableSet.of( + DataSegment.builder() + .dataSource("ds") + .interval(new Interval("2012-01-01/P1D")) + .version(context.getVersion()) + .build() + ) + ); + } + } + ); while (ts.getStatus("id1").get().isRunnable()) { Thread.sleep(100); @@ -97,7 +99,13 @@ public class TaskConsumerTest } @Override - public List getSegmentsForInterval(String dataSource, Interval interval) throws IOException + public List getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException + { + return ImmutableList.of(); + } + + @Override + public List getUnusedSegmentsForInterval(String dataSource, Interval interval) { return ImmutableList.of(); } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 8c196001125..0755a15a3a8 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -408,7 +408,7 @@ public class TaskQueueTest } }; - callback.notify(vt.getTask().run(new TaskContext(vt.getVersion(), null), null, callback)); + callback.notify(vt.getTask().run(new TaskContext(vt.getVersion(), null, null), null, callback)); // OK, finally ready to test stuff. Assert.assertTrue("pass1", structThingy.pass1); diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java index add1d45824e..46f6acfc629 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java @@ -1,26 +1,13 @@ package com.metamx.druid.loading; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; import com.google.inject.Inject; -import com.metamx.common.ISE; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.db.DbConnectorConfig; -import org.codehaus.jackson.map.ObjectMapper; import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.joda.time.Interval; -import org.skife.jdbi.v2.DBI; -import org.skife.jdbi.v2.FoldController; -import org.skife.jdbi.v2.Folder3; -import org.skife.jdbi.v2.Handle; -import org.skife.jdbi.v2.StatementContext; -import org.skife.jdbi.v2.tweak.HandleCallback; -import java.sql.SQLException; -import java.util.List; +import java.util.Collection; import java.util.Map; /** @@ -30,80 +17,20 @@ public class S3SegmentKiller implements SegmentKiller private static final Logger log = new Logger(S3SegmentKiller.class); private final RestS3Service s3Client; - private final DBI dbi; - private final DbConnectorConfig config; - private final ObjectMapper jsonMapper; @Inject public S3SegmentKiller( - RestS3Service s3Client, - DBI dbi, - DbConnectorConfig config, - ObjectMapper jsonMapper + RestS3Service s3Client ) { this.s3Client = s3Client; - this.dbi = dbi; - this.config = config; - this.jsonMapper = jsonMapper; } @Override - public List kill(final String datasource, final Interval interval) throws ServiceException + public void kill(Collection segments) throws ServiceException { - // TODO -- Awkward for workers to use the DB! - - List matchingSegments = dbi.withHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) throws Exception - { - return handle.createQuery( - String.format( - "SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and end <= :end and used = 0", - config.getSegmentTable() - ) - ) - .bind("dataSource", datasource) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()) - .fold( - Lists.newArrayList(), - new Folder3, Map>() - { - @Override - public List fold( - List accumulator, - Map stringObjectMap, - FoldController foldController, - StatementContext statementContext - ) throws SQLException - { - try { - DataSegment segment = jsonMapper.readValue( - (String) stringObjectMap.get("payload"), - DataSegment.class - ); - - accumulator.add(segment); - - return accumulator; - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ); - } - } - ); - - log.info("Found %,d segments for %s for interval %s.", matchingSegments.size(), datasource, interval); - for (final DataSegment segment : matchingSegments) { - // Remove from S3 + for (final DataSegment segment : segments) { Map loadSpec = segment.getLoadSpec(); String s3Bucket = MapUtils.getString(loadSpec, "bucket"); String s3Path = MapUtils.getString(loadSpec, "key"); @@ -118,7 +45,5 @@ public class S3SegmentKiller implements SegmentKiller s3Client.deleteObject(s3Bucket, s3DescriptorPath); } } - - return matchingSegments; } } diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java index 56a14d2e933..8f8746d5324 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java @@ -2,13 +2,13 @@ package com.metamx.druid.loading; import com.metamx.druid.client.DataSegment; import org.jets3t.service.ServiceException; -import org.joda.time.Interval; +import java.util.Collection; import java.util.List; /** */ public interface SegmentKiller { - public List kill(String datasource, Interval interval) throws ServiceException; + public void kill(Collection segments) throws ServiceException; } From 0e469c6f4c2c627b4c4c4232a67a1dd7e357d572 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 31 Jan 2013 08:09:22 -0800 Subject: [PATCH 12/63] TaskStatus: Fix javadocs --- .../main/java/com/metamx/druid/merger/common/TaskStatus.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java index 6f26cb5d3b0..fe79d4f2e5e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java @@ -34,9 +34,7 @@ import java.util.Set; /** * Represents the status of a task. The task may be ongoing ({@link #isComplete()} false) or it may be - * complete ({@link #isComplete()} true). Ongoing tasks may request segments to be created, but only - * complete tasks may request segments to be nuked or spawn other tasks. Failed tasks may not request - * anything. + * complete ({@link #isComplete()} true). * * TaskStatus objects are immutable. */ From fe38ed2547079364d30ed4e113b1c680a53ed889 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 31 Jan 2013 08:10:59 -0800 Subject: [PATCH 13/63] Introduce TaskExistsException, thrown by TaskStorage.insert when appropriate --- .../merger/coordinator/DbTaskStorage.java | 50 +++++++++++-------- .../merger/coordinator/LocalTaskStorage.java | 34 +++++++++---- .../coordinator/TaskExistsException.java | 22 ++++++++ .../druid/merger/coordinator/TaskQueue.java | 12 +++-- .../druid/merger/coordinator/TaskStorage.java | 2 +- .../merger/coordinator/TaskQueueTest.java | 2 +- 6 files changed, 88 insertions(+), 34 deletions(-) create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/TaskExistsException.java diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java index d1a71039274..b63ff8fc8c7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java @@ -33,6 +33,7 @@ import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.exceptions.StatementException; import org.skife.jdbi.v2.tweak.HandleCallback; import java.util.List; @@ -67,29 +68,38 @@ public class DbTaskStorage implements TaskStorage log.info("Inserting task %s with status: %s", task.getId(), status); - dbi.withHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) throws Exception + try { + dbi.withHandle( + new HandleCallback() { - handle.createStatement( - String.format( - "INSERT INTO %s (id, created_date, payload, status_code, status_payload) VALUES (:id, :created_date, :payload, :status_code, :status_payload)", - dbConnectorConfig.getTaskTable() - ) - ) - .bind("id", task.getId()) - .bind("created_date", new DateTime().toString()) - .bind("payload", jsonMapper.writeValueAsString(task)) - .bind("status_code", status.getStatusCode().toString()) - .bind("status_payload", jsonMapper.writeValueAsString(status)) - .execute(); + @Override + public Void withHandle(Handle handle) throws Exception + { + handle.createStatement( + String.format( + "INSERT INTO %s (id, created_date, payload, status_code, status_payload) VALUES (:id, :created_date, :payload, :status_code, :status_payload)", + dbConnectorConfig.getTaskTable() + ) + ) + .bind("id", task.getId()) + .bind("created_date", new DateTime().toString()) + .bind("payload", jsonMapper.writeValueAsString(task)) + .bind("status_code", status.getStatusCode().toString()) + .bind("status_payload", jsonMapper.writeValueAsString(status)) + .execute(); - return null; + return null; + } } - } - ); + ); + } catch (StatementException e) { + // Might be a duplicate task ID. + if(getTask(task.getId()).isPresent()) { + throw new TaskExistsException(task.getId(), e); + } else { + throw e; + } + } } @Override diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java index a4055a637e3..e169890f233 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java @@ -21,15 +21,12 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; -import javax.annotation.Nullable; import java.util.List; import java.util.Map; @@ -54,7 +51,11 @@ public class LocalTaskStorage implements TaskStorage task.getId(), status.getId() ); - Preconditions.checkState(!tasks.containsKey(task.getId()), "Task ID must not already be present: %s", task.getId()); + + if(tasks.containsKey(task.getId())) { + throw new TaskExistsException(task.getId()); + } + log.info("Inserting task %s with status: %s", task.getId(), status); tasks.put(task.getId(), new TaskStuff(task, status)); } @@ -64,7 +65,7 @@ public class LocalTaskStorage implements TaskStorage { Preconditions.checkNotNull(taskid, "taskid"); if(tasks.containsKey(taskid)) { - return Optional.of(tasks.get(taskid).task); + return Optional.of(tasks.get(taskid).getTask()); } else { return Optional.absent(); } @@ -85,7 +86,7 @@ public class LocalTaskStorage implements TaskStorage { Preconditions.checkNotNull(taskid, "taskid"); if(tasks.containsKey(taskid)) { - return Optional.of(tasks.get(taskid).status); + return Optional.of(tasks.get(taskid).getStatus()); } else { return Optional.absent(); } @@ -106,7 +107,7 @@ public class LocalTaskStorage implements TaskStorage { Preconditions.checkNotNull(taskid, "taskid"); if(tasks.containsKey(taskid)) { - return tasks.get(taskid).version; + return tasks.get(taskid).getVersion(); } else { return Optional.absent(); } @@ -117,8 +118,8 @@ public class LocalTaskStorage implements TaskStorage { final ImmutableList.Builder listBuilder = ImmutableList.builder(); for(final TaskStuff taskStuff : tasks.values()) { - if(taskStuff.status.isRunnable()) { - listBuilder.add(taskStuff.task); + if(taskStuff.getStatus().isRunnable()) { + listBuilder.add(taskStuff.getTask()); } } @@ -147,6 +148,21 @@ public class LocalTaskStorage implements TaskStorage this.version = version; } + public Task getTask() + { + return task; + } + + public TaskStatus getStatus() + { + return status; + } + + public Optional getVersion() + { + return version; + } + private TaskStuff withStatus(TaskStatus _status) { return new TaskStuff(task, _status, version); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskExistsException.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskExistsException.java new file mode 100644 index 00000000000..ddc146c3167 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskExistsException.java @@ -0,0 +1,22 @@ +package com.metamx.druid.merger.coordinator; + +public class TaskExistsException extends RuntimeException +{ + private final String taskId; + + public TaskExistsException(String taskId, Throwable t) + { + super(String.format("Task exists: %s", taskId), t); + this.taskId = taskId; + } + + public TaskExistsException(String taskId) + { + this(taskId, null); + } + + public String getTaskId() + { + return taskId; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java index a3587a955d9..bd5e4bafdc1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import com.google.common.base.Throwables; import com.google.common.collect.HashMultimap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -184,9 +185,14 @@ public class TaskQueue try { Preconditions.checkState(active, "Queue is not active!"); - // If this throws, we don't want to insert the task into our queue. - // (This is how we detect duplicates) - taskStorage.insert(task, TaskStatus.running(task.getId())); + // If this throws with any sort of exception, including TaskExistsException, we don't want to + // insert the task into our queue. + try { + taskStorage.insert(task, TaskStatus.running(task.getId())); + } catch(TaskExistsException e) { + log.warn("Attempt to add task twice: %s", task.getId()); + throw Throwables.propagate(e); + } queue.add(task); workMayBeAvailable.signalAll(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java index 258fd22444c..54732a1173f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java @@ -29,7 +29,7 @@ public interface TaskStorage { /** * Adds a task to the storage facility with a particular status. If the task ID already exists, this method - * will throw an exception. + * will throw a {@link TaskExistsException}. */ public void insert(Task task, TaskStatus status); diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 0755a15a3a8..b850bbbdff7 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -87,7 +87,7 @@ public class TaskQueueTest thrown = null; try { tq.add(newTask("T5", "G5", "baz", new Interval("2013-02-01/PT1H"))); - } catch(IllegalStateException e) { + } catch(TaskExistsException e) { thrown = e; } From 779c54d8f298e8a63191d6ddfcc09370e2fa7524 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 31 Jan 2013 08:13:44 -0800 Subject: [PATCH 14/63] TaskToolbox: Replace IndexerCoordinatorConfig with TaskConfig --- .../druid/merger/common/TaskToolbox.java | 7 +-- .../merger/common/config/TaskConfig.java | 21 ++++++++ .../config/IndexerCoordinatorConfig.java | 11 ---- .../http/IndexerCoordinatorNode.java | 12 ++++- .../druid/merger/worker/http/WorkerNode.java | 9 ++-- .../coordinator/RemoteTaskRunnerTest.java | 52 +------------------ 6 files changed, 43 insertions(+), 69 deletions(-) create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index 26336b45c25..e01dcb0c020 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -25,6 +25,7 @@ import com.metamx.druid.loading.S3SegmentGetterConfig; import com.metamx.druid.loading.S3ZippedSegmentPuller; import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.loading.SegmentPuller; +import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.loading.SegmentPusher; @@ -40,7 +41,7 @@ import java.util.Map; */ public class TaskToolbox { - private final IndexerCoordinatorConfig config; + private final TaskConfig config; private final ServiceEmitter emitter; private final RestS3Service s3Client; private final SegmentPusher segmentPusher; @@ -48,7 +49,7 @@ public class TaskToolbox private final ObjectMapper objectMapper; public TaskToolbox( - IndexerCoordinatorConfig config, + TaskConfig config, ServiceEmitter emitter, RestS3Service s3Client, SegmentPusher segmentPusher, @@ -64,7 +65,7 @@ public class TaskToolbox this.objectMapper = objectMapper; } - public IndexerCoordinatorConfig getConfig() + public TaskConfig getConfig() { return config; } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java new file mode 100644 index 00000000000..c66009cd8ac --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java @@ -0,0 +1,21 @@ +package com.metamx.druid.merger.common.config; + +import com.metamx.druid.merger.common.task.Task; +import org.skife.config.Config; +import org.skife.config.Default; + +import java.io.File; + +public abstract class TaskConfig +{ + @Config("druid.merger.taskDir") + public abstract File getBaseTaskDir(); + + @Config("druid.merger.rowFlushBoundary") + @Default("500000") + public abstract long getRowFlushBoundary(); + + public File getTaskDir(final Task task) { + return new File(getBaseTaskDir(), task.getId()); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java index 6d6218c3bff..b727f805bb2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java @@ -52,9 +52,6 @@ public abstract class IndexerCoordinatorConfig @Default("local") public abstract String getStorageImpl(); - @Config("druid.merger.taskDir") - public abstract File getBaseTaskDir(); - @Config("druid.merger.whitelist.enabled") @Default("false") public abstract boolean isWhitelistEnabled(); @@ -63,10 +60,6 @@ public abstract class IndexerCoordinatorConfig @Default("") public abstract String getWhitelistDatasourcesString(); - public File getTaskDir(final Task task) { - return new File(getBaseTaskDir(), task.getId()); - } - public Set getWhitelistDatasources() { if(whitelistDatasources == null) { @@ -80,10 +73,6 @@ public abstract class IndexerCoordinatorConfig return whitelistDatasources; } - @Config("druid.merger.rowFlushBoundary") - @Default("500000") - public abstract long getRowFlushBoundary(); - @Config("druid.indexer.strategy") @Default("noop") public abstract String getStrategyImpl(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 3d03d7e10b8..3d084d60712 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -54,6 +54,7 @@ import com.metamx.druid.loading.S3SegmentPusherConfig; import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; +import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; import com.metamx.druid.merger.coordinator.DbTaskStorage; import com.metamx.druid.merger.coordinator.LocalTaskRunner; @@ -134,6 +135,7 @@ public class IndexerCoordinatorNode extends RegisteringNode private DbConnectorConfig dbConnectorConfig = null; private DBI dbi = null; private IndexerCoordinatorConfig config = null; + private TaskConfig taskConfig = null; private TaskToolbox taskToolbox = null; private MergerDBCoordinator mergerDBCoordinator = null; private TaskStorage taskStorage = null; @@ -213,6 +215,7 @@ public class IndexerCoordinatorNode extends RegisteringNode initializeMonitors(); initializeDB(); initializeIndexerCoordinatorConfig(); + initializeTaskConfig(); initializeMergeDBCoordinator(); initializeTaskToolbox(); initializeTaskStorage(); @@ -409,6 +412,13 @@ public class IndexerCoordinatorNode extends RegisteringNode } } + private void initializeTaskConfig() + { + if (taskConfig == null) { + taskConfig = configFactory.build(TaskConfig.class); + } + } + public void initializeTaskToolbox() throws S3ServiceException { if (taskToolbox == null) { @@ -426,7 +436,7 @@ public class IndexerCoordinatorNode extends RegisteringNode final SegmentKiller segmentKiller = new S3SegmentKiller( s3Client ); - taskToolbox = new TaskToolbox(config, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper); + taskToolbox = new TaskToolbox(taskConfig, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 2ee0a1ecaff..14c8e73ac1f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -41,6 +41,7 @@ import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; +import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.worker.TaskMonitor; @@ -98,7 +99,7 @@ public class WorkerNode extends RegisteringNode private List monitors = null; private ServiceEmitter emitter = null; - private IndexerCoordinatorConfig coordinatorConfig = null; // TODO needed for task toolbox, but shouldn't be + private TaskConfig taskConfig = null; private WorkerConfig workerConfig = null; private TaskToolbox taskToolbox = null; private CuratorFramework curatorFramework = null; @@ -272,8 +273,8 @@ public class WorkerNode extends RegisteringNode private void initializeMergerConfig() { - if (coordinatorConfig == null) { - coordinatorConfig = configFactory.build(IndexerCoordinatorConfig.class); + if (taskConfig == null) { + taskConfig = configFactory.build(TaskConfig.class); } if (workerConfig == null) { @@ -298,7 +299,7 @@ public class WorkerNode extends RegisteringNode final SegmentKiller segmentKiller = new S3SegmentKiller( s3Client ); - taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper); + taskToolbox = new TaskToolbox(taskConfig, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper); } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index b8620eb42df..ec53b0257b6 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -11,6 +11,7 @@ import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; +import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.task.DefaultMergeTask; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; @@ -277,38 +278,8 @@ public class RemoteTaskRunnerTest cf, workerCuratorCoordinator, new TaskToolbox( - new IndexerCoordinatorConfig() + new TaskConfig() { - @Override - public String getServerName() - { - return "worker1"; - } - - @Override - public String getLeaderLatchPath() - { - return null; - } - - @Override - public int getNumLocalThreads() - { - return 1; - } - - @Override - public String getRunnerImpl() - { - return null; - } - - @Override - public String getStorageImpl() - { - return null; - } - @Override public File getBaseTaskDir() { @@ -320,30 +291,11 @@ public class RemoteTaskRunnerTest } } - @Override - public boolean isWhitelistEnabled() - { - return false; - } - - @Override - public String getWhitelistDatasourcesString() - { - return null; - } - @Override public long getRowFlushBoundary() { return 0; } - - - @Override - public String getStrategyImpl() - { - return null; - } }, null, null, null, null, jsonMapper ), Executors.newSingleThreadExecutor() From 65576cbade1f96083fc54c1659ceb6d26859cd95 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 31 Jan 2013 08:16:56 -0800 Subject: [PATCH 15/63] TaskStorageQueryAdapter: Rename some methods and tweak javadocs for clarity --- .../coordinator/TaskStorageQueryAdapter.java | 16 ++++++++-------- .../http/IndexerCoordinatorResource.java | 2 +- .../druid/merger/coordinator/TaskQueueTest.java | 2 +- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java index 2c9dbc6181b..e9cc37c9842 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java @@ -50,10 +50,11 @@ public class TaskStorageQueryAdapter } /** - * Returns all recursive task statuses for a particular task from the same task group. Includes that task, - * plus any tasks it spawned, and so on. Excludes spawned tasks that ended up in a different task group. + * Returns all recursive task statuses for a particular task, staying within the same task group. Includes that + * task, plus any tasks it spawned, and so on. Does not include spawned tasks that ended up in a different task + * group. Does not include this task's parents or siblings. */ - public Map> getGroupRecursiveStatuses(final String taskid) + public Map> getSameGroupChildStatuses(final String taskid) { final Optional taskOptional = storage.getTask(taskid); final Optional statusOptional = storage.getStatus(taskid); @@ -64,7 +65,7 @@ public class TaskStorageQueryAdapter if(taskOptional.isPresent() && statusOptional.isPresent()) { for(final Task nextTask : statusOptional.get().getNextTasks()) { if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) { - resultBuilder.putAll(getGroupRecursiveStatuses(nextTask.getId())); + resultBuilder.putAll(getSameGroupChildStatuses(nextTask.getId())); } } } @@ -73,12 +74,11 @@ public class TaskStorageQueryAdapter } /** - * Like {@link #getGroupRecursiveStatuses}, but flattens all recursive statuses for the same task group into a - * single, merged status. + * Like {@link #getSameGroupChildStatuses}, but flattens the recursive statuses into a single, merged status. */ - public Optional getGroupMergedStatus(final String taskid) + public Optional getSameGroupMergedStatus(final String taskid) { - final Map> statuses = getGroupRecursiveStatuses(taskid); + final Map> statuses = getSameGroupChildStatuses(taskid); int nSuccesses = 0; int nFailures = 0; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index 7465b1c231e..6073a7f4325 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -112,7 +112,7 @@ public class IndexerCoordinatorResource @Produces("application/json") public Response doStatus(@PathParam("taskid") String taskid) { - final Optional status = taskStorageQueryAdapter.getGroupMergedStatus(taskid); + final Optional status = taskStorageQueryAdapter.getSameGroupMergedStatus(taskid); if (!status.isPresent()) { return Response.status(Response.Status.NOT_FOUND).build(); } else { diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index b850bbbdff7..73a2d4bd6f1 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -416,7 +416,7 @@ public class TaskQueueTest Assert.assertTrue("id1 isSuccess", ts.getStatus("id1").get().isSuccess()); Assert.assertTrue( "id1 isSuccess (merged)", - new TaskStorageQueryAdapter(ts).getGroupMergedStatus("id1").get().isSuccess() + new TaskStorageQueryAdapter(ts).getSameGroupMergedStatus("id1").get().isSuccess() ); Assert.assertTrue("id2 isRunnable", ts.getStatus("id2").get().isRunnable()); } From 86ec3a372ed89d61bae31cb645b48bbad8d4cbaa Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 31 Jan 2013 08:17:56 -0800 Subject: [PATCH 16/63] TaskGroup: Add javadocs --- .../druid/merger/coordinator/TaskGroup.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java index 0287f93a1bc..853e18c918c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java @@ -74,10 +74,18 @@ public class TaskGroup return new ImmediateCommitStyle(); } + /** + * Returns number of tasks in this group. + */ public int size() { return taskMap.size(); } + /** + * Adds a task to this group. + * @param task task to add + * @return true iff this group did not already contain the task + */ public boolean add(final Task task) { Preconditions.checkArgument( task.getGroupId().equals(groupId), @@ -86,7 +94,6 @@ public class TaskGroup groupId ); - // Act sort of like a Set if(taskMap.containsKey(task.getId())) { return false; } else { @@ -95,10 +102,18 @@ public class TaskGroup } } + /** + * Returns true if this group contains a particular task. + */ public boolean contains(final String taskId) { return taskMap.containsKey(taskId); } + /** + * Removes a task from this group. + * @param taskId task ID to remove + * @return the removed task, or null if the task was not in this group + */ public Task remove(final String taskId) { return taskMap.remove(taskId); From 34e76f868fccc927921eb541c30b2ffb85afef17 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 31 Jan 2013 08:20:33 -0800 Subject: [PATCH 17/63] TaskQueue: Clarify "Naptime!" log message --- .../java/com/metamx/druid/merger/coordinator/TaskQueue.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java index bd5e4bafdc1..1afbfd78c1d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java @@ -159,7 +159,7 @@ public class TaskQueue try { - log.info("Naptime!"); + log.info("Naptime! Shutting down until we are started again."); queue.clear(); running.clear(); From 1e35e6ad46db2c620582adfe62ed3cf67d883e96 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 31 Jan 2013 08:21:10 -0800 Subject: [PATCH 18/63] TaskQueue: Tweak to implementation of findTaskGroupForTask --- .../metamx/druid/merger/coordinator/TaskQueue.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java index 1afbfd78c1d..6c7058475f1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java @@ -26,6 +26,7 @@ import com.google.common.base.Predicate; import com.google.common.base.Throwables; import com.google.common.collect.HashMultimap; import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; @@ -42,6 +43,7 @@ import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -514,7 +516,7 @@ public class TaskQueue giant.lock(); try { - final List maybeTaskGroup = Lists.newArrayList( + final Iterator maybeTaskGroup = FunctionalIterable.create(findTaskGroupsForInterval(task.getDataSource(), task.getInterval())) .filter( new Predicate() @@ -526,14 +528,13 @@ public class TaskQueue } } ) - ); + .iterator(); - if (maybeTaskGroup.size() == 1) { - return Optional.of(maybeTaskGroup.get(0)); - } else if (maybeTaskGroup.size() == 0) { + + if (!maybeTaskGroup.hasNext()) { return Optional.absent(); } else { - throw new IllegalStateException(String.format("WTF?! Task %s is in multiple task groups!", task.getId())); + return Optional.of(Iterators.getOnlyElement(maybeTaskGroup)); } } finally { From f946fc3ee68d8fd92388503a1e54496e0e1475fb Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 31 Jan 2013 08:21:36 -0800 Subject: [PATCH 19/63] TaskConsumer: Commit to segment db in one transaction --- .../coordinator/MergerDBCoordinator.java | 246 ++++++++++-------- .../merger/coordinator/exec/TaskConsumer.java | 41 ++- .../merger/coordinator/TaskConsumerTest.java | 168 ++++++++---- 3 files changed, 292 insertions(+), 163 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java index 6f72bff2da2..c485d38aa87 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java @@ -28,6 +28,7 @@ import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.merger.common.TaskStatus; import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -37,6 +38,8 @@ import org.skife.jdbi.v2.Folder3; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.ResultIterator; import org.skife.jdbi.v2.StatementContext; +import org.skife.jdbi.v2.TransactionCallback; +import org.skife.jdbi.v2.TransactionStatus; import org.skife.jdbi.v2.tweak.HandleCallback; import javax.annotation.Nullable; @@ -51,8 +54,6 @@ public class MergerDBCoordinator { private static final Logger log = new Logger(MergerDBCoordinator.class); - private final Object lock = new Object(); - private final ObjectMapper jsonMapper; private final DbConnectorConfig dbConnectorConfig; private final DBI dbi; @@ -68,132 +69,153 @@ public class MergerDBCoordinator this.dbi = dbi; } - public List getUsedSegmentsForInterval(final String dataSource, final Interval interval) throws IOException + public List getUsedSegmentsForInterval(final String dataSource, final Interval interval) + throws IOException { - synchronized (lock) { + // XXX Could be reading from a cache if we can assume we're the only one editing the DB - // XXX Could be reading from a cache if we can assume we're the only one editing the DB - - final VersionedIntervalTimeline timeline = dbi.withHandle( - new HandleCallback>() + final VersionedIntervalTimeline timeline = dbi.withHandle( + new HandleCallback>() + { + @Override + public VersionedIntervalTimeline withHandle(Handle handle) throws Exception { - @Override - public VersionedIntervalTimeline withHandle(Handle handle) throws Exception - { - final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( - Ordering.natural() + final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( + Ordering.natural() + ); + + final ResultIterator> dbSegments = + handle.createQuery( + String.format( + "SELECT payload FROM %s WHERE used = 1 AND dataSource = :dataSource", + dbConnectorConfig.getSegmentTable() + ) + ) + .bind("dataSource", dataSource) + .iterator(); + + while (dbSegments.hasNext()) { + + final Map dbSegment = dbSegments.next(); + + DataSegment segment = jsonMapper.readValue( + (String) dbSegment.get("payload"), + DataSegment.class ); - final ResultIterator> dbSegments = - handle.createQuery( - String.format( - "SELECT payload FROM %s WHERE used = 1 AND dataSource = :dataSource", - dbConnectorConfig.getSegmentTable() - ) - ) - .bind("dataSource", dataSource) - .iterator(); - - while (dbSegments.hasNext()) { - - final Map dbSegment = dbSegments.next(); - - DataSegment segment = jsonMapper.readValue( - (String) dbSegment.get("payload"), - DataSegment.class - ); - - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); - - } - - dbSegments.close(); - - return timeline; + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); } - } - ); - final List segments = Lists.transform( - timeline.lookup(interval), - new Function, DataSegment>() + dbSegments.close(); + + return timeline; + + } + } + ); + + final List segments = Lists.transform( + timeline.lookup(interval), + new Function, DataSegment>() + { + @Override + public DataSegment apply(TimelineObjectHolder input) + { + return input.getObject().getChunk(0).getObject(); + } + } + ); + + return segments; + } + + public void commitTaskStatus(final TaskStatus taskStatus) + { + try { + dbi.inTransaction( + new TransactionCallback() { @Override - public DataSegment apply(@Nullable TimelineObjectHolder input) + public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception { - return input.getObject().getChunk(0).getObject(); + for(final DataSegment segment : taskStatus.getSegments()) + { + log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), taskStatus.getId()); + announceHistoricalSegment(handle, segment); + } + + for(final DataSegment segment : taskStatus.getSegmentsNuked()) + { + log.info("Deleting segment[%s] for task[%s]", segment.getIdentifier(), taskStatus.getId()); + deleteSegment(handle, segment); + } + + return null; } } ); - - return segments; - + } + catch (Exception e) { + throw new RuntimeException(String.format("Exception commit task to DB: %s", taskStatus.getId()), e); } } public void announceHistoricalSegment(final DataSegment segment) throws Exception { - synchronized (lock) { - try { - List> exists = dbi.withHandle( - new HandleCallback>>() - { - @Override - public List> withHandle(Handle handle) throws Exception - { - return handle.createQuery( - String.format( - "SELECT id FROM %s WHERE id = ':identifier'", - dbConnectorConfig.getSegmentTable() - ) - ).bind( - "identifier", - segment.getIdentifier() - ).list(); - } - } - ); - - if (!exists.isEmpty()) { - log.info("Found [%s] in DB, not updating DB", segment.getIdentifier()); - return; + dbi.withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception + { + announceHistoricalSegment(handle, segment); + return null; + } } + ); + } - dbi.withHandle( - new HandleCallback() - { - @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)", - dbConnectorConfig.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(); + private void announceHistoricalSegment(final Handle handle, final DataSegment segment) throws Exception + { + try { + final List> exists = handle.createQuery( + String.format( + "SELECT id FROM %s WHERE id = ':identifier'", + dbConnectorConfig.getSegmentTable() + ) + ).bind( + "identifier", + segment.getIdentifier() + ).list(); - return null; - } - } - ); - - log.info("Published segment [%s] to DB", segment.getIdentifier()); - } - catch (Exception e) { - log.error(e, "Exception inserting into DB"); - throw new RuntimeException(e); + if (!exists.isEmpty()) { + log.info("Found [%s] in DB, not updating DB", segment.getIdentifier()); + return; } + + 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)", + dbConnectorConfig.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(); + + log.info("Published segment [%s] to DB", segment.getIdentifier()); + } + catch (Exception e) { + log.error(e, "Exception inserting into DB"); + throw e; } } @@ -205,17 +227,21 @@ public class MergerDBCoordinator @Override public Void withHandle(Handle handle) throws Exception { - handle.createStatement( - String.format("DELETE from %s WHERE id = :id", dbConnectorConfig.getSegmentTable()) - ).bind("id", segment.getIdentifier()) - .execute(); - + deleteSegment(handle, segment); return null; } } ); } + private void deleteSegment(final Handle handle, final DataSegment segment) + { + handle.createStatement( + String.format("DELETE from %s WHERE id = :id", dbConnectorConfig.getSegmentTable()) + ).bind("id", segment.getIdentifier()) + .execute(); + } + public List getUnusedSegmentsForInterval(final String dataSource, final Interval interval) { List matchingSegments = dbi.withHandle( diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index 74c1b4fb46e..2a21d67ef35 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -187,14 +187,38 @@ public class TaskConsumer implements Runnable public void run() { try { - if (statusFromRunner.getSegments().size() > 0) { - // TODO -- Publish in transaction - publishSegments(task, context, statusFromRunner.getSegments()); + // Validate status + for (final DataSegment segment : statusFromRunner.getSegments()) { + verifyDataSourceAndInterval(task, context, segment); + + // Verify version (must be equal to our context version) + if (!context.getVersion().equals(segment.getVersion())) { + throw new IllegalStateException( + String.format( + "Segment for task[%s] has invalid version: %s", + task.getId(), + segment.getIdentifier() + ) + ); + } } - if (statusFromRunner.getSegmentsNuked().size() > 0) { - deleteSegments(task, context, statusFromRunner.getSegmentsNuked()); + for (final DataSegment segment : statusFromRunner.getSegmentsNuked()) { + verifyDataSourceAndInterval(task, context, segment); + + // Verify version (must be less than our context version) + if (segment.getVersion().compareTo(context.getVersion()) >= 0) { + throw new IllegalStateException( + String.format( + "Segment-to-nuke for task[%s] has invalid version: %s", + task.getId(), + segment.getIdentifier() + ) + ); + } } + + mergerDBCoordinator.commitTaskStatus(statusFromRunner); } catch (Exception e) { log.error(e, "Exception while publishing segments for task: %s", task); @@ -211,11 +235,18 @@ public class TaskConsumer implements Runnable segmentBytes += segment.getSize(); } + int segmentNukedBytes = 0; + for (DataSegment segment : statusFromRunner.getSegmentsNuked()) { + segmentNukedBytes += segment.getSize(); + } + builder.setUser3(statusFromRunner.getStatusCode().toString()); emitter.emit(builder.build("indexer/time/run/millis", statusFromRunner.getDuration())); emitter.emit(builder.build("indexer/segment/count", statusFromRunner.getSegments().size())); emitter.emit(builder.build("indexer/segment/bytes", segmentBytes)); + emitter.emit(builder.build("indexer/segmentNuked/count", statusFromRunner.getSegmentsNuked().size())); + emitter.emit(builder.build("indexer/segmentNuked/bytes", segmentNukedBytes)); if (statusFromRunner.isFailure()) { log.makeAlert("Failed to index") diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java index c811d007b57..6fdd8725da9 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java @@ -10,11 +10,15 @@ import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.task.AbstractTask; import com.metamx.druid.merger.coordinator.exec.TaskConsumer; +import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Event; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; import junit.framework.Assert; +import org.easymock.EasyMock; import org.joda.time.Interval; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.IOException; @@ -24,68 +28,122 @@ import java.util.concurrent.Executors; public class TaskConsumerTest { - @Test - public void testSimple() + private TaskStorage ts = null; + private TaskQueue tq = null; + private TaskRunner tr = null; + private MockMergerDBCoordinator mdc = null; + private TaskConsumer tc = null; + + @Before + public void setUp() { - final TaskStorage ts = new LocalTaskStorage(); - final TaskQueue tq = new TaskQueue(ts); - final TaskRunner tr = new LocalTaskRunner( + EmittingLogger.registerEmitter(EasyMock.createMock(ServiceEmitter.class)); + + ts = new LocalTaskStorage(); + tq = new TaskQueue(ts); + tr = new LocalTaskRunner( new TaskToolbox(null, null, null, null, null, null), Executors.newSingleThreadExecutor() ); - final MockMergerDBCoordinator mdc = newMockMDC(); - final TaskConsumer tc = new TaskConsumer(tq, tr, mdc, newMockEmitter()); + mdc = newMockMDC(); + tc = new TaskConsumer(tq, tr, mdc, newMockEmitter()); tq.start(); tc.start(); + } - try { - tq.add( - new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + @After + public void tearDown() + { + tc.stop(); + tq.stop(); + } + + @Test + public void testSimple() throws Exception + { + tq.add( + new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + { + @Override + public Type getType() { - @Override - public Type getType() - { - return Type.TEST; - } - - @Override - public TaskStatus run( - TaskContext context, TaskToolbox toolbox, TaskCallback callback - ) throws Exception - { - return TaskStatus.success(getId()).withSegments( - ImmutableSet.of( - DataSegment.builder() - .dataSource("ds") - .interval(new Interval("2012-01-01/P1D")) - .version(context.getVersion()) - .build() - ) - ); - } + return Type.TEST; } - ); - while (ts.getStatus("id1").get().isRunnable()) { - Thread.sleep(100); - } + @Override + public TaskStatus run( + TaskContext context, TaskToolbox toolbox, TaskCallback callback + ) throws Exception + { + return TaskStatus.success(getId()).withSegments( + ImmutableSet.of( + DataSegment.builder() + .dataSource("ds") + .interval(new Interval("2012-01-01/P1D")) + .version(context.getVersion()) + .build() + ) + ); + } + } + ); - final TaskStatus status = ts.getStatus("id1").get(); - Assert.assertTrue("nextTasks", status.getNextTasks().isEmpty()); - Assert.assertEquals("segments.size", 1, status.getSegments().size()); - Assert.assertEquals("segmentsNuked.size", 0, status.getSegmentsNuked().size()); - Assert.assertEquals("segments published", status.getSegments(), mdc.getPublished()); - Assert.assertEquals("segments nuked", status.getSegmentsNuked(), mdc.getNuked()); + while (ts.getStatus("id1").get().isRunnable()) { + Thread.sleep(100); } - catch (Exception e) { - throw Throwables.propagate(e); - } - finally { - tc.stop(); - tq.stop(); + + final TaskStatus status = ts.getStatus("id1").get(); + Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("nextTasks.size", 0, status.getNextTasks().size()); + Assert.assertEquals("segments.size", 1, status.getSegments().size()); + Assert.assertEquals("segmentsNuked.size", 0, status.getSegmentsNuked().size()); + Assert.assertEquals("segments published", status.getSegments(), mdc.getPublished()); + Assert.assertEquals("segments nuked", status.getSegmentsNuked(), mdc.getNuked()); + } + + @Test + public void testBadVersion() throws Exception + { + tq.add( + new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + { + @Override + public Type getType() + { + return Type.TEST; + } + + @Override + public TaskStatus run( + TaskContext context, TaskToolbox toolbox, TaskCallback callback + ) throws Exception + { + return TaskStatus.success(getId()).withSegments( + ImmutableSet.of( + DataSegment.builder() + .dataSource("ds") + .interval(new Interval("2012-01-01/P1D")) + .version(context.getVersion() + "1!!!1!!") + .build() + ) + ); + } + } + ); + + while (ts.getStatus("id1").get().isRunnable()) { + Thread.sleep(100); } + + final TaskStatus status = ts.getStatus("id1").get(); + Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode()); + Assert.assertEquals("nextTasks.size", 0, status.getNextTasks().size()); + Assert.assertEquals("segments.size", 0, status.getSegments().size()); + Assert.assertEquals("segmentsNuked.size", 0, status.getSegmentsNuked().size()); + Assert.assertEquals("segments published", status.getSegments(), mdc.getPublished()); + Assert.assertEquals("segments nuked", status.getSegmentsNuked(), mdc.getNuked()); } private static class MockMergerDBCoordinator extends MergerDBCoordinator @@ -111,7 +169,21 @@ public class TaskConsumerTest } @Override - public void announceHistoricalSegment(DataSegment segment) throws Exception + public void commitTaskStatus(TaskStatus taskStatus) + { + for(final DataSegment segment : taskStatus.getSegments()) + { + announceHistoricalSegment(segment); + } + + for(final DataSegment segment : taskStatus.getSegmentsNuked()) + { + deleteSegment(segment); + } + } + + @Override + public void announceHistoricalSegment(DataSegment segment) { published.add(segment); } From 393bec0539e3e747cc7b9ecc1950cbce6fa15491 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 1 Feb 2013 12:16:06 -0800 Subject: [PATCH 20/63] TaskQueue: Behavior tweaks, simplification - Lock tasks on add if possible - Detect "already added" using exception test instead of bookkeeping map - Update task status after commitRunnable instead of before commitRunnable --- .../druid/merger/coordinator/TaskQueue.java | 70 +++++++++++-------- 1 file changed, 41 insertions(+), 29 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java index 6c7058475f1..e228b401025 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java @@ -73,7 +73,6 @@ public class TaskQueue { private final List queue = Lists.newLinkedList(); private final Map> running = Maps.newHashMap(); - private final Multimap seenNextTasks = HashMultimap.create(); private final TaskStorage taskStorage; @@ -199,6 +198,11 @@ public class TaskQueue queue.add(task); workMayBeAvailable.signalAll(); + // Attempt to add this task to a running task group. Silently continue if this is not possible. + // The main reason this is here is so when subtasks are added, they end up in the same task group + // as their parent whenever possible. + tryLock(task); + return true; } finally { @@ -274,26 +278,26 @@ public class TaskQueue * Finally, if this task is not supposed to be running, this method will simply do nothing. * * @param task task to update - * @param status new task status + * @param originalStatus new task status * @param commitRunnable operation to perform if this task is ready to commit * * @throws NullPointerException if task or status is null * @throws IllegalArgumentException if the task ID does not match the status ID * @throws IllegalStateException if this queue is currently shut down */ - public void notify(final Task task, final TaskStatus status, final Runnable commitRunnable) + public void notify(final Task task, final TaskStatus originalStatus, final Runnable commitRunnable) { giant.lock(); try { Preconditions.checkNotNull(task, "task"); - Preconditions.checkNotNull(status, "status"); + Preconditions.checkNotNull(originalStatus, "status"); Preconditions.checkState(active, "Queue is not active!"); Preconditions.checkArgument( - task.getId().equals(status.getId()), + task.getId().equals(originalStatus.getId()), "Mismatching task ids[%s/%s]", task.getId(), - status.getId() + originalStatus.getId() ); final TaskGroup taskGroup; @@ -306,20 +310,13 @@ public class TaskQueue taskGroup = maybeTaskGroup.get(); } - // Update status in DB - // TODO: We can either do this first, in which case we run the risk of having a task marked done in the DB but - // TODO: not committed here; or we can do it last, in which case we run the risk of having a task marked running - // TODO: in the DB but committed here. Currently, we err on the former side because we don't want a ticking time - // TODO: bomb in the DB (a task marked running that we have forgotten about, which will potentially be re- - // TODO: started much later when a coordinator bootstraps). - // TODO: - // TODO: Eventually we should have this status update enter a retry queue instead of throwing an exception - // TODO: if it fails. - taskStorage.setStatus(task.getId(), status); + // This is what we want to write to the DB when we're done. + // Not final, since we might need to reassign the var later if the commitRunnable fails. + TaskStatus statusToSave = originalStatus; // Should we commit? - if (taskGroup.getCommitStyle().shouldCommit(task, status)) { - log.info("Committing %s status for task: %s", status.getStatusCode(), task.getId()); + if (taskGroup.getCommitStyle().shouldCommit(task, statusToSave)) { + log.info("Committing %s status for task: %s", statusToSave.getStatusCode(), task.getId()); // Add next tasks try { @@ -330,12 +327,10 @@ public class TaskQueue // We want to allow tasks to submit RUNNING statuses with the same nextTasks over and over. // So, we need to remember which ones we've already spawned and not do them again. - for (final Task nextTask : status.getNextTasks()) { - if (!seenNextTasks.containsEntry(task.getId(), nextTask.getId())) { + for (final Task nextTask : statusToSave.getNextTasks()) { + try { add(nextTask); - tryLock(nextTask); - seenNextTasks.put(task.getId(), nextTask.getId()); - } else { + } catch (TaskExistsException e) { log.info("Already added followup task %s to original task: %s", nextTask.getId(), task.getId()); } } @@ -343,19 +338,36 @@ public class TaskQueue catch (Exception e) { log.makeAlert(e, "Failed to commit task") .addData("task", task.getId()) - .addData("statusCode", status.getStatusCode()) + .addData("statusCode", statusToSave.getStatusCode()) .emit(); - // TODO -- If this fails, it should enter a retry queue instead of throwing an exception - taskStorage.setStatus(task.getId(), TaskStatus.failure(task.getId()).withDuration(status.getDuration())); + // Rewrite status + statusToSave = TaskStatus.failure(task.getId()).withDuration(statusToSave.getDuration()); } } else { - log.info("Not committing %s status for task: %s", status.getStatusCode(), task); + log.info("Not committing %s status for task: %s", statusToSave.getStatusCode(), task); } - if (status.isComplete()) { + boolean didSetStatus = false; + + try { + taskStorage.setStatus(task.getId(), statusToSave); + didSetStatus = true; + } catch(Exception e) { + // TODO: This could be a task-status-submission retry queue instead of retrying the entire task, + // TODO: which is heavy and probably not necessary. + log.warn(e, "Status could not be persisted! Reinserting task: %s", task.getId()); + + log.makeAlert(e, "Failed to persist task status") + .addData("task", task.getId()) + .addData("statusCode", statusToSave.getStatusCode()) + .emit(); + + queue.add(task); + } + + if(didSetStatus && statusToSave.isComplete()) { unlock(task); - seenNextTasks.removeAll(task.getId()); log.info("Task done: %s", task); } } From 58bc305e26fbc3e80ac57642fd4d773b3ab74a04 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Feb 2013 12:58:38 -0800 Subject: [PATCH 21/63] changes to remote task runner to kill nodes better --- .../merger/coordinator/RemoteTaskRunner.java | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 0892328db1c..1bead5a3346 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -175,27 +175,31 @@ public class RemoteTaskRunner implements TaskRunner public void run() { if (currentlyTerminating.isEmpty()) { - if (zkWorkers.size() <= workerSetupManager.getWorkerSetupData().getMinNumWorkers()) { + final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers(); + if (zkWorkers.size() <= minNumWorkers) { return; } - int workerCount = 0; - List thoseLazyWorkers = Lists.newArrayList(); - for (WorkerWrapper workerWrapper : zkWorkers.values()) { - workerCount++; - - if (workerCount > workerSetupManager.getWorkerSetupData().getMinNumWorkers() && - workerWrapper.getRunningTasks().isEmpty() && - System.currentTimeMillis() - workerWrapper.getLastCompletedTaskTime().getMillis() - > config.getMaxWorkerIdleTimeMillisBeforeDeletion() - ) { - thoseLazyWorkers.add(workerWrapper); - } - } + List thoseLazyWorkers = Lists.newArrayList( + FunctionalIterable + .create(zkWorkers.values()) + .filter( + new Predicate() + { + @Override + public boolean apply(WorkerWrapper input) + { + return input.getRunningTasks().isEmpty() + && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() + > config.getMaxWorkerIdleTimeMillisBeforeDeletion(); + } + } + ) + ); AutoScalingData terminated = strategy.terminate( Lists.transform( - thoseLazyWorkers, + thoseLazyWorkers.subList(minNumWorkers, thoseLazyWorkers.size() - 1), new Function() { @Override From 9e881978452ce3bec19e67995e7a725d87e11abe Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Feb 2013 14:08:52 -0800 Subject: [PATCH 22/63] change endpoints for killing segments --- .../com/metamx/druid/http/InfoResource.java | 18 +++++++++++++++--- .../com/metamx/druid/http/MasterResource.java | 9 --------- .../src/main/resources/static/js/kill-0.0.1.js | 9 ++------- 3 files changed, 17 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/http/InfoResource.java b/server/src/main/java/com/metamx/druid/http/InfoResource.java index 090e311ad31..e434bff995d 100644 --- a/server/src/main/java/com/metamx/druid/http/InfoResource.java +++ b/server/src/main/java/com/metamx/druid/http/InfoResource.java @@ -33,7 +33,10 @@ import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; +import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.rules.Rule; +import com.metamx.druid.merge.ClientKillQuery; +import org.joda.time.Interval; import javax.annotation.Nullable; import javax.inject.Inject; @@ -58,6 +61,7 @@ import java.util.TreeSet; @Path("/info") public class InfoResource { + private final DruidMaster master; private final ServerInventoryManager serverInventoryManager; private final DatabaseSegmentManager databaseSegmentManager; private final DatabaseRuleManager databaseRuleManager; @@ -65,12 +69,14 @@ public class InfoResource @Inject public InfoResource( + DruidMaster master, ServerInventoryManager serverInventoryManager, DatabaseSegmentManager databaseSegmentManager, DatabaseRuleManager databaseRuleManager, DruidClusterInfo druidClusterInfo ) { + this.master = master; this.serverInventoryManager = serverInventoryManager; this.databaseSegmentManager = databaseSegmentManager; this.databaseRuleManager = databaseRuleManager; @@ -362,11 +368,17 @@ public class InfoResource @DELETE @Path("/datasources/{dataSourceName}") public Response deleteDataSource( - @PathParam("dataSourceName") final String dataSourceName + @PathParam("dataSourceName") final String dataSourceName, + @QueryParam("kill") final String kill, + @QueryParam("interval") final String interval ) { - if (!databaseSegmentManager.removeDatasource(dataSourceName)) { - return Response.status(Response.Status.NOT_FOUND).build(); + if (kill != null && Boolean.valueOf(kill)) { + master.killSegments(new ClientKillQuery(dataSourceName, new Interval(interval))); + } else { + if (!databaseSegmentManager.removeDatasource(dataSourceName)) { + return Response.status(Response.Status.NOT_FOUND).build(); + } } return Response.status(Response.Status.OK).build(); diff --git a/server/src/main/java/com/metamx/druid/http/MasterResource.java b/server/src/main/java/com/metamx/druid/http/MasterResource.java index c73e74e528c..b725ed7f358 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterResource.java +++ b/server/src/main/java/com/metamx/druid/http/MasterResource.java @@ -111,15 +111,6 @@ public class MasterResource return resp; } - @POST - @Path("/kill") - @Consumes("application/json") - public Response killSegments(ClientKillQuery killQuery) - { - master.killSegments(killQuery); - return Response.ok().build(); - } - @GET @Path("/loadstatus") @Produces("application/json") diff --git a/server/src/main/resources/static/js/kill-0.0.1.js b/server/src/main/resources/static/js/kill-0.0.1.js index 495c6c47f2d..666651128f1 100644 --- a/server/src/main/resources/static/js/kill-0.0.1.js +++ b/server/src/main/resources/static/js/kill-0.0.1.js @@ -20,14 +20,9 @@ $(document).ready(function() { Yes : function() { var selected = $('#datasources option:selected').text(); var interval = $('#interval').val(); - var toSend = { - "dataSource" : selected, - "interval" : interval - } $.ajax({ - type: 'POST', - url:'/master/kill', - data: JSON.stringify(toSend), + type: 'DELETE', + url:'/info/datasources/' + selected +'?kill=true&interval=' + interval, contentType:"application/json; charset=utf-8", dataType:"json", error: function(xhr, status, error) { From 4ec67f612ef87e6aba7f0fb1a359ddeee00814bc Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 1 Feb 2013 16:29:58 -0800 Subject: [PATCH 23/63] Merger: Emit indexer/segment/bytes once per segment. No longer emit indexer/segment/count. --- .../merger/coordinator/exec/TaskConsumer.java | 22 +++++++------------ 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index 2a21d67ef35..af4d2b478f9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -230,23 +230,17 @@ public class TaskConsumer implements Runnable // Emit event and log, if the task is done if (statusFromRunner.isComplete()) { - int segmentBytes = 0; - for (DataSegment segment : statusFromRunner.getSegments()) { - segmentBytes += segment.getSize(); - } - - int segmentNukedBytes = 0; - for (DataSegment segment : statusFromRunner.getSegmentsNuked()) { - segmentNukedBytes += segment.getSize(); - } - builder.setUser3(statusFromRunner.getStatusCode().toString()); + for (DataSegment segment : statusFromRunner.getSegments()) { + emitter.emit(builder.build("indexer/segment/bytes", segment.getSize())); + } + + for (DataSegment segmentNuked : statusFromRunner.getSegmentsNuked()) { + emitter.emit(builder.build("indexer/segmentNuked/bytes", segmentNuked.getSize())); + } + emitter.emit(builder.build("indexer/time/run/millis", statusFromRunner.getDuration())); - emitter.emit(builder.build("indexer/segment/count", statusFromRunner.getSegments().size())); - emitter.emit(builder.build("indexer/segment/bytes", segmentBytes)); - emitter.emit(builder.build("indexer/segmentNuked/count", statusFromRunner.getSegmentsNuked().size())); - emitter.emit(builder.build("indexer/segmentNuked/bytes", segmentNukedBytes)); if (statusFromRunner.isFailure()) { log.makeAlert("Failed to index") From d9f2e68f7fadf8b0e2bae378b2a2ed726eae20ea Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Feb 2013 17:16:24 -0800 Subject: [PATCH 24/63] better logs --- .../com/metamx/druid/merger/coordinator/RemoteTaskRunner.java | 2 +- .../merger/coordinator/scaling/EC2AutoScalingStrategy.java | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 1bead5a3346..32acc66ae43 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -563,7 +563,7 @@ public class RemoteTaskRunner implements TaskRunner ); if (workerQueue.isEmpty()) { - log.info("Worker nodes do not have capacity to run any more tasks!"); + log.info("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values()); if (currentlyProvisioning.isEmpty()) { AutoScalingData provisioned = strategy.provision(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index 8d51da61afd..2a50a8b55fd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -130,6 +130,10 @@ public class EC2AutoScalingStrategy implements ScalingStrategy @Override public AutoScalingData terminate(List ids) { + if (ids.isEmpty()) { + return new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()); + } + DescribeInstancesResult result = amazonEC2Client.describeInstances( new DescribeInstancesRequest() .withFilters( From efd0dc0062e056f459e9c25dcc166d05868a01e0 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 11:09:49 -0800 Subject: [PATCH 25/63] refactor scaling stuff --- .../query/search/FragmentSearchQuerySpec.java | 2 +- .../InsensitiveContainsSearchQuerySpec.java | 7 +- .../merger/coordinator/RemoteTaskRunner.java | 181 ++++---------- .../druid/merger/coordinator/TaskQueue.java | 7 + .../http/IndexerCoordinatorNode.java | 77 ++++-- .../coordinator/scaling/AutoScalingData.java | 9 + ...Strategy.java => AutoScalingStrategy.java} | 9 +- .../scaling/EC2AutoScalingStrategy.java | 4 +- ...tegy.java => NoopAutoScalingStrategy.java} | 6 +- .../ResourceManagementSchedulerConfig.java | 42 ++++ .../scaling/ResourceManagementStrategy.java | 38 +++ .../scaling/ResourceManagmentScheduler.java | 138 +++++++++++ .../coordinator/scaling/ScalingStats.java | 88 +++++++ .../SimpleResourceManagementStrategy.java | 222 ++++++++++++++++++ .../SimpleResourceManagmentConfig.java | 41 ++++ .../com/metamx/druid/merger/TestTask.java | 71 ++++++ .../coordinator/RemoteTaskRunnerTest.java | 65 +---- .../SimpleResourceManagementStrategyTest.java | 142 +++++++++++ .../scaling/TestAutoScalingStrategy.java | 45 ++++ 19 files changed, 959 insertions(+), 235 deletions(-) rename merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/{ScalingStrategy.java => AutoScalingStrategy.java} (81%) rename merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/{NoopScalingStrategy.java => NoopAutoScalingStrategy.java} (90%) create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagmentScheduler.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java create mode 100644 merger/src/test/java/com/metamx/druid/merger/TestTask.java create mode 100644 merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java create mode 100644 merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/TestAutoScalingStrategy.java diff --git a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java index e0a73b4bc3c..7786d2c01bd 100644 --- a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java @@ -73,7 +73,7 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec public boolean accept(String dimVal) { for (String value : values) { - if (!dimVal.toLowerCase().contains(value)) { + if (dimVal == null || !dimVal.toLowerCase().contains(value)) { return false; } } diff --git a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java index 87a6246a5b2..1de1c7360fa 100644 --- a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java @@ -59,6 +59,9 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec @Override public boolean accept(String dimVal) { + if (dimVal == null) { + return false; + } return dimVal.toLowerCase().contains(value); } @@ -77,8 +80,8 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec public String toString() { return "InsensitiveContainsSearchQuerySpec{" + - "value=" + value + - ", sortSpec=" + sortSpec + + "value=" + value + + ", sortSpec=" + sortSpec + "}"; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 32acc66ae43..c2a201c9632 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -19,27 +19,21 @@ package com.metamx.druid.merger.coordinator; -import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Predicate; import com.google.common.base.Throwables; -import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Sets; import com.google.common.primitives.Ints; import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.PeriodGranularity; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskHolder; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; -import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; -import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.Worker; import com.metamx.emitter.EmittingLogger; @@ -52,29 +46,25 @@ import com.netflix.curator.utils.ZKPaths; import org.apache.zookeeper.CreateMode; import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; -import org.joda.time.Duration; -import org.joda.time.Period; -import java.util.Arrays; +import java.util.Collection; import java.util.Comparator; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; /** - * The RemoteTaskRunner encapsulates all interactions with Zookeeper and keeps track of which workers - * are running which tasks. The RemoteTaskRunner is event driven and updates state according to ephemeral node - * changes in ZK. + * The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes and manage retries in failure + * scenarios. The RemoteTaskRunner keeps track of which workers are running which tasks and manages coordinator and + * worker interactions over Zookeeper. The RemoteTaskRunner is event driven and updates state according to ephemeral + * node changes in ZK. *

    - * The RemoteTaskRunner will assign tasks to a node until the node hits capacity. RemoteTaskRunners have scaling - * strategies to help them decide when to create or delete new resources. When tasks are assigned to the remote - * task runner and no workers have capacity to handle the task, provisioning will be done according to the strategy. - * The remote task runner periodically runs a check to see if any worker nodes have not had any work for a - * specified period of time. If so, the worker node will be terminated. + * The RemoteTaskRunner will assign tasks to a node until the node hits capacity. At that point, task assignment will + * fail. The RemoteTaskRunner depends on another manager to create additional worker resources. + * For example, {@link com.metamx.druid.merger.coordinator.scaling.ResourceManagmentScheduler} can take care of these duties. + * *

    * If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will automatically retry any tasks * that were associated with the node. @@ -90,7 +80,6 @@ public class RemoteTaskRunner implements TaskRunner private final PathChildrenCache workerPathCache; private final ScheduledExecutorService scheduledExec; private final RetryPolicyFactory retryPolicyFactory; - private final ScalingStrategy strategy; private final WorkerSetupManager workerSetupManager; // all workers that exist in ZK @@ -98,12 +87,8 @@ public class RemoteTaskRunner implements TaskRunner // all tasks that are assigned or need to be assigned private final Map tasks = new ConcurrentHashMap(); - private final ConcurrentSkipListSet currentlyProvisioning = new ConcurrentSkipListSet(); - private final ConcurrentSkipListSet currentlyTerminating = new ConcurrentSkipListSet(); private final Object statusLock = new Object(); - private volatile DateTime lastProvisionTime = new DateTime(); - private volatile DateTime lastTerminateTime = new DateTime(); private volatile boolean started = false; public RemoteTaskRunner( @@ -113,7 +98,6 @@ public class RemoteTaskRunner implements TaskRunner PathChildrenCache workerPathCache, ScheduledExecutorService scheduledExec, RetryPolicyFactory retryPolicyFactory, - ScalingStrategy strategy, WorkerSetupManager workerSetupManager ) { @@ -123,7 +107,6 @@ public class RemoteTaskRunner implements TaskRunner this.workerPathCache = workerPathCache; this.scheduledExec = scheduledExec; this.retryPolicyFactory = retryPolicyFactory; - this.strategy = strategy; this.workerSetupManager = workerSetupManager; } @@ -131,6 +114,10 @@ public class RemoteTaskRunner implements TaskRunner public void start() { try { + if (started) { + return; + } + workerPathCache.getListenable().addListener( new PathChildrenCacheListener() { @@ -157,81 +144,6 @@ public class RemoteTaskRunner implements TaskRunner ); workerPathCache.start(); - // Schedule termination of worker nodes periodically - Period period = new Period(config.getTerminateResourcesDuration()); - PeriodGranularity granularity = new PeriodGranularity(period, config.getTerminateResourcesOriginDateTime(), null); - final long startTime = granularity.next(granularity.truncate(new DateTime().getMillis())); - - ScheduledExecutors.scheduleAtFixedRate( - scheduledExec, - new Duration( - System.currentTimeMillis(), - startTime - ), - config.getTerminateResourcesDuration(), - new Runnable() - { - @Override - public void run() - { - if (currentlyTerminating.isEmpty()) { - final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers(); - if (zkWorkers.size() <= minNumWorkers) { - return; - } - - List thoseLazyWorkers = Lists.newArrayList( - FunctionalIterable - .create(zkWorkers.values()) - .filter( - new Predicate() - { - @Override - public boolean apply(WorkerWrapper input) - { - return input.getRunningTasks().isEmpty() - && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() - > config.getMaxWorkerIdleTimeMillisBeforeDeletion(); - } - } - ) - ); - - AutoScalingData terminated = strategy.terminate( - Lists.transform( - thoseLazyWorkers.subList(minNumWorkers, thoseLazyWorkers.size() - 1), - new Function() - { - @Override - public String apply(WorkerWrapper input) - { - return input.getWorker().getIp(); - } - } - ) - ); - - if (terminated != null) { - currentlyTerminating.addAll(terminated.getNodeIds()); - lastTerminateTime = new DateTime(); - } - } else { - Duration durSinceLastTerminate = new Duration(new DateTime(), lastTerminateTime); - if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration())) { - log.makeAlert("Worker node termination taking too long") - .addData("millisSinceLastTerminate", durSinceLastTerminate.getMillis()) - .addData("terminatingCount", currentlyTerminating.size()) - .emit(); - } - - log.info( - "%s still terminating. Wait for all nodes to terminate before trying again.", - currentlyTerminating - ); - } - } - } - ); started = true; } catch (Exception e) { @@ -243,6 +155,10 @@ public class RemoteTaskRunner implements TaskRunner public void stop() { try { + if (!started) { + return; + } + for (WorkerWrapper workerWrapper : zkWorkers.values()) { workerWrapper.close(); } @@ -255,16 +171,16 @@ public class RemoteTaskRunner implements TaskRunner } } - public boolean hasStarted() - { - return started; - } - - public int getNumWorkers() + public int getNumAvailableWorkers() { return zkWorkers.size(); } + public Collection getAvailableWorkers() + { + return zkWorkers.values(); + } + public boolean isTaskRunning(String taskId) { for (WorkerWrapper workerWrapper : zkWorkers.values()) { @@ -275,6 +191,13 @@ public class RemoteTaskRunner implements TaskRunner return false; } + /** + * A task will be run only if there is no current knowledge in the RemoteTaskRunner of the task. + * + * @param task task to run + * @param context task context to run under + * @param callback callback to be called exactly once + */ @Override public void run(Task task, TaskContext context, TaskCallback callback) { @@ -288,11 +211,18 @@ public class RemoteTaskRunner implements TaskRunner assignTask(taskWrapper); } + /** + * Ensures no workers are already running a task before assigning the task to a worker. + * It is possible that a worker is running a task the RTR has no knowledge of. This is common when the RTR + * needs to bootstrap after a restart. + * + * @param taskWrapper - a wrapper containing task metadata + */ private void assignTask(TaskWrapper taskWrapper) { WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); - // If the task already exists, we don't need to announce it + // If a worker is already running this task, we don't need to announce it if (workerWrapper != null) { final Worker worker = workerWrapper.getWorker(); try { @@ -395,8 +325,6 @@ public class RemoteTaskRunner implements TaskRunner private void addWorker(final Worker worker) { try { - currentlyProvisioning.removeAll(strategy.ipLookup(Arrays.asList(worker.getIp()))); - final String workerStatusPath = JOINER.join(config.getStatusPath(), worker.getHost()); final PathChildrenCache statusCache = new PathChildrenCache(cf, workerStatusPath, true); final WorkerWrapper workerWrapper = new WorkerWrapper( @@ -460,12 +388,12 @@ public class RemoteTaskRunner implements TaskRunner } else { final TaskCallback callback = taskWrapper.getCallback(); - // Cleanup - if (callback != null) { - callback.notify(taskStatus); - } - if (taskStatus.isComplete()) { + // Cleanup + if (callback != null) { + callback.notify(taskStatus); + } + // Worker is done with this task workerWrapper.setLastCompletedTaskTime(new DateTime()); tasks.remove(taskId); @@ -510,8 +438,6 @@ public class RemoteTaskRunner implements TaskRunner */ private void removeWorker(final Worker worker) { - currentlyTerminating.remove(worker.getHost()); - WorkerWrapper workerWrapper = zkWorkers.get(worker.getHost()); if (workerWrapper != null) { try { @@ -564,27 +490,6 @@ public class RemoteTaskRunner implements TaskRunner if (workerQueue.isEmpty()) { log.info("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values()); - - if (currentlyProvisioning.isEmpty()) { - AutoScalingData provisioned = strategy.provision(); - if (provisioned != null) { - currentlyProvisioning.addAll(provisioned.getNodeIds()); - lastProvisionTime = new DateTime(); - } - } else { - Duration durSinceLastProvision = new Duration(new DateTime(), lastProvisionTime); - if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration())) { - log.makeAlert("Worker node provisioning taking too long") - .addData("millisSinceLastProvision", durSinceLastProvision.getMillis()) - .addData("provisioningCount", currentlyProvisioning.size()) - .emit(); - } - - log.info( - "%s still provisioning. Wait for all provisioned nodes to complete before requesting new worker.", - currentlyProvisioning - ); - } return null; } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java index e228b401025..90f19000e60 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java @@ -42,6 +42,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -376,6 +377,12 @@ public class TaskQueue } } + public Collection getAvailableTasks() + { + // TODO: actually implement this + return Lists.newArrayList(); + } + /** * Unlock some work. Does not update the task storage facility. Throws an exception if this work is not currently * running. diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 3d084d60712..242c7873b6d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -75,8 +75,12 @@ import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; -import com.metamx.druid.merger.coordinator.scaling.NoopScalingStrategy; -import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; +import com.metamx.druid.merger.coordinator.scaling.NoopAutoScalingStrategy; +import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerConfig; +import com.metamx.druid.merger.coordinator.scaling.ResourceManagmentScheduler; +import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy; +import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy; +import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; @@ -529,35 +533,19 @@ public class IndexerCoordinatorNode extends RegisteringNode .build() ); - ScalingStrategy strategy; - if (config.getStrategyImpl().equalsIgnoreCase("ec2")) { - strategy = new EC2AutoScalingStrategy( - jsonMapper, - new AmazonEC2Client( - new BasicAWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) - ), - configFactory.build(EC2AutoScalingStrategyConfig.class), - workerSetupManager - ); - } else if (config.getStrategyImpl().equalsIgnoreCase("noop")) { - strategy = new NoopScalingStrategy(); - } else { - throw new ISE("Invalid strategy implementation: %s", config.getStrategyImpl()); - } - - return new RemoteTaskRunner( + RemoteTaskRunner remoteTaskRunner = new RemoteTaskRunner( jsonMapper, configFactory.build(RemoteTaskRunnerConfig.class), curatorFramework, new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true), retryScheduledExec, new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)), - strategy, workerSetupManager ); + + initializeWorkerScaling(remoteTaskRunner); + + return remoteTaskRunner; } }; @@ -577,6 +565,49 @@ public class IndexerCoordinatorNode extends RegisteringNode } } + private void initializeWorkerScaling(RemoteTaskRunner taskRunner) + { + final ScheduledExecutorService scalingScheduledExec = Executors.newScheduledThreadPool( + 1, + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("ScalingExec--%d") + .build() + ); + + AutoScalingStrategy strategy; + if (config.getStrategyImpl().equalsIgnoreCase("ec2")) { + strategy = new EC2AutoScalingStrategy( + jsonMapper, + new AmazonEC2Client( + new BasicAWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ), + configFactory.build(EC2AutoScalingStrategyConfig.class), + workerSetupManager + ); + } else if (config.getStrategyImpl().equalsIgnoreCase("noop")) { + strategy = new NoopAutoScalingStrategy(); + } else { + throw new ISE("Invalid strategy implementation: %s", config.getStrategyImpl()); + } + + ResourceManagmentScheduler resourceManagmentScheduler = new ResourceManagmentScheduler( + taskQueue, + taskRunner, + new SimpleResourceManagementStrategy( + strategy, + configFactory.build(SimpleResourceManagmentConfig.class), + workerSetupManager + ), + configFactory.build(ResourceManagementSchedulerConfig.class), + scalingScheduledExec + ); + lifecycle.addManagedInstance(resourceManagmentScheduler); + } + public static class Builder { private ObjectMapper jsonMapper = null; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java index 5a1bb4980e5..0ca74a9b38c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java @@ -43,4 +43,13 @@ public class AutoScalingData { return nodes; } + + @Override + public String toString() + { + return "AutoScalingData{" + + "nodeIds=" + nodeIds + + ", nodes=" + nodes + + '}'; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingStrategy.java similarity index 81% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingStrategy.java index 150de1357e0..7ab92a0b985 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingStrategy.java @@ -22,8 +22,9 @@ package com.metamx.druid.merger.coordinator.scaling; import java.util.List; /** + * The AutoScalingStrategy has the actual methods to provision and terminate worker nodes. */ -public interface ScalingStrategy +public interface AutoScalingStrategy { public AutoScalingData provision(); @@ -31,8 +32,8 @@ public interface ScalingStrategy /** * Provides a lookup of ip addresses to node ids - * @param ips - * @return + * @param ips - nodes ips + * @return node ids */ - public List ipLookup(List ips); + public List ipToIdLookup(List ips); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index 2a50a8b55fd..d64899a2739 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -43,7 +43,7 @@ import java.util.List; /** */ -public class EC2AutoScalingStrategy implements ScalingStrategy +public class EC2AutoScalingStrategy implements AutoScalingStrategy { private static final EmittingLogger log = new EmittingLogger(EC2AutoScalingStrategy.class); @@ -187,7 +187,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy } @Override - public List ipLookup(List ips) + public List ipToIdLookup(List ips) { DescribeInstancesResult result = amazonEC2Client.describeInstances( new DescribeInstancesRequest() diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopAutoScalingStrategy.java similarity index 90% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopAutoScalingStrategy.java index 2b412ca6202..d4a5f355c6f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopAutoScalingStrategy.java @@ -26,9 +26,9 @@ import java.util.List; /** * This class just logs when scaling should occur. */ -public class NoopScalingStrategy implements ScalingStrategy +public class NoopAutoScalingStrategy implements AutoScalingStrategy { - private static final EmittingLogger log = new EmittingLogger(NoopScalingStrategy.class); + private static final EmittingLogger log = new EmittingLogger(NoopAutoScalingStrategy.class); @Override public AutoScalingData provision() @@ -45,7 +45,7 @@ public class NoopScalingStrategy implements ScalingStrategy } @Override - public List ipLookup(List ips) + public List ipToIdLookup(List ips) { log.info("I'm not a real strategy so I'm returning what I got %s", ips); return ips; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java new file mode 100644 index 00000000000..c732b5c6c1a --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java @@ -0,0 +1,42 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.merger.coordinator.scaling; + +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.skife.config.Config; +import org.skife.config.Default; + +/** + */ +public abstract class ResourceManagementSchedulerConfig +{ + @Config("druid.indexer.provisionResources.duration") + @Default("PT1H") + public abstract Duration getProvisionResourcesDuration(); + + @Config("druid.indexer.terminateResources.duration") + @Default("PT1H") + public abstract Duration getTerminateResourcesDuration(); + + @Config("druid.indexer.terminateResources.originDateTime") + @Default("2012-01-01T00:55:00.000Z") + public abstract DateTime getTerminateResourcesOriginDateTime(); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java new file mode 100644 index 00000000000..37483082e98 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java @@ -0,0 +1,38 @@ +/* + * 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.merger.coordinator.scaling; + +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.WorkerWrapper; + +import java.util.Collection; + +/** + * The ResourceManagementStrategy decides if worker nodes should be provisioned or determined + * based on the available tasks in the system and the state of the workers in the system. + */ +public interface ResourceManagementStrategy +{ + public void doProvision(Collection availableTasks, Collection workerWrappers); + + public void doTerminate(Collection availableTasks, Collection workerWrappers); + + public ScalingStats getStats(); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagmentScheduler.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagmentScheduler.java new file mode 100644 index 00000000000..b2ae623b978 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagmentScheduler.java @@ -0,0 +1,138 @@ +/* + * 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.merger.coordinator.scaling; + +import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import com.metamx.druid.PeriodGranularity; +import com.metamx.druid.merger.coordinator.RemoteTaskRunner; +import com.metamx.druid.merger.coordinator.TaskQueue; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Period; + +import java.util.concurrent.ScheduledExecutorService; + +/** + * The ResourceManagmentScheduler manages when worker nodes should potentially be created or destroyed. + * It uses a {@link TaskQueue} to return the available tasks in the system and a {@link RemoteTaskRunner} to return + * the status of the worker nodes in the system. + * The ResourceManagmentScheduler does not contain the logic to decide whether provision or termination should actually occur. + * That decision is made in the {@link ResourceManagementStrategy}. + */ +public class ResourceManagmentScheduler +{ + private static final Logger log = new Logger(ResourceManagmentScheduler.class); + + private final TaskQueue taskQueue; + private final RemoteTaskRunner remoteTaskRunner; + private final ResourceManagementStrategy resourceManagementStrategy; + private final ResourceManagementSchedulerConfig config; + private final ScheduledExecutorService exec; + + private final Object lock = new Object(); + private volatile boolean started = false; + + public ResourceManagmentScheduler( + TaskQueue taskQueue, + RemoteTaskRunner remoteTaskRunner, + ResourceManagementStrategy resourceManagementStrategy, + ResourceManagementSchedulerConfig config, + ScheduledExecutorService exec + ) + { + this.taskQueue = taskQueue; + this.remoteTaskRunner = remoteTaskRunner; + this.resourceManagementStrategy = resourceManagementStrategy; + this.config = config; + this.exec = exec; + } + + @LifecycleStart + public void start() + { + synchronized (lock) { + if (started) { + return; + } + + ScheduledExecutors.scheduleAtFixedRate( + exec, + config.getProvisionResourcesDuration(), + new Runnable() + { + @Override + public void run() + { + resourceManagementStrategy.doProvision( + taskQueue.getAvailableTasks(), + remoteTaskRunner.getAvailableWorkers() + ); + } + } + ); + + // Schedule termination of worker nodes periodically + Period period = new Period(config.getTerminateResourcesDuration()); + PeriodGranularity granularity = new PeriodGranularity(period, config.getTerminateResourcesOriginDateTime(), null); + final long startTime = granularity.next(granularity.truncate(new DateTime().getMillis())); + + ScheduledExecutors.scheduleAtFixedRate( + exec, + new Duration( + System.currentTimeMillis(), + startTime + ), + config.getTerminateResourcesDuration(), + new Runnable() + { + @Override + public void run() + { + resourceManagementStrategy.doTerminate( + taskQueue.getAvailableTasks(), + remoteTaskRunner.getAvailableWorkers() + ); + } + } + ); + + started = true; + } + } + + @LifecycleStop + public void stop() + { + synchronized (lock) { + if (!started) { + return; + } + exec.shutdown(); + } + } + + public ScalingStats getStats() + { + return resourceManagementStrategy.getStats(); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java new file mode 100644 index 00000000000..d632a61baae --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java @@ -0,0 +1,88 @@ +package com.metamx.druid.merger.coordinator.scaling; + +import com.google.common.collect.Lists; +import com.google.common.collect.MinMaxPriorityQueue; +import org.joda.time.DateTime; +import org.joda.time.DateTimeComparator; + +import java.util.List; + +/** + */ +public class ScalingStats +{ + private static enum EVENT + { + PROVISION, + TERMINATE + } + + private final MinMaxPriorityQueue recentNodes; + + public ScalingStats(int capacity) + { + this.recentNodes = MinMaxPriorityQueue + .orderedBy(DateTimeComparator.getInstance()) + .maximumSize(capacity) + .create(); + } + + public void addProvisionEvent(AutoScalingData data) + { + recentNodes.add( + new ScalingEvent( + data, + new DateTime(), + EVENT.PROVISION + ) + ); + } + + public void addTerminateEvent(AutoScalingData data) + { + recentNodes.add( + new ScalingEvent( + data, + new DateTime(), + EVENT.TERMINATE + ) + ); + } + + public List toList() + { + List retVal = Lists.newArrayList(); + while (!recentNodes.isEmpty()) { + retVal.add(recentNodes.poll()); + } + return retVal; + } + + public static class ScalingEvent + { + private final AutoScalingData data; + private final DateTime timestamp; + private final EVENT event; + + private ScalingEvent( + AutoScalingData data, + DateTime timestamp, + EVENT event + ) + { + this.data = data; + this.timestamp = timestamp; + this.event = event; + } + + @Override + public String toString() + { + return "ScalingEvent{" + + "data=" + data + + ", timestamp=" + timestamp + + ", event=" + event + + '}'; + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java new file mode 100644 index 00000000000..99c581e5cce --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -0,0 +1,222 @@ +/* + * 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.merger.coordinator.scaling; + +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.WorkerWrapper; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.emitter.EmittingLogger; +import org.joda.time.DateTime; +import org.joda.time.Duration; + +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentSkipListSet; + +/** + */ +public class SimpleResourceManagementStrategy implements ResourceManagementStrategy +{ + private static final EmittingLogger log = new EmittingLogger(SimpleResourceManagementStrategy.class); + + private final AutoScalingStrategy autoScalingStrategy; + private final SimpleResourceManagmentConfig config; + private final WorkerSetupManager workerSetupManager; + private final ScalingStats scalingStats; + + private final ConcurrentSkipListSet currentlyProvisioning = new ConcurrentSkipListSet(); + private final ConcurrentSkipListSet currentlyTerminating = new ConcurrentSkipListSet(); + + private volatile DateTime lastProvisionTime = new DateTime(); + private volatile DateTime lastTerminateTime = new DateTime(); + + public SimpleResourceManagementStrategy( + AutoScalingStrategy autoScalingStrategy, + SimpleResourceManagmentConfig config, + WorkerSetupManager workerSetupManager + ) + { + this.autoScalingStrategy = autoScalingStrategy; + this.config = config; + this.workerSetupManager = workerSetupManager; + this.scalingStats = new ScalingStats(config.getNumEventsToTrack()); + } + + @Override + public void doProvision(Collection availableTasks, Collection workerWrappers) + { + boolean nothingProvisioning = Sets.difference( + currentlyProvisioning, + Sets.newHashSet( + autoScalingStrategy.ipToIdLookup( + Lists.newArrayList( + Iterables.transform( + workerWrappers, new Function() + { + @Override + public String apply(WorkerWrapper input) + { + return input.getWorker().getIp(); + } + } + ) + ) + ) + ) + ).isEmpty(); + + boolean moreTasksThanWorkerCapacity = !Sets.difference( + Sets.newHashSet(availableTasks), + Sets.newHashSet( + Iterables.concat( + Iterables.transform( + workerWrappers, + new Function>() + { + @Override + public Set apply(WorkerWrapper input) + { + return input.getRunningTasks(); + } + } + ) + ) + ) + ).isEmpty(); + + if (nothingProvisioning && moreTasksThanWorkerCapacity) { + AutoScalingData provisioned = autoScalingStrategy.provision(); + + if (provisioned != null) { + currentlyProvisioning.addAll(provisioned.getNodeIds()); + lastProvisionTime = new DateTime(); + scalingStats.addProvisionEvent(provisioned); + } + } else { + Duration durSinceLastProvision = new Duration(new DateTime(), lastProvisionTime); + if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration())) { + log.makeAlert("Worker node provisioning taking too long") + .addData("millisSinceLastProvision", durSinceLastProvision.getMillis()) + .addData("provisioningCount", currentlyProvisioning.size()) + .emit(); + } + + log.info( + "%s still provisioning. Wait for all provisioned nodes to complete before requesting new worker.", + currentlyProvisioning + ); + } + } + + @Override + public void doTerminate(Collection availableTasks, Collection workerWrappers) + { + boolean nothingTerminating = Sets.difference( + currentlyTerminating, + Sets.newHashSet( + autoScalingStrategy.ipToIdLookup( + Lists.newArrayList( + Iterables.transform( + workerWrappers, new Function() + { + @Override + public String apply(WorkerWrapper input) + { + return input.getWorker().getIp(); + } + } + ) + ) + ) + ) + ).isEmpty(); + + if (nothingTerminating) { + final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers(); + if (workerWrappers.size() <= minNumWorkers) { + return; + } + + List thoseLazyWorkers = Lists.newArrayList( + FunctionalIterable + .create(workerWrappers) + .filter( + new Predicate() + { + @Override + public boolean apply(WorkerWrapper input) + { + return input.getRunningTasks().isEmpty() + && System.currentTimeMillis() - input.getLastCompletedTaskTime() + .getMillis() + > config.getMaxWorkerIdleTimeMillisBeforeDeletion(); + } + } + ) + ); + + AutoScalingData terminated = autoScalingStrategy.terminate( + Lists.transform( + thoseLazyWorkers.subList(minNumWorkers, thoseLazyWorkers.size() - 1), + new Function() + { + @Override + public String apply(WorkerWrapper input) + { + return input.getWorker().getIp(); + } + } + ) + ); + + if (terminated != null) { + currentlyTerminating.addAll(terminated.getNodeIds()); + lastTerminateTime = new DateTime(); + scalingStats.addProvisionEvent(terminated); + } + } else { + Duration durSinceLastTerminate = new Duration(new DateTime(), lastTerminateTime); + if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration())) { + log.makeAlert("Worker node termination taking too long") + .addData("millisSinceLastTerminate", durSinceLastTerminate.getMillis()) + .addData("terminatingCount", currentlyTerminating.size()) + .emit(); + } + + log.info( + "%s still terminating. Wait for all nodes to terminate before trying again.", + currentlyTerminating + ); + } + } + + @Override + public ScalingStats getStats() + { + return scalingStats; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java new file mode 100644 index 00000000000..a8a5b52ca89 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java @@ -0,0 +1,41 @@ +/* + * 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.merger.coordinator.scaling; + +import org.joda.time.Duration; +import org.skife.config.Config; +import org.skife.config.Default; + +/** + */ +public abstract class SimpleResourceManagmentConfig +{ + @Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion") + @Default("600000") + public abstract int getMaxWorkerIdleTimeMillisBeforeDeletion(); + + @Config("druid.indexer.maxScalingDuration") + @Default("PT1H") + public abstract Duration getMaxScalingDuration(); + + @Config("druid.indexer.numEventsToTrack") + @Default("20") + public abstract int getNumEventsToTrack(); +} diff --git a/merger/src/test/java/com/metamx/druid/merger/TestTask.java b/merger/src/test/java/com/metamx/druid/merger/TestTask.java new file mode 100644 index 00000000000..142d2c88254 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/TestTask.java @@ -0,0 +1,71 @@ +/* + * 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.merger; + +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskCallback; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.task.DefaultMergeTask; +import com.metamx.druid.merger.coordinator.TaskContext; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.annotate.JsonTypeName; + +import java.util.List; + +/** + */ +@JsonTypeName("test") +public class TestTask extends DefaultMergeTask +{ + private final String id; + + public TestTask( + @JsonProperty("id") String id, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segments") List segments, + @JsonProperty("aggregations") List aggregators + ) + { + super(dataSource, segments, aggregators); + + this.id = id; + } + + @Override + @JsonProperty + public String getId() + { + return id; + } + + @Override + public Type getType() + { + return Type.TEST; + } + + @Override + public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception + { + return TaskStatus.success("task1"); + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index ec53b0257b6..c1f264b87ef 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -7,6 +7,7 @@ import com.metamx.common.ISE; 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.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; @@ -14,11 +15,10 @@ import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.task.DefaultMergeTask; import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; -import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; +import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.TaskMonitor; @@ -326,7 +326,6 @@ public class RemoteTaskRunnerTest pathChildrenCache, scheduledExec, new RetryPolicyFactory(new TestRetryPolicyConfig()), - new TestScalingStrategy(), workerSetupManager ); @@ -337,7 +336,7 @@ public class RemoteTaskRunnerTest jsonMapper.writeValueAsBytes(worker1) ); int count = 0; - while (remoteTaskRunner.getNumWorkers() == 0) { + while (remoteTaskRunner.getNumAvailableWorkers() == 0) { Thread.sleep(500); if (count > 10) { throw new ISE("WTF?! Still can't find worker!"); @@ -367,27 +366,6 @@ public class RemoteTaskRunnerTest } } - private static class TestScalingStrategy implements ScalingStrategy - { - @Override - public AutoScalingData provision() - { - return null; - } - - @Override - public AutoScalingData terminate(List nodeIds) - { - return null; - } - - @Override - public List ipLookup(List ips) - { - return ips; - } - } - private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig { @Override @@ -444,41 +422,4 @@ public class RemoteTaskRunnerTest return 1000; } } - - @JsonTypeName("test") - private static class TestTask extends DefaultMergeTask - { - private final String id; - - public TestTask( - @JsonProperty("id") String id, - @JsonProperty("dataSource") String dataSource, - @JsonProperty("segments") List segments, - @JsonProperty("aggregations") List aggregators - ) - { - super(dataSource, segments, aggregators); - - this.id = id; - } - - @Override - @JsonProperty - public String getId() - { - return id; - } - - @Override - public Type getType() - { - return Type.TEST; - } - - @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception - { - return TaskStatus.success("task1"); - } - } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java new file mode 100644 index 00000000000..10398da005f --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -0,0 +1,142 @@ +/* + * 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.merger.coordinator.scaling; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.TestTask; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.WorkerWrapper; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Interval; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Set; + +/** + */ +public class SimpleResourceManagementStrategyTest +{ + private AutoScalingStrategy autoScalingStrategy; + private WorkerSetupManager workerSetupManager; + private Task testTask; + private SimpleResourceManagementStrategy simpleResourceManagementStrategy; + + @Before + public void setUp() throws Exception + { + workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); + autoScalingStrategy = EasyMock.createMock(AutoScalingStrategy.class); + + testTask = new TestTask( + "task1", + "dummyDs", + Lists.newArrayList( + new DataSegment( + "dummyDs", + new Interval(new DateTime(), new DateTime()), + new DateTime().toString(), + null, + null, + null, + null, + 0 + ) + ), Lists.newArrayList() + ); + simpleResourceManagementStrategy = new SimpleResourceManagementStrategy( + new TestAutoScalingStrategy(), + new SimpleResourceManagmentConfig() + { + @Override + public int getMaxWorkerIdleTimeMillisBeforeDeletion() + { + return 0; + } + + @Override + public Duration getMaxScalingDuration() + { + return null; + } + + @Override + public int getNumEventsToTrack() + { + return 1; + } + }, + workerSetupManager + ); + } + + @Test + public void testSuccessfulProvision() throws Exception + { + EasyMock.expect(autoScalingStrategy.provision()).andReturn( + new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()) + ); + EasyMock.replay(autoScalingStrategy); + + simpleResourceManagementStrategy.doProvision( + Arrays.asList( + testTask + ), + Arrays.asList( + new TestWorkerWrapper(testTask) + ) + ); + + EasyMock.verify(autoScalingStrategy); + } + + @Test + public void testDoTerminate() throws Exception + { + + } + + private static class TestWorkerWrapper extends WorkerWrapper + { + private final Task testTask; + + private TestWorkerWrapper( + Task testTask + ) + { + super(null, null, null); + + this.testTask = testTask; + } + + @Override + public Set getRunningTasks() + { + return Sets.newHashSet(testTask.getId()); + } + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/TestAutoScalingStrategy.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/TestAutoScalingStrategy.java new file mode 100644 index 00000000000..8213da61848 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/TestAutoScalingStrategy.java @@ -0,0 +1,45 @@ +/* + * 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.merger.coordinator.scaling; + +import java.util.List; + +/** + */ +public class TestAutoScalingStrategy implements AutoScalingStrategy +{ + @Override + public AutoScalingData provision() + { + return null; + } + + @Override + public AutoScalingData terminate(List ids) + { + return null; + } + + @Override + public List ipToIdLookup(List ips) + { + return null; + } +} From e10025c84172ba448b6ac834e5fe784eb6fe5c84 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 8 Feb 2013 16:50:23 -0800 Subject: [PATCH 26/63] filter out unneeded segments for TimeBoundary query --- .../druid/client/CachingClusteredClient.java | 31 ++++++++++++ .../metamx/druid/query/QueryToolChest.java | 26 ++++++---- .../group/GroupByQueryQueryToolChest.java | 2 +- .../SegmentMetadataQueryQueryToolChest.java | 2 +- .../search/SearchQueryQueryToolChest.java | 2 +- .../TimeBoundaryQueryQueryToolChest.java | 49 ++++++++++++++++++- .../TimeseriesQueryQueryToolChest.java | 2 +- .../com/metamx/druid/utils/JodaUtils.java | 14 ++++++ .../druid/coordination/ServerManagerTest.java | 2 +- 9 files changed, 115 insertions(+), 15 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 163f1986a53..a07478ab751 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -20,6 +20,7 @@ package com.metamx.druid.client; import com.google.common.base.Function; +import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -58,6 +59,7 @@ import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -152,6 +154,35 @@ public class CachingClusteredClient implements QueryRunner } } + // Let tool chest filter out unneeded segments + final Set filteredSegmentDescriptors = Sets.newLinkedHashSet(toolChest.filterSegments( + query, + Iterables.transform( + segments, new Function, SegmentDescriptor>() + { + @Override + public SegmentDescriptor apply( + @Nullable Pair input + ) + { + return input.rhs; + } + } + ) + )); + + // remove unneeded segments from list of segments to query + segments = Sets.newLinkedHashSet(Iterables.filter(segments, new Predicate>() + { + @Override + public boolean apply( + @Nullable Pair input + ) + { + return filteredSegmentDescriptors.contains(input.rhs); + } + })); + final byte[] queryCacheKey; if(strategy != null) { queryCacheKey = strategy.computeCacheKey(query); diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index bec2170ec92..36847f7f77f 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -20,19 +20,23 @@ package com.metamx.druid.query; import com.google.common.base.Function; +import com.metamx.common.Pair; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; +import com.metamx.druid.query.segment.QuerySegmentSpec; +import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; +import org.joda.time.Interval; /** * The broker-side (also used by server in some cases) API for a specific Query type. This API is still undergoing * evolution and is only semi-stable, so proprietary Query implementations should be ready for the potential * maintenance burden when upgrading versions. */ -public interface QueryToolChest> +public abstract class QueryToolChest> { - public QueryRunner mergeResults(QueryRunner runner); + public abstract QueryRunner mergeResults(QueryRunner runner); /** * This method doesn't belong here, but it's here for now just to make it work. @@ -40,11 +44,15 @@ public interface QueryToolChest> * @param seqOfSequences * @return */ - public Sequence mergeSequences(Sequence> seqOfSequences); - public ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); - public Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); - public TypeReference getResultTypeReference(); - public CacheStrategy getCacheStrategy(QueryType query); - public QueryRunner preMergeQueryDecoration(QueryRunner runner); - public QueryRunner postMergeQueryDecoration(QueryRunner runner); + public abstract Sequence mergeSequences(Sequence> seqOfSequences); + public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); + public abstract Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); + public abstract TypeReference getResultTypeReference(); + public abstract CacheStrategy getCacheStrategy(QueryType query); + public abstract QueryRunner preMergeQueryDecoration(QueryRunner runner); + public abstract QueryRunner postMergeQueryDecoration(QueryRunner runner); + + public Iterable filterSegments(QueryType query, Iterable intervals) { + return intervals; + } } diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index 9dcf6110322..b93f46a4562 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -53,7 +53,7 @@ import java.util.Properties; /** */ -public class GroupByQueryQueryToolChest implements QueryToolChest +public class GroupByQueryQueryToolChest extends QueryToolChest { private static final TypeReference TYPE_REFERENCE = new TypeReference(){}; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 160c23cd958..226fc58969f 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -50,7 +50,7 @@ import java.util.Map; import java.util.Set; -public class SegmentMetadataQueryQueryToolChest implements QueryToolChest +public class SegmentMetadataQueryQueryToolChest extends QueryToolChest { private static final TypeReference TYPE_REFERENCE = new TypeReference(){}; private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4}; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index ce3fcc86114..a321c7486c5 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -64,7 +64,7 @@ import java.util.Set; /** */ -public class SearchQueryQueryToolChest implements QueryToolChest, SearchQuery> +public class SearchQueryQueryToolChest extends QueryToolChest, SearchQuery> { private static final byte SEARCH_QUERY = 0x2; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 5ee6c321bbb..b566cd84de3 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -21,8 +21,11 @@ package com.metamx.druid.query.timeboundary; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; @@ -33,20 +36,26 @@ import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.MetricManipulationFn; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChest; +import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; +import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; +import org.joda.time.Interval; import javax.annotation.Nullable; import java.nio.ByteBuffer; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; /** */ public class TimeBoundaryQueryQueryToolChest - implements QueryToolChest, TimeBoundaryQuery> + extends QueryToolChest, TimeBoundaryQuery> { private static final byte TIMEBOUNDARY_QUERY = 0x3; @@ -57,6 +66,44 @@ public class TimeBoundaryQueryQueryToolChest { }; + + @Override + public Iterable filterSegments(TimeBoundaryQuery query, Iterable input) { + long minMillis = Long.MAX_VALUE; + long maxMillis = Long.MIN_VALUE; + SegmentDescriptor min = null; + SegmentDescriptor max = null; + + // keep track of all segments in a given shard + Map> segmentGroups = Maps.newHashMap(); + + Iterable condensedIntervals = JodaUtils.condenseIntervals(query.getIntervals()); + for(SegmentDescriptor e : input) { + if(Iterables.isEmpty(condensedIntervals) || JodaUtils.overlaps(e.getInterval(), condensedIntervals)) { + final long start = e.getInterval().getStartMillis(); + final long end = e.getInterval().getEndMillis(); + final String version = e.getVersion(); + + if(segmentGroups.containsKey(version)) { + segmentGroups.get(version).add(e); + } else { + segmentGroups.put(version, Sets.newHashSet(e)); + } + + if(min == null || start < minMillis) { + min = e; + minMillis = start; + } + if(max == null || end > maxMillis) { + max = e; + maxMillis = end; + } + } + } + + return Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion())); + } + @Override public QueryRunner> mergeResults( final QueryRunner> runner diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 9c633507ec5..c5f8c280c9a 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -60,7 +60,7 @@ import java.util.Map; /** */ -public class TimeseriesQueryQueryToolChest implements QueryToolChest, TimeseriesQuery> +public class TimeseriesQueryQueryToolChest extends QueryToolChest, TimeseriesQuery> { private static final byte TIMESERIES_QUERY = 0x0; diff --git a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java index 53b14618001..c7feeadc44c 100644 --- a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java +++ b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java @@ -19,12 +19,15 @@ package com.metamx.druid.utils; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.guava.Comparators; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Iterator; import java.util.TreeSet; @@ -63,6 +66,17 @@ public class JodaUtils return retVal; } + public static boolean overlaps(final Interval i, Iterable intervals) { + return Iterables.any(intervals, new Predicate() + { + @Override + public boolean apply(@Nullable Interval input) + { + return input.overlaps(i); + } + }); + } + public static DateTime minDateTime(DateTime... times) { if (times == null) { diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index 84982d05737..165fb402107 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -391,7 +391,7 @@ public class ServerManagerTest } } - public static class NoopQueryToolChest> implements QueryToolChest + public static class NoopQueryToolChest> extends QueryToolChest { @Override public QueryRunner mergeResults(QueryRunner runner) From e9f546c4cabdf489e17f061d076a341c78312078 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 16:51:40 -0800 Subject: [PATCH 27/63] add default implementations to QueryToolChest methods --- .../com/metamx/druid/query/QueryToolChest.java | 15 ++++++++++++--- .../group/GroupByQueryQueryToolChest.java | 18 ------------------ .../SegmentMetadataQueryQueryToolChest.java | 12 ------------ .../search/SearchQueryQueryToolChest.java | 6 ------ .../TimeBoundaryQueryQueryToolChest.java | 12 ------------ .../TimeseriesQueryQueryToolChest.java | 6 ------ .../druid/coordination/ServerManagerTest.java | 18 ------------------ 7 files changed, 12 insertions(+), 75 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 36847f7f77f..7c462cc08d8 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -48,9 +48,18 @@ public abstract class QueryToolChest makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); public abstract TypeReference getResultTypeReference(); - public abstract CacheStrategy getCacheStrategy(QueryType query); - public abstract QueryRunner preMergeQueryDecoration(QueryRunner runner); - public abstract QueryRunner postMergeQueryDecoration(QueryRunner runner); + + public CacheStrategy getCacheStrategy(QueryType query) { + return null; + } + + public QueryRunner preMergeQueryDecoration(QueryRunner runner) { + return runner; + } + + public QueryRunner postMergeQueryDecoration(QueryRunner runner) { + return runner; + } public Iterable filterSegments(QueryType query, Iterable intervals) { return intervals; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index b93f46a4562..1c66774d3bc 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -176,22 +176,4 @@ public class GroupByQueryQueryToolChest extends QueryToolChest getCacheStrategy(GroupByQuery query) - { - return null; - } - - @Override - public QueryRunner preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } } diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 226fc58969f..2b979117800 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -219,18 +219,6 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - private Ordering getOrdering() { return new Ordering() diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index a321c7486c5..cefc32b5e3d 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -262,12 +262,6 @@ public class SearchQueryQueryToolChest extends QueryToolChest> postMergeQueryDecoration(final QueryRunner> runner) - { - return runner; - } - private static class SearchThresholdAdjustingQueryRunner implements QueryRunner> { private final QueryRunner> runner; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index b566cd84de3..a4647026e79 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -215,18 +215,6 @@ public class TimeBoundaryQueryQueryToolChest }; } - @Override - public QueryRunner> preMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - - @Override - public QueryRunner> postMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - public Ordering> getOrdering() { return Ordering.natural(); diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index c5f8c280c9a..538897480b3 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -258,12 +258,6 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest>(runner, Period.months(1)); } - @Override - public QueryRunner> postMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - public Ordering> getOrdering() { return Ordering.natural(); diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index 165fb402107..de148a9219c 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -422,23 +422,5 @@ public class ServerManagerTest { return new TypeReference(){}; } - - @Override - public CacheStrategy getCacheStrategy(QueryType query) - { - return null; - } - - @Override - public QueryRunner preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } } } From 6b87ef2921cf1dd5a1124d872dc32e5668656fbe Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 16:55:21 -0800 Subject: [PATCH 28/63] assume segment intervals have already been filtered based on query interval --- .../TimeBoundaryQueryQueryToolChest.java | 37 ++++++++----------- 1 file changed, 15 insertions(+), 22 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index a4647026e79..8ad7378faf2 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -21,7 +21,6 @@ package com.metamx.druid.query.timeboundary; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -39,15 +38,12 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; -import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -77,27 +73,24 @@ public class TimeBoundaryQueryQueryToolChest // keep track of all segments in a given shard Map> segmentGroups = Maps.newHashMap(); - Iterable condensedIntervals = JodaUtils.condenseIntervals(query.getIntervals()); for(SegmentDescriptor e : input) { - if(Iterables.isEmpty(condensedIntervals) || JodaUtils.overlaps(e.getInterval(), condensedIntervals)) { - final long start = e.getInterval().getStartMillis(); - final long end = e.getInterval().getEndMillis(); - final String version = e.getVersion(); + final long start = e.getInterval().getStartMillis(); + final long end = e.getInterval().getEndMillis(); + final String version = e.getVersion(); - if(segmentGroups.containsKey(version)) { - segmentGroups.get(version).add(e); - } else { - segmentGroups.put(version, Sets.newHashSet(e)); - } + if(segmentGroups.containsKey(version)) { + segmentGroups.get(version).add(e); + } else { + segmentGroups.put(version, Sets.newHashSet(e)); + } - if(min == null || start < minMillis) { - min = e; - minMillis = start; - } - if(max == null || end > maxMillis) { - max = e; - maxMillis = end; - } + if(min == null || start < minMillis) { + min = e; + minMillis = start; + } + if(max == null || end > maxMillis) { + max = e; + maxMillis = end; } } From b27a459f71c0fed20144a1ef9190a3166e5bca9b Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 11 Feb 2013 18:13:01 -0800 Subject: [PATCH 29/63] more refactoring and UT passing at this point --- .../merger/common/task/V8toV9UpgradeTask.java | 3 +- .../merger/coordinator/LocalTaskRunner.java | 20 + .../merger/coordinator/RemoteTaskRunner.java | 487 +++++++++--------- .../druid/merger/coordinator/TaskRunner.java | 8 + .../druid/merger/coordinator/TaskWrapper.java | 74 ++- .../merger/coordinator/WorkerWrapper.java | 11 + .../config/RemoteTaskRunnerConfig.java | 17 - .../http/IndexerCoordinatorNode.java | 5 +- .../scaling/ResourceManagementStrategy.java | 6 +- .../scaling/ResourceManagmentScheduler.java | 30 +- .../coordinator/scaling/ScalingStats.java | 38 +- .../SimpleResourceManagementStrategy.java | 132 ++--- .../SimpleResourceManagmentConfig.java | 4 + .../com/metamx/druid/merger/TestTask.java | 13 +- .../coordinator/RemoteTaskRunnerTest.java | 124 ++--- .../SimpleResourceManagementStrategyTest.java | 162 +++++- .../com/metamx/druid/master/DruidMaster.java | 37 +- 17 files changed, 745 insertions(+), 426 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java index 1a718fdaf20..0c152c11978 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java @@ -1,5 +1,6 @@ package com.metamx.druid.merger.common.task; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -31,7 +32,7 @@ public class V8toV9UpgradeTask extends AbstractTask @Override public TaskStatus run( - TaskContext context, TaskToolbox toolbox + TaskContext context, TaskToolbox toolbox, TaskCallback callback ) throws Exception { throw new UnsupportedOperationException(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index ddb07438f9a..d517833aa58 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -20,6 +20,7 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Throwables; +import com.google.common.collect.Lists; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.merger.common.TaskCallback; @@ -29,6 +30,7 @@ import com.metamx.druid.merger.common.task.Task; import org.apache.commons.io.FileUtils; import java.io.File; +import java.util.Collection; import java.util.concurrent.ExecutorService; /** @@ -108,4 +110,22 @@ public class LocalTaskRunner implements TaskRunner } ); } + + @Override + public Collection getRunningTasks() + { + return Lists.newArrayList(); + } + + @Override + public Collection getPendingTasks() + { + return Lists.newArrayList(); + } + + @Override + public Collection getWorkers() + { + return Lists.newArrayList(); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index c2a201c9632..843462e5e4f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -22,6 +22,7 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Joiner; import com.google.common.base.Predicate; import com.google.common.base.Throwables; +import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Sets; import com.google.common.primitives.Ints; @@ -49,9 +50,16 @@ import org.joda.time.DateTime; import java.util.Collection; import java.util.Comparator; +import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -62,9 +70,8 @@ import java.util.concurrent.TimeUnit; * node changes in ZK. *

    * The RemoteTaskRunner will assign tasks to a node until the node hits capacity. At that point, task assignment will - * fail. The RemoteTaskRunner depends on another manager to create additional worker resources. + * fail. The RemoteTaskRunner depends on another component to create additional worker resources. * For example, {@link com.metamx.druid.merger.coordinator.scaling.ResourceManagmentScheduler} can take care of these duties. - * *

    * If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will automatically retry any tasks * that were associated with the node. @@ -84,8 +91,12 @@ public class RemoteTaskRunner implements TaskRunner // all workers that exist in ZK private final Map zkWorkers = new ConcurrentHashMap(); - // all tasks that are assigned or need to be assigned - private final Map tasks = new ConcurrentHashMap(); + // all tasks that have been assigned to a worker + private final ConcurrentSkipListMap runningTasks = new ConcurrentSkipListMap(); + // tasks that have not yet run + private final ConcurrentSkipListMap pendingTasks = new ConcurrentSkipListMap(); + + private final ExecutorService runPendingTasksExec = Executors.newSingleThreadExecutor(); private final Object statusLock = new Object(); @@ -118,6 +129,7 @@ public class RemoteTaskRunner implements TaskRunner return; } + // Add listener for creation/deletion of workers workerPathCache.getListenable().addListener( new PathChildrenCacheListener() { @@ -171,64 +183,147 @@ public class RemoteTaskRunner implements TaskRunner } } - public int getNumAvailableWorkers() - { - return zkWorkers.size(); - } - - public Collection getAvailableWorkers() + @Override + public Collection getWorkers() { return zkWorkers.values(); } + @Override + public Collection getRunningTasks() + { + return runningTasks.values(); + } + + @Override + public Collection getPendingTasks() + { + return pendingTasks.values(); + } + public boolean isTaskRunning(String taskId) { - for (WorkerWrapper workerWrapper : zkWorkers.values()) { - if (workerWrapper.getRunningTasks().contains(taskId)) { - return true; - } - } - return false; + return runningTasks.containsKey(taskId); } /** * A task will be run only if there is no current knowledge in the RemoteTaskRunner of the task. * - * @param task task to run - * @param context task context to run under + * @param task task to run + * @param context task context to run under * @param callback callback to be called exactly once */ @Override public void run(Task task, TaskContext context, TaskCallback callback) { - if (tasks.containsKey(task.getId())) { - throw new ISE("Assigned a task[%s] that already exists, WTF is happening?!", task.getId()); + if (runningTasks.containsKey(task.getId()) || pendingTasks.containsKey(task.getId())) { + throw new ISE("Assigned a task[%s] that is already running, WTF is happening?!", task.getId()); } TaskWrapper taskWrapper = new TaskWrapper( - task, context, callback, retryPolicyFactory.makeRetryPolicy() + task, context, callback, retryPolicyFactory.makeRetryPolicy(), System.currentTimeMillis() ); - tasks.put(taskWrapper.getTask().getId(), taskWrapper); - assignTask(taskWrapper); + addPendingTask(taskWrapper); + } + + private void addPendingTask(final TaskWrapper taskWrapper) + { + pendingTasks.put(taskWrapper.getTask().getId(), taskWrapper); + runPendingTasks(); + } + + /** + * This method uses a single threaded executor to extract all pending tasks and attempt to run them. Any tasks that + * are successfully started by a worker will be moved from pendingTasks to runningTasks. This method is thread-safe. + * This method should be run each time there is new worker capacity or if new tasks are assigned. + */ + private void runPendingTasks() + { + Future future = runPendingTasksExec.submit( + new Callable() + { + @Override + public Void call() throws Exception + { + // make a copy of the pending tasks because assignTask may delete tasks from pending and move them into moving + List copy = Lists.newArrayList(pendingTasks.values()); + for (TaskWrapper taskWrapper : copy) { + assignTask(taskWrapper); + } + + return null; + } + } + ); + + try { + future.get(); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e.getCause()); + } + } + + + private void retryTask(final TaskWrapper taskWrapper, final String workerId, final String taskId) + { + scheduledExec.schedule( + new Runnable() + { + @Override + public void run() + { + runningTasks.remove(taskId); + cleanup(workerId, taskId); + addPendingTask(taskWrapper); + } + }, + taskWrapper.getRetryPolicy().getAndIncrementRetryDelay().getMillis(), + TimeUnit.MILLISECONDS + ); + } + + private void cleanup(final String workerId, final String taskId) + { + try { + final String statusPath = JOINER.join(config.getStatusPath(), workerId, taskId); + cf.delete().guaranteed().forPath(statusPath); + } + catch (Exception e) { + log.warn("Tried to delete a status path that didn't exist! Must've gone away already?"); + } + + try { + final String taskPath = JOINER.join(config.getTaskPath(), workerId, taskId); + cf.delete().guaranteed().forPath(taskPath); + } + catch (Exception e) { + log.warn("Tried to delete a task path that didn't exist! Must've gone away already?"); + } } /** * Ensures no workers are already running a task before assigning the task to a worker. - * It is possible that a worker is running a task the RTR has no knowledge of. This is common when the RTR + * It is possible that a worker is running a task that the RTR has no knowledge of. This is common when the RTR * needs to bootstrap after a restart. * * @param taskWrapper - a wrapper containing task metadata */ private void assignTask(TaskWrapper taskWrapper) { - WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); + try { + WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); + + // If a worker is already running this task, we don't need to announce it + if (workerWrapper != null) { + final Worker worker = workerWrapper.getWorker(); - // If a worker is already running this task, we don't need to announce it - if (workerWrapper != null) { - final Worker worker = workerWrapper.getWorker(); - try { log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskWrapper.getTask().getId()); + runningTasks.put(taskWrapper.getTask().getId(), pendingTasks.remove(taskWrapper.getTask().getId())); - final ChildData workerData = workerWrapper.getStatusCache() + final ChildData statusData = workerWrapper.getStatusCache() .getCurrentData( JOINER.join( config.getStatusPath(), @@ -237,82 +332,70 @@ public class RemoteTaskRunner implements TaskRunner ) ); - if (workerData != null && workerData.getData() != null) { - final TaskStatus taskStatus = jsonMapper.readValue( - workerData.getData(), - TaskStatus.class - ); + final TaskStatus taskStatus = jsonMapper.readValue( + statusData.getData(), + TaskStatus.class + ); - TaskCallback callback = taskWrapper.getCallback(); - if (callback != null) { - callback.notify(taskStatus); - } + TaskCallback callback = taskWrapper.getCallback(); + if (callback != null) { + callback.notify(taskStatus); + } - if (taskStatus.isComplete()) { - new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()).run(); - } - } else { - log.warn("Worker data was null for worker: %s", worker.getHost()); + if (taskStatus.isComplete()) { + cleanup(worker.getHost(), taskWrapper.getTask().getId()); + } + } else { + // Announce the task or retry if there is not enough capacity + workerWrapper = findWorkerForTask(); + if (workerWrapper != null) { + announceTask(workerWrapper.getWorker(), taskWrapper); } } - catch (Exception e) { - log.error(e, "Task exists, but hit exception!"); - retryTask(new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()), taskWrapper); - } - } else { - // Announce the task or retry if there is not enough capacity - workerWrapper = findWorkerForTask(); - if (workerWrapper != null) { - announceTask(workerWrapper.getWorker(), taskWrapper); - } else { - retryTask(null, taskWrapper); - } + } + catch (Exception e) { + throw Throwables.propagate(e); } } /** - * Retries a task that has failed. + * Creates a ZK entry under a specific path associated with a worker. The worker is responsible for + * removing the task ZK entry and creating a task status ZK entry. * - * @param pre - A runnable that is executed before the retry occurs - * @param taskWrapper - a container for task properties + * @param theWorker The worker the task is assigned to + * @param taskWrapper The task to be assigned */ - private void retryTask( - final Runnable pre, - final TaskWrapper taskWrapper - ) + private void announceTask(Worker theWorker, TaskWrapper taskWrapper) throws Exception { + final Task task = taskWrapper.getTask(); - final RetryPolicy retryPolicy = taskWrapper.getRetryPolicy(); + final TaskContext taskContext = taskWrapper.getTaskContext(); - log.info("Registering retry for failed task[%s]", task.getId()); + log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId()); - if (retryPolicy.hasExceededRetryThreshold()) { - log.makeAlert("Task exceeded maximum retry count") - .addData("task", task.getId()) - .addData("retryCount", retryPolicy.getNumRetries()) - .emit(); - return; + byte[] rawBytes = jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)); + + if (rawBytes.length > config.getMaxNumBytes()) { + throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes()); } - scheduledExec.schedule( - new Runnable() - { - @Override - public void run() - { - if (pre != null) { - pre.run(); - } + cf.create() + .withMode(CreateMode.EPHEMERAL) + .forPath( + JOINER.join( + config.getTaskPath(), + theWorker.getHost(), + task.getId() + ), + jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)) + ); - if (tasks.containsKey(task.getId())) { - log.info("Retry[%d] for task[%s]", retryPolicy.getNumRetries(), task.getId()); - assignTask(taskWrapper); - } - } - }, - retryPolicy.getAndIncrementRetryDelay().getMillis(), - TimeUnit.MILLISECONDS - ); + synchronized (statusLock) { + // Syncing state with Zookeeper - wait for task to go into running queue + while (pendingTasks.containsKey(task.getId())) { + statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis()); + } + } } /** @@ -340,94 +423,100 @@ public class RemoteTaskRunner implements TaskRunner @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { - synchronized (statusLock) { - try { - if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || - event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { - final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); - final TaskStatus taskStatus; - // This can fail if a worker writes a bogus status. Retry if so. - try { - taskStatus = jsonMapper.readValue( - event.getData().getData(), TaskStatus.class - ); + try { + if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || + event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { + final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); + final TaskStatus taskStatus; - if (!taskStatus.getId().equals(taskId)) { - // Sanity check - throw new ISE( - "Worker[%s] status id does not match payload id: %s != %s", - worker.getHost(), - taskId, - taskStatus.getId() - ); - } - } - catch (Exception e) { - log.warn(e, "Worker[%s] wrote bogus status for task: %s", worker.getHost(), taskId); - retryTask(new CleanupPaths(worker.getHost(), taskId), tasks.get(taskId)); - throw Throwables.propagate(e); - } - - log.info( - "Worker[%s] wrote %s status for task: %s", - worker.getHost(), - taskStatus.getStatusCode(), - taskId + // This can fail if a worker writes a bogus status. Retry if so. + try { + taskStatus = jsonMapper.readValue( + event.getData().getData(), TaskStatus.class ); - statusLock.notify(); - - final TaskWrapper taskWrapper = tasks.get(taskId); - if (taskWrapper == null) { - log.warn( - "WTF?! Worker[%s] announcing a status for a task I didn't know about: %s", + if (!taskStatus.getId().equals(taskId)) { + // Sanity check + throw new ISE( + "Worker[%s] status id does not match payload id: %s != %s", worker.getHost(), - taskId + taskId, + taskStatus.getId() ); - } else { - final TaskCallback callback = taskWrapper.getCallback(); - - if (taskStatus.isComplete()) { - // Cleanup - if (callback != null) { - callback.notify(taskStatus); - } - - // Worker is done with this task - workerWrapper.setLastCompletedTaskTime(new DateTime()); - tasks.remove(taskId); - cf.delete().guaranteed().inBackground().forPath(event.getData().getPath()); - } } } + catch (Exception e) { + log.warn(e, "Worker[%s] wrote bogus status for task: %s", worker.getHost(), taskId); + retryTask(runningTasks.get(taskId), worker.getHost(), taskId); + throw Throwables.propagate(e); + } + + log.info( + "Worker[%s] wrote %s status for task: %s", + worker.getHost(), + taskStatus.getStatusCode(), + taskId + ); + + if (pendingTasks.containsKey(taskId)) { + runningTasks.put(taskId, pendingTasks.remove(taskId)); + } + + synchronized (statusLock) { + statusLock.notify(); + } + + final TaskWrapper taskWrapper = runningTasks.get(taskId); + if (taskWrapper == null) { + log.warn( + "WTF?! Worker[%s] announcing a status for a task I didn't know about: %s", + worker.getHost(), + taskId + ); + } + + if (taskStatus.isComplete()) { + if (taskWrapper != null) { + final TaskCallback callback = taskWrapper.getCallback(); + + // Cleanup + if (callback != null) { + callback.notify(taskStatus); + } + } + + // Worker is done with this task + workerWrapper.setLastCompletedTaskTime(new DateTime()); + cf.delete().guaranteed().inBackground().forPath(event.getData().getPath()); + runningTasks.remove(taskId); + runPendingTasks(); + } } - catch (Exception e) { - log.makeAlert(e, "Failed to handle new worker status") - .addData("worker", worker.getHost()) - .addData("znode", event.getData().getPath()) - .emit(); - } + } + catch (Exception e) { + log.makeAlert(e, "Failed to handle new worker status") + .addData("worker", worker.getHost()) + .addData("znode", event.getData().getPath()) + .emit(); } } } ); zkWorkers.put(worker.getHost(), workerWrapper); statusCache.start(); + + runPendingTasks(); } - catch (Exception e) { + + catch ( + Exception e + ) + + { throw Throwables.propagate(e); } - } - private WorkerWrapper findWorkerRunningTask(TaskWrapper taskWrapper) - { - for (WorkerWrapper workerWrapper : zkWorkers.values()) { - if (workerWrapper.getRunningTasks().contains(taskWrapper.getTask().getId())) { - return workerWrapper; - } - } - return null; } /** @@ -445,9 +534,9 @@ public class RemoteTaskRunner implements TaskRunner tasksToRetry.addAll(cf.getChildren().forPath(JOINER.join(config.getTaskPath(), worker.getHost()))); for (String taskId : tasksToRetry) { - TaskWrapper taskWrapper = tasks.get(taskId); + TaskWrapper taskWrapper = runningTasks.get(taskId); if (taskWrapper != null) { - retryTask(new CleanupPaths(worker.getHost(), taskId), tasks.get(taskId)); + retryTask(runningTasks.get(taskId), worker.getHost(), taskId); } } @@ -500,81 +589,13 @@ public class RemoteTaskRunner implements TaskRunner } } - /** - * Creates a ZK entry under a specific path associated with a worker. The worker is responsible for - * removing the task ZK entry and creating a task status ZK entry. - * - * @param theWorker The worker the task is assigned to - * @param taskWrapper The task to be assigned - */ - private void announceTask(Worker theWorker, TaskWrapper taskWrapper) + private WorkerWrapper findWorkerRunningTask(TaskWrapper taskWrapper) { - synchronized (statusLock) { - final Task task = taskWrapper.getTask(); - final TaskContext taskContext = taskWrapper.getTaskContext(); - try { - log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId()); - - tasks.put(task.getId(), taskWrapper); - - byte[] rawBytes = jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)); - - if (rawBytes.length > config.getMaxNumBytes()) { - throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes()); - } - - cf.create() - .withMode(CreateMode.EPHEMERAL) - .forPath( - JOINER.join( - config.getTaskPath(), - theWorker.getHost(), - task.getId() - ), - jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)) - ); - - // Syncing state with Zookeeper - while (findWorkerRunningTask(taskWrapper) == null) { - statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis()); - } - } - catch (Exception e) { - log.error(e, "Exception creating task[%s] for worker node[%s]", task.getId(), theWorker.getHost()); - throw Throwables.propagate(e); - } - } - } - - private class CleanupPaths implements Runnable - { - private final String workerId; - private final String taskId; - - private CleanupPaths(String workerId, String taskId) - { - this.workerId = workerId; - this.taskId = taskId; - } - - @Override - public void run() - { - try { - final String statusPath = JOINER.join(config.getStatusPath(), workerId, taskId); - cf.delete().guaranteed().forPath(statusPath); - } - catch (Exception e) { - log.warn("Tried to delete a status path that didn't exist! Must've gone away already?"); - } - - try { - final String taskPath = JOINER.join(config.getTaskPath(), workerId, taskId); - cf.delete().guaranteed().forPath(taskPath); - } - catch (Exception e) { - log.warn("Tried to delete a task path that didn't exist! Must've gone away already?"); + for (WorkerWrapper workerWrapper : zkWorkers.values()) { + if (workerWrapper.getRunningTasks().contains(taskWrapper.getTask().getId())) { + return workerWrapper; } } + return null; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java index 826a3b639cb..4743b2bf4b4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java @@ -22,6 +22,8 @@ package com.metamx.druid.merger.coordinator; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.task.Task; +import java.util.Collection; + /** * Interface for handing off tasks. Used by a {@link com.metamx.druid.merger.coordinator.exec.TaskConsumer} to run tasks that * have been locked. @@ -37,4 +39,10 @@ public interface TaskRunner * @param callback callback to be called exactly once */ public void run(Task task, TaskContext context, TaskCallback callback); + + public Collection getRunningTasks(); + + public Collection getPendingTasks(); + + public Collection getWorkers(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java index 22e7d4152b4..6acecf03f05 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java @@ -19,24 +19,33 @@ package com.metamx.druid.merger.coordinator; +import com.google.common.primitives.Longs; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.task.Task; /** */ -public class TaskWrapper +public class TaskWrapper implements Comparable { private final Task task; private final TaskContext taskContext; private final TaskCallback callback; private final RetryPolicy retryPolicy; + private final long createdTime; - public TaskWrapper(Task task, TaskContext taskContext, TaskCallback callback, RetryPolicy retryPolicy) + public TaskWrapper( + Task task, + TaskContext taskContext, + TaskCallback callback, + RetryPolicy retryPolicy, + long createdTime + ) { this.task = task; this.taskContext = taskContext; this.callback = callback; this.retryPolicy = retryPolicy; + this.createdTime = createdTime; } public Task getTask() @@ -58,4 +67,65 @@ public class TaskWrapper { return retryPolicy; } + + public long getCreatedTime() + { + return createdTime; + } + + @Override + public int compareTo(TaskWrapper taskWrapper) + { + return Longs.compare(createdTime, taskWrapper.getCreatedTime()); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + TaskWrapper that = (TaskWrapper) o; + + if (callback != null ? !callback.equals(that.callback) : that.callback != null) { + return false; + } + if (retryPolicy != null ? !retryPolicy.equals(that.retryPolicy) : that.retryPolicy != null) { + return false; + } + if (task != null ? !task.equals(that.task) : that.task != null) { + return false; + } + if (taskContext != null ? !taskContext.equals(that.taskContext) : that.taskContext != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = task != null ? task.hashCode() : 0; + result = 31 * result + (taskContext != null ? taskContext.hashCode() : 0); + result = 31 * result + (callback != null ? callback.hashCode() : 0); + result = 31 * result + (retryPolicy != null ? retryPolicy.hashCode() : 0); + return result; + } + + @Override + public String toString() + { + return "TaskWrapper{" + + "task=" + task + + ", taskContext=" + taskContext + + ", callback=" + callback + + ", retryPolicy=" + retryPolicy + + ", createdTime=" + createdTime + + '}'; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index c6353698cbd..2e164e9de75 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -104,4 +104,15 @@ public class WorkerWrapper implements Closeable { statusCache.close(); } + + @Override + public String toString() + { + return "WorkerWrapper{" + + "worker=" + worker + + ", statusCache=" + statusCache + + ", cacheConverter=" + cacheConverter + + ", lastCompletedTaskTime=" + lastCompletedTaskTime + + '}'; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java index 44b3a1d4c8c..14e1656e8b8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -20,7 +20,6 @@ package com.metamx.druid.merger.coordinator.config; import com.metamx.druid.merger.common.config.IndexerZkConfig; -import org.joda.time.DateTime; import org.joda.time.Duration; import org.skife.config.Config; import org.skife.config.Default; @@ -29,22 +28,6 @@ import org.skife.config.Default; */ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig { - @Config("druid.indexer.terminateResources.duration") - @Default("PT1H") - public abstract Duration getTerminateResourcesDuration(); - - @Config("druid.indexer.terminateResources.originDateTime") - @Default("2012-01-01T00:55:00.000Z") - public abstract DateTime getTerminateResourcesOriginDateTime(); - - @Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion") - @Default("600000") - public abstract int getMaxWorkerIdleTimeMillisBeforeDeletion(); - - @Config("druid.indexer.maxScalingDuration") - @Default("PT1H") - public abstract Duration getMaxScalingDuration(); - @Config("druid.indexer.taskAssignmentTimeoutDuration") @Default("PT5M") public abstract Duration getTaskAssignmentTimeoutDuration(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 242c7873b6d..4ba09762485 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -543,7 +543,7 @@ public class IndexerCoordinatorNode extends RegisteringNode workerSetupManager ); - initializeWorkerScaling(remoteTaskRunner); + initializeResourceManagement(remoteTaskRunner); return remoteTaskRunner; } @@ -565,7 +565,7 @@ public class IndexerCoordinatorNode extends RegisteringNode } } - private void initializeWorkerScaling(RemoteTaskRunner taskRunner) + private void initializeResourceManagement(TaskRunner taskRunner) { final ScheduledExecutorService scalingScheduledExec = Executors.newScheduledThreadPool( 1, @@ -595,7 +595,6 @@ public class IndexerCoordinatorNode extends RegisteringNode } ResourceManagmentScheduler resourceManagmentScheduler = new ResourceManagmentScheduler( - taskQueue, taskRunner, new SimpleResourceManagementStrategy( strategy, diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java index 37483082e98..92599ae00a2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java @@ -19,7 +19,7 @@ package com.metamx.druid.merger.coordinator.scaling; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.TaskWrapper; import com.metamx.druid.merger.coordinator.WorkerWrapper; import java.util.Collection; @@ -30,9 +30,9 @@ import java.util.Collection; */ public interface ResourceManagementStrategy { - public void doProvision(Collection availableTasks, Collection workerWrappers); + public boolean doProvision(Collection runningTasks, Collection workerWrappers); - public void doTerminate(Collection availableTasks, Collection workerWrappers); + public boolean doTerminate(Collection runningTasks, Collection workerWrappers); public ScalingStats getStats(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagmentScheduler.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagmentScheduler.java index b2ae623b978..9595b026043 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagmentScheduler.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagmentScheduler.java @@ -24,8 +24,7 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.PeriodGranularity; -import com.metamx.druid.merger.coordinator.RemoteTaskRunner; -import com.metamx.druid.merger.coordinator.TaskQueue; +import com.metamx.druid.merger.coordinator.TaskRunner; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; @@ -33,18 +32,17 @@ import org.joda.time.Period; import java.util.concurrent.ScheduledExecutorService; /** - * The ResourceManagmentScheduler manages when worker nodes should potentially be created or destroyed. - * It uses a {@link TaskQueue} to return the available tasks in the system and a {@link RemoteTaskRunner} to return - * the status of the worker nodes in the system. - * The ResourceManagmentScheduler does not contain the logic to decide whether provision or termination should actually occur. - * That decision is made in the {@link ResourceManagementStrategy}. + * The ResourceManagmentScheduler schedules a check for when worker nodes should potentially be created or destroyed. + * It uses a {@link TaskRunner} to return all pending tasks in the system and the status of the worker nodes in + * the system. + * The ResourceManagmentScheduler does not contain the logic to decide whether provision or termination should actually + * occur. That decision is made in the {@link ResourceManagementStrategy}. */ public class ResourceManagmentScheduler { private static final Logger log = new Logger(ResourceManagmentScheduler.class); - private final TaskQueue taskQueue; - private final RemoteTaskRunner remoteTaskRunner; + private final TaskRunner taskRunner; private final ResourceManagementStrategy resourceManagementStrategy; private final ResourceManagementSchedulerConfig config; private final ScheduledExecutorService exec; @@ -53,15 +51,13 @@ public class ResourceManagmentScheduler private volatile boolean started = false; public ResourceManagmentScheduler( - TaskQueue taskQueue, - RemoteTaskRunner remoteTaskRunner, + TaskRunner taskRunner, ResourceManagementStrategy resourceManagementStrategy, ResourceManagementSchedulerConfig config, ScheduledExecutorService exec ) { - this.taskQueue = taskQueue; - this.remoteTaskRunner = remoteTaskRunner; + this.taskRunner = taskRunner; this.resourceManagementStrategy = resourceManagementStrategy; this.config = config; this.exec = exec; @@ -84,8 +80,8 @@ public class ResourceManagmentScheduler public void run() { resourceManagementStrategy.doProvision( - taskQueue.getAvailableTasks(), - remoteTaskRunner.getAvailableWorkers() + taskRunner.getPendingTasks(), + taskRunner.getWorkers() ); } } @@ -109,8 +105,8 @@ public class ResourceManagmentScheduler public void run() { resourceManagementStrategy.doTerminate( - taskQueue.getAvailableTasks(), - remoteTaskRunner.getAvailableWorkers() + taskRunner.getPendingTasks(), + taskRunner.getWorkers() ); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java index d632a61baae..7d65b9aa2ef 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java @@ -3,26 +3,37 @@ package com.metamx.druid.merger.coordinator.scaling; import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; import org.joda.time.DateTime; -import org.joda.time.DateTimeComparator; +import java.util.Collections; +import java.util.Comparator; import java.util.List; /** */ public class ScalingStats { - private static enum EVENT + public static enum EVENT { PROVISION, TERMINATE } + private static final Comparator comparator = new Comparator() + + { + @Override + public int compare(ScalingEvent s1, ScalingEvent s2) + { + return -s1.getTimestamp().compareTo(s2.getTimestamp()); + } + }; + private final MinMaxPriorityQueue recentNodes; public ScalingStats(int capacity) { this.recentNodes = MinMaxPriorityQueue - .orderedBy(DateTimeComparator.getInstance()) + .orderedBy(comparator) .maximumSize(capacity) .create(); } @@ -51,10 +62,8 @@ public class ScalingStats public List toList() { - List retVal = Lists.newArrayList(); - while (!recentNodes.isEmpty()) { - retVal.add(recentNodes.poll()); - } + List retVal = Lists.newArrayList(recentNodes); + Collections.sort(retVal, comparator); return retVal; } @@ -75,6 +84,21 @@ public class ScalingStats this.event = event; } + public AutoScalingData getData() + { + return data; + } + + public DateTime getTimestamp() + { + return timestamp; + } + + public EVENT getEvent() + { + return event; + } + @Override public String toString() { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java index 99c581e5cce..cf464253d54 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -23,9 +23,8 @@ import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.TaskWrapper; import com.metamx.druid.merger.coordinator.WorkerWrapper; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.EmittingLogger; @@ -34,7 +33,6 @@ import org.joda.time.Duration; import java.util.Collection; import java.util.List; -import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; /** @@ -67,54 +65,39 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat } @Override - public void doProvision(Collection availableTasks, Collection workerWrappers) + public boolean doProvision(Collection pendingTasks, Collection workerWrappers) { - boolean nothingProvisioning = Sets.difference( - currentlyProvisioning, - Sets.newHashSet( - autoScalingStrategy.ipToIdLookup( - Lists.newArrayList( - Iterables.transform( - workerWrappers, new Function() - { - @Override - public String apply(WorkerWrapper input) - { - return input.getWorker().getIp(); - } - } - ) - ) + List workerNodeIds = autoScalingStrategy.ipToIdLookup( + Lists.newArrayList( + Iterables.transform( + workerWrappers, + new Function() + { + @Override + public String apply(WorkerWrapper input) + { + return input.getWorker().getIp(); + } + } ) ) - ).isEmpty(); + ); - boolean moreTasksThanWorkerCapacity = !Sets.difference( - Sets.newHashSet(availableTasks), - Sets.newHashSet( - Iterables.concat( - Iterables.transform( - workerWrappers, - new Function>() - { - @Override - public Set apply(WorkerWrapper input) - { - return input.getRunningTasks(); - } - } - ) - ) - ) - ).isEmpty(); + for (String workerNodeId : workerNodeIds) { + currentlyProvisioning.remove(workerNodeId); + } - if (nothingProvisioning && moreTasksThanWorkerCapacity) { + boolean nothingProvisioning = currentlyProvisioning.isEmpty(); + + if (nothingProvisioning && hasTaskPendingBeyondThreshold(pendingTasks)) { AutoScalingData provisioned = autoScalingStrategy.provision(); if (provisioned != null) { currentlyProvisioning.addAll(provisioned.getNodeIds()); lastProvisionTime = new DateTime(); scalingStats.addProvisionEvent(provisioned); + + return true; } } else { Duration durSinceLastProvision = new Duration(new DateTime(), lastProvisionTime); @@ -130,35 +113,39 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat currentlyProvisioning ); } + + return false; } @Override - public void doTerminate(Collection availableTasks, Collection workerWrappers) + public boolean doTerminate(Collection pendingTasks, Collection workerWrappers) { - boolean nothingTerminating = Sets.difference( - currentlyTerminating, - Sets.newHashSet( - autoScalingStrategy.ipToIdLookup( - Lists.newArrayList( - Iterables.transform( - workerWrappers, new Function() - { - @Override - public String apply(WorkerWrapper input) - { - return input.getWorker().getIp(); - } - } - ) - ) + List workerNodeIds = autoScalingStrategy.ipToIdLookup( + Lists.newArrayList( + Iterables.transform( + workerWrappers, + new Function() + { + @Override + public String apply(WorkerWrapper input) + { + return input.getWorker().getIp(); + } + } ) ) - ).isEmpty(); + ); + + for (String workerNodeId : workerNodeIds) { + currentlyTerminating.remove(workerNodeId); + } + + boolean nothingTerminating = currentlyTerminating.isEmpty(); if (nothingTerminating) { final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers(); if (workerWrappers.size() <= minNumWorkers) { - return; + return false; } List thoseLazyWorkers = Lists.newArrayList( @@ -171,14 +158,17 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat public boolean apply(WorkerWrapper input) { return input.getRunningTasks().isEmpty() - && System.currentTimeMillis() - input.getLastCompletedTaskTime() - .getMillis() - > config.getMaxWorkerIdleTimeMillisBeforeDeletion(); + && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() + >= config.getMaxWorkerIdleTimeMillisBeforeDeletion(); } } ) ); + if (thoseLazyWorkers.isEmpty()) { + return false; + } + AutoScalingData terminated = autoScalingStrategy.terminate( Lists.transform( thoseLazyWorkers.subList(minNumWorkers, thoseLazyWorkers.size() - 1), @@ -196,7 +186,9 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat if (terminated != null) { currentlyTerminating.addAll(terminated.getNodeIds()); lastTerminateTime = new DateTime(); - scalingStats.addProvisionEvent(terminated); + scalingStats.addTerminateEvent(terminated); + + return true; } } else { Duration durSinceLastTerminate = new Duration(new DateTime(), lastTerminateTime); @@ -212,6 +204,8 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat currentlyTerminating ); } + + return false; } @Override @@ -219,4 +213,16 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat { return scalingStats; } + + private boolean hasTaskPendingBeyondThreshold(Collection pendingTasks) + { + long now = System.currentTimeMillis(); + for (TaskWrapper pendingTask : pendingTasks) { + if (new Duration(pendingTask.getCreatedTime(), now).isEqual(config.getMaxPendingTaskDuration()) || + new Duration(pendingTask.getCreatedTime(), now).isLongerThan(config.getMaxPendingTaskDuration())) { + return true; + } + } + return false; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java index a8a5b52ca89..79f6b7f5db9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java @@ -38,4 +38,8 @@ public abstract class SimpleResourceManagmentConfig @Config("druid.indexer.numEventsToTrack") @Default("20") public abstract int getNumEventsToTrack(); + + @Config("druid.indexer.maxPendingTaskDuration") + @Default("20") + public abstract Duration getMaxPendingTaskDuration(); } diff --git a/merger/src/test/java/com/metamx/druid/merger/TestTask.java b/merger/src/test/java/com/metamx/druid/merger/TestTask.java index 142d2c88254..44b0852e3af 100644 --- a/merger/src/test/java/com/metamx/druid/merger/TestTask.java +++ b/merger/src/test/java/com/metamx/druid/merger/TestTask.java @@ -37,17 +37,20 @@ import java.util.List; public class TestTask extends DefaultMergeTask { private final String id; + private final TaskStatus status; public TestTask( @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments, - @JsonProperty("aggregations") List aggregators + @JsonProperty("aggregations") List aggregators, + @JsonProperty("taskStatus") TaskStatus status ) { super(dataSource, segments, aggregators); this.id = id; + this.status = status; } @Override @@ -63,9 +66,15 @@ public class TestTask extends DefaultMergeTask return Type.TEST; } + @JsonProperty + public TaskStatus getStatus() + { + return status; + } + @Override public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception { - return TaskStatus.success("task1"); + return status; } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index e4527b70d99..9d6b6ca4e76 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -13,7 +13,6 @@ import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; -import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; @@ -40,7 +39,6 @@ import org.junit.Before; import org.junit.Test; import java.io.File; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -65,7 +63,7 @@ public class RemoteTaskRunnerTest private ScheduledExecutorService scheduledExec; - private Task task1; + private TestTask task1; private Worker worker1; @@ -111,7 +109,9 @@ public class RemoteTaskRunnerTest 0, 0 ) - ), Lists.newArrayList() + ), + Lists.newArrayList(), + TaskStatus.success("task1") ); makeRemoteTaskRunner(); @@ -137,26 +137,16 @@ public class RemoteTaskRunnerTest } @Test - public void testAlreadyExecutedTask() throws Exception + public void testExceptionThrownWithExistingTask() throws Exception { - final CountDownLatch latch = new CountDownLatch(1); remoteTaskRunner.run( new TestTask( task1.getId(), task1.getDataSource(), - Lists.newArrayList(), - Lists.newArrayList() - ) - { - @Override - public TaskStatus run( - TaskContext context, TaskToolbox toolbox, TaskCallback callback - ) throws Exception - { - latch.await(); - return super.run(context, toolbox, callback); - } - }, + task1.getSegments(), + Lists.newArrayList(), + TaskStatus.running(task1.getId()) + ), new TaskContext(new DateTime().toString(), Sets.newHashSet(), Sets.newHashSet()), null ); @@ -170,11 +160,14 @@ public class RemoteTaskRunnerTest ), null ); - latch.countDown(); fail("ISE expected"); } catch (ISE expected) { - latch.countDown(); + } + finally { + cf.delete().guaranteed().forPath( + String.format("%s/worker1/task1", statusPath) + ); } } @@ -199,7 +192,9 @@ public class RemoteTaskRunnerTest 0, 0 ) - ), Lists.newArrayList() + ), + Lists.newArrayList(), + TaskStatus.success("foo") ), new TaskContext(new DateTime().toString(), Sets.newHashSet(), Sets.newHashSet()), null @@ -212,29 +207,18 @@ public class RemoteTaskRunnerTest } @Test - public void testRunWithExistingCompletedTask() throws Exception + public void testRunWithCallback() throws Exception { - cf.create().creatingParentsIfNeeded().forPath( - String.format("%s/worker1/task1", statusPath), - jsonMapper.writeValueAsBytes( - TaskStatus.success("task1") - ) - ); - - // Really don't like this way of waiting for the task to appear - int count = 0; - while (!remoteTaskRunner.isTaskRunning("task1")) { - Thread.sleep(500); - if (count > 10) { - throw new ISE("WTF?! Task still not announced in ZK?"); - } - count++; - } - final MutableBoolean callbackCalled = new MutableBoolean(false); remoteTaskRunner.run( - task1, - null, + new TestTask( + task1.getId(), + task1.getDataSource(), + task1.getSegments(), + Lists.newArrayList(), + TaskStatus.running(task1.getId()) + ), + new TaskContext(new DateTime().toString(), Sets.newHashSet(), Sets.newHashSet()), new TaskCallback() { @Override @@ -245,6 +229,34 @@ public class RemoteTaskRunnerTest } ); + // Really don't like this way of waiting for the task to appear + int count = 0; + while (!remoteTaskRunner.isTaskRunning(task1.getId())) { + Thread.sleep(500); + if (count > 10) { + throw new ISE("WTF?! Task still not announced in ZK?"); + } + count++; + } + + Assert.assertTrue(remoteTaskRunner.getRunningTasks().size() == 1); + + // Complete the task + cf.setData().forPath( + String.format("%s/worker1/task1", statusPath), + jsonMapper.writeValueAsBytes(TaskStatus.success(task1.getId())) + ); + + // Really don't like this way of waiting for the task to disappear + count = 0; + while (remoteTaskRunner.isTaskRunning(task1.getId())) { + Thread.sleep(500); + if (count > 10) { + throw new ISE("WTF?! Task still not announced in ZK?"); + } + count++; + } + Assert.assertTrue("TaskCallback was not called!", callbackCalled.booleanValue()); } @@ -326,7 +338,7 @@ public class RemoteTaskRunnerTest null, null ) - ); + ).atLeastOnce(); EasyMock.replay(workerSetupManager); remoteTaskRunner = new RemoteTaskRunner( @@ -346,7 +358,7 @@ public class RemoteTaskRunnerTest jsonMapper.writeValueAsBytes(worker1) ); int count = 0; - while (remoteTaskRunner.getNumAvailableWorkers() == 0) { + while (remoteTaskRunner.getWorkers().size() == 0) { Thread.sleep(500); if (count > 10) { throw new ISE("WTF?! Still can't find worker!"); @@ -378,30 +390,6 @@ public class RemoteTaskRunnerTest private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig { - @Override - public Duration getTerminateResourcesDuration() - { - return null; - } - - @Override - public DateTime getTerminateResourcesOriginDateTime() - { - return null; - } - - @Override - public int getMaxWorkerIdleTimeMillisBeforeDeletion() - { - return 0; - } - - @Override - public Duration getMaxScalingDuration() - { - return null; - } - @Override public String getAnnouncementPath() { diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java index 10398da005f..5755a2b991e 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -24,9 +24,14 @@ import com.google.common.collect.Sets; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.TestTask; +import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.TaskWrapper; import com.metamx.druid.merger.coordinator.WorkerWrapper; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.worker.Worker; +import junit.framework.Assert; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -35,6 +40,7 @@ import org.junit.Before; import org.junit.Test; import java.util.Arrays; +import java.util.List; import java.util.Set; /** @@ -58,18 +64,21 @@ public class SimpleResourceManagementStrategyTest Lists.newArrayList( new DataSegment( "dummyDs", - new Interval(new DateTime(), new DateTime()), + new Interval("2012-01-01/2012-01-02"), new DateTime().toString(), null, null, null, null, + 0, 0 ) - ), Lists.newArrayList() + ), + Lists.newArrayList(), + TaskStatus.success("task1") ); simpleResourceManagementStrategy = new SimpleResourceManagementStrategy( - new TestAutoScalingStrategy(), + autoScalingStrategy, new SimpleResourceManagmentConfig() { @Override @@ -89,6 +98,12 @@ public class SimpleResourceManagementStrategyTest { return 1; } + + @Override + public Duration getMaxPendingTaskDuration() + { + return new Duration(0); + } }, workerSetupManager ); @@ -97,27 +112,155 @@ public class SimpleResourceManagementStrategyTest @Test public void testSuccessfulProvision() throws Exception { + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList()); EasyMock.expect(autoScalingStrategy.provision()).andReturn( new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()) ); EasyMock.replay(autoScalingStrategy); - simpleResourceManagementStrategy.doProvision( - Arrays.asList( - testTask + boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( + Arrays.asList( + new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) ), Arrays.asList( new TestWorkerWrapper(testTask) ) ); + Assert.assertTrue(provisionedSomething); + Assert.assertTrue(simpleResourceManagementStrategy.getStats().toList().size() == 1); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.PROVISION + ); + EasyMock.verify(autoScalingStrategy); } @Test - public void testDoTerminate() throws Exception + public void testSomethingProvisioning() throws Exception { + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList()).times(2); + EasyMock.expect(autoScalingStrategy.provision()).andReturn( + new AutoScalingData(Lists.newArrayList("fake"), Lists.newArrayList("faker")) + ); + EasyMock.replay(autoScalingStrategy); + boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( + Arrays.asList( + new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) + ), + Arrays.asList( + new TestWorkerWrapper(testTask) + ) + ); + + Assert.assertTrue(provisionedSomething); + Assert.assertTrue(simpleResourceManagementStrategy.getStats().toList().size() == 1); + DateTime createdTime = simpleResourceManagementStrategy.getStats().toList().get(0).getTimestamp(); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.PROVISION + ); + + provisionedSomething = simpleResourceManagementStrategy.doProvision( + Arrays.asList( + new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) + ), + Arrays.asList( + new TestWorkerWrapper(testTask) + ) + ); + + Assert.assertFalse(provisionedSomething); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.PROVISION + ); + DateTime anotherCreatedTime = simpleResourceManagementStrategy.getStats().toList().get(0).getTimestamp(); + Assert.assertTrue( + createdTime.equals(anotherCreatedTime) + ); + + EasyMock.verify(autoScalingStrategy); + } + + @Test + public void testDoSuccessfulTerminate() throws Exception + { + EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); + EasyMock.replay(workerSetupManager); + + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList()); + EasyMock.expect(autoScalingStrategy.terminate(EasyMock.>anyObject())).andReturn( + new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()) + ); + EasyMock.replay(autoScalingStrategy); + + boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( + Arrays.asList( + new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) + ), + Arrays.asList( + new TestWorkerWrapper(null) + ) + ); + + Assert.assertTrue(terminatedSomething); + Assert.assertTrue(simpleResourceManagementStrategy.getStats().toList().size() == 1); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE + ); + + EasyMock.verify(workerSetupManager); + EasyMock.verify(autoScalingStrategy); + } + + @Test + public void testSomethingTerminating() throws Exception + { + EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); + EasyMock.replay(workerSetupManager); + + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList()).times(2); + EasyMock.expect(autoScalingStrategy.terminate(EasyMock.>anyObject())).andReturn( + new AutoScalingData(Lists.newArrayList("foobar"), Lists.newArrayList("foobrick")) + ); + EasyMock.replay(autoScalingStrategy); + + boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( + Arrays.asList( + new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) + ), + Arrays.asList( + new TestWorkerWrapper(null) + ) + ); + + Assert.assertTrue(terminatedSomething); + Assert.assertTrue(simpleResourceManagementStrategy.getStats().toList().size() == 1); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE + ); + + terminatedSomething = simpleResourceManagementStrategy.doTerminate( + Arrays.asList( + new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) + ), + Arrays.asList( + new TestWorkerWrapper(null) + ) + ); + + Assert.assertFalse(terminatedSomething); + Assert.assertTrue(simpleResourceManagementStrategy.getStats().toList().size() == 1); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE + ); + + EasyMock.verify(workerSetupManager); + EasyMock.verify(autoScalingStrategy); } private static class TestWorkerWrapper extends WorkerWrapper @@ -128,7 +271,7 @@ public class SimpleResourceManagementStrategyTest Task testTask ) { - super(null, null, null); + super(new Worker("host", "ip", 3, "version"), null, null); this.testTask = testTask; } @@ -136,6 +279,9 @@ public class SimpleResourceManagementStrategyTest @Override public Set getRunningTasks() { + if (testTask == null) { + return Sets.newHashSet(); + } return Sets.newHashSet(testTask.getId()); } } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 3d9c8ecffe0..ef91bcdcc7a 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -21,6 +21,7 @@ package com.metamx.druid.master; import com.google.common.base.Function; import com.google.common.base.Predicate; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -42,9 +43,12 @@ import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; +import com.metamx.druid.merge.ClientKillQuery; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.HttpResponseHandler; import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; import com.netflix.curator.x.discovery.ServiceProvider; @@ -53,6 +57,7 @@ import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.Duration; import javax.annotation.Nullable; +import java.net.URL; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -87,6 +92,9 @@ public class DruidMaster private final Map loadManagementPeons; private final ServiceProvider serviceProvider; + private final HttpClient httpClient; + private final HttpResponseHandler responseHandler; + private final ObjectMapper jsonMapper; public DruidMaster( @@ -100,7 +108,9 @@ public class DruidMaster ServiceEmitter emitter, ScheduledExecutorFactory scheduledExecutorFactory, Map loadManagementPeons, - ServiceProvider serviceProvider + ServiceProvider serviceProvider, + HttpClient httpClient, + HttpResponseHandler responseHandler ) { this.config = config; @@ -121,6 +131,8 @@ public class DruidMaster this.loadManagementPeons = loadManagementPeons; this.serviceProvider = serviceProvider; + this.httpClient = httpClient; + this.responseHandler = responseHandler; } public boolean isClusterMaster() @@ -332,6 +344,27 @@ public class DruidMaster } } + public void killSegments(ClientKillQuery killQuery) + { + try { + httpClient.post( + new URL( + String.format( + "http://%s:%s/mmx/merger/v1/index", + serviceProvider.getInstance().getAddress(), + serviceProvider.getInstance().getPort() + ) + ) + ) + .setContent("application/json", jsonMapper.writeValueAsBytes(killQuery)) + .go(responseHandler) + .get(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + public Set getAvailableDataSegments() { Set availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator())); @@ -590,7 +623,7 @@ public class DruidMaster public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) { // Display info about all historical servers - Iterable servers =FunctionalIterable + Iterable servers = FunctionalIterable .create(serverInventoryManager.getInventory()) .filter( new Predicate() From d4009c8c1ce9a101295209e742f4a2de7fe2c9ad Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 18:40:14 -0800 Subject: [PATCH 30/63] refactor filterSegments to use `TimelineObjectHolder`s --- .../druid/client/CachingClusteredClient.java | 51 +++++-------------- .../metamx/druid/query/QueryToolChest.java | 11 +++- .../TimeBoundaryQueryQueryToolChest.java | 17 ++++--- 3 files changed, 34 insertions(+), 45 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index a07478ab751..20c9ca2fea8 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -139,49 +139,26 @@ public class CachingClusteredClient implements QueryRunner // build set of segments to query Set> segments = Sets.newLinkedHashSet(); + List> serversLookup = Lists.newLinkedList(); + for (Interval interval : rewrittenQuery.getIntervals()) { - List> serversLookup = timeline.lookup(interval); - - for (TimelineObjectHolder holder : serversLookup) { - for (PartitionChunk chunk : holder.getObject()) { - ServerSelector selector = chunk.getObject(); - final SegmentDescriptor descriptor = new SegmentDescriptor( - holder.getInterval(), holder.getVersion(), chunk.getChunkNumber() - ); - - segments.add(Pair.of(selector, descriptor)); - } - } + serversLookup.addAll(timeline.lookup(interval)); } // Let tool chest filter out unneeded segments - final Set filteredSegmentDescriptors = Sets.newLinkedHashSet(toolChest.filterSegments( - query, - Iterables.transform( - segments, new Function, SegmentDescriptor>() - { - @Override - public SegmentDescriptor apply( - @Nullable Pair input - ) - { - return input.rhs; - } - } - ) - )); + final List> filteredServersLookup = + toolChest.filterSegments(query, serversLookup); - // remove unneeded segments from list of segments to query - segments = Sets.newLinkedHashSet(Iterables.filter(segments, new Predicate>() - { - @Override - public boolean apply( - @Nullable Pair input - ) - { - return filteredSegmentDescriptors.contains(input.rhs); + for (TimelineObjectHolder holder : filteredServersLookup) { + for (PartitionChunk chunk : holder.getObject()) { + ServerSelector selector = chunk.getObject(); + final SegmentDescriptor descriptor = new SegmentDescriptor( + holder.getInterval(), holder.getVersion(), chunk.getChunkNumber() + ); + + segments.add(Pair.of(selector, descriptor)); } - })); + } final byte[] queryCacheKey; if(strategy != null) { diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 7c462cc08d8..6822e654471 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -23,12 +23,16 @@ import com.google.common.base.Function; import com.metamx.common.Pair; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; +import com.metamx.druid.TimelineObjectHolder; +import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; import org.joda.time.Interval; +import java.util.List; + /** * The broker-side (also used by server in some cases) API for a specific Query type. This API is still undergoing * evolution and is only semi-stable, so proprietary Query implementations should be ready for the potential @@ -61,7 +65,10 @@ public abstract class QueryToolChest filterSegments(QueryType query, Iterable intervals) { - return intervals; + public List> filterSegments( + QueryType query, + List> segments + ) { + return segments; } } diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 8ad7378faf2..388a3c6df85 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -29,6 +29,8 @@ import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.Query; +import com.metamx.druid.TimelineObjectHolder; +import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.collect.OrderedMergeSequence; import com.metamx.druid.query.BySegmentSkippingQueryRunner; import com.metamx.druid.query.CacheStrategy; @@ -64,16 +66,19 @@ public class TimeBoundaryQueryQueryToolChest @Override - public Iterable filterSegments(TimeBoundaryQuery query, Iterable input) { + public List> filterSegments( + TimeBoundaryQuery query, + List> input + ) { long minMillis = Long.MAX_VALUE; long maxMillis = Long.MIN_VALUE; - SegmentDescriptor min = null; - SegmentDescriptor max = null; + TimelineObjectHolder min = null; + TimelineObjectHolder max = null; // keep track of all segments in a given shard - Map> segmentGroups = Maps.newHashMap(); + Map>> segmentGroups = Maps.newHashMap(); - for(SegmentDescriptor e : input) { + for(TimelineObjectHolder e : input) { final long start = e.getInterval().getStartMillis(); final long end = e.getInterval().getEndMillis(); final String version = e.getVersion(); @@ -94,7 +99,7 @@ public class TimeBoundaryQueryQueryToolChest } } - return Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion())); + return Lists.newArrayList(Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion()))); } @Override From 633cccce6918cece6ebdb9ed2174c9a419dd9872 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 17:02:33 -0800 Subject: [PATCH 31/63] bump up to 0.3.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ec6db24d01c..aa77ea6894b 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT druid druid From 7ab2e0552aa954b973d470bf39eb7c23274fc7e9 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 18:42:43 -0800 Subject: [PATCH 32/63] cleanup imports --- .../java/com/metamx/druid/client/CachingClusteredClient.java | 2 -- .../src/main/java/com/metamx/druid/query/QueryToolChest.java | 4 ---- .../query/timeboundary/TimeBoundaryQueryQueryToolChest.java | 1 - 3 files changed, 7 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 20c9ca2fea8..ad08ede04eb 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -20,7 +20,6 @@ package com.metamx.druid.client; import com.google.common.base.Function; -import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -59,7 +58,6 @@ import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 6822e654471..701a4512403 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -20,16 +20,12 @@ package com.metamx.druid.query; import com.google.common.base.Function; -import com.metamx.common.Pair; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.client.selector.ServerSelector; -import com.metamx.druid.query.segment.QuerySegmentSpec; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; -import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 388a3c6df85..b27a6fd5b80 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -37,7 +37,6 @@ import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.MetricManipulationFn; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChest; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; From 68cd2e10a7c42150cb9f5acdef9ca66d7c98b959 Mon Sep 17 00:00:00 2001 From: xvrl Date: Tue, 12 Feb 2013 09:45:26 -0800 Subject: [PATCH 33/63] remove unnecessary segment grouping --- .../TimeBoundaryQueryQueryToolChest.java | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index b27a6fd5b80..8e41f1f6eb8 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -74,19 +74,9 @@ public class TimeBoundaryQueryQueryToolChest TimelineObjectHolder min = null; TimelineObjectHolder max = null; - // keep track of all segments in a given shard - Map>> segmentGroups = Maps.newHashMap(); - for(TimelineObjectHolder e : input) { final long start = e.getInterval().getStartMillis(); final long end = e.getInterval().getEndMillis(); - final String version = e.getVersion(); - - if(segmentGroups.containsKey(version)) { - segmentGroups.get(version).add(e); - } else { - segmentGroups.put(version, Sets.newHashSet(e)); - } if(min == null || start < minMillis) { min = e; @@ -98,7 +88,7 @@ public class TimeBoundaryQueryQueryToolChest } } - return Lists.newArrayList(Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion()))); + return min == max ? Lists.newArrayList(min) : Lists.newArrayList(min , max); } @Override From 07b0b4611bae8063698e220372678e0907b155e4 Mon Sep 17 00:00:00 2001 From: xvrl Date: Tue, 12 Feb 2013 09:52:21 -0800 Subject: [PATCH 34/63] bump up to 0.3.0 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 6 +++--- examples/pom.xml | 2 +- index-common/pom.xml | 4 ++-- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 9 files changed, 12 insertions(+), 12 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 82c917ebecb..36bb84de0d4 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 8d058d643cf..350828b9316 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 9b2567dc753..88994a529d5 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT @@ -66,4 +66,4 @@ - \ No newline at end of file + diff --git a/examples/pom.xml b/examples/pom.xml index fa195a3f0f4..b24a0f7dd42 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 71a7d3192cc..9d17c6b2c53 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT @@ -94,4 +94,4 @@ - \ No newline at end of file + diff --git a/indexer/pom.xml b/indexer/pom.xml index 413582985a2..d38a3a17769 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index c24a262529d..980aa96d5f0 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/realtime/pom.xml b/realtime/pom.xml index 8a71c169fd4..a8c50128f2b 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index bf0a71f4d75..544108cef5f 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT From c7b49737003af88eb84a2b1ca354cfb214623ca1 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 13 Feb 2013 14:19:11 -0800 Subject: [PATCH 35/63] bug fixes for RTR --- .../merger/coordinator/LocalTaskRunner.java | 4 +- .../merger/coordinator/RemoteTaskRunner.java | 210 ++++++++++-------- .../druid/merger/coordinator/RetryPolicy.java | 5 + .../druid/merger/coordinator/TaskMaster.java | 5 + .../druid/merger/coordinator/TaskRunner.java | 4 +- ...skWrapper.java => TaskRunnerWorkItem.java} | 37 ++- .../coordinator/TaskRunnerWorkQueue.java | 35 +++ .../http/IndexerCoordinatorNode.java | 100 +++++---- .../scaling/EC2AutoScalingStrategy.java | 1 - ....java => ResourceManagementScheduler.java} | 11 +- .../ResourceManagementSchedulerConfig.java | 2 +- .../ResourceManagementSchedulerFactory.java | 29 +++ .../scaling/ResourceManagementStrategy.java | 6 +- .../SimpleResourceManagementStrategy.java | 33 +-- .../SimpleResourceManagmentConfig.java | 6 +- .../druid/merger/worker/TaskMonitor.java | 2 + .../merger/worker/config/WorkerConfig.java | 1 + .../coordinator/RemoteTaskRunnerTest.java | 9 +- .../SimpleResourceManagementStrategyTest.java | 26 +-- .../com/metamx/druid/master/DruidMaster.java | 61 ++--- 20 files changed, 356 insertions(+), 231 deletions(-) rename merger/src/main/java/com/metamx/druid/merger/coordinator/{TaskWrapper.java => TaskRunnerWorkItem.java} (77%) create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkQueue.java rename merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/{ResourceManagmentScheduler.java => ResourceManagementScheduler.java} (90%) create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerFactory.java diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index d517833aa58..49f9dcbb237 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -112,13 +112,13 @@ public class LocalTaskRunner implements TaskRunner } @Override - public Collection getRunningTasks() + public Collection getRunningTasks() { return Lists.newArrayList(); } @Override - public Collection getPendingTasks() + public Collection getPendingTasks() { return Lists.newArrayList(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 843462e5e4f..d532b9b492b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -55,7 +55,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -71,7 +70,7 @@ import java.util.concurrent.TimeUnit; *

    * The RemoteTaskRunner will assign tasks to a node until the node hits capacity. At that point, task assignment will * fail. The RemoteTaskRunner depends on another component to create additional worker resources. - * For example, {@link com.metamx.druid.merger.coordinator.scaling.ResourceManagmentScheduler} can take care of these duties. + * For example, {@link com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler} can take care of these duties. *

    * If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will automatically retry any tasks * that were associated with the node. @@ -92,9 +91,9 @@ public class RemoteTaskRunner implements TaskRunner // all workers that exist in ZK private final Map zkWorkers = new ConcurrentHashMap(); // all tasks that have been assigned to a worker - private final ConcurrentSkipListMap runningTasks = new ConcurrentSkipListMap(); + private final TaskRunnerWorkQueue runningTasks = new TaskRunnerWorkQueue(); // tasks that have not yet run - private final ConcurrentSkipListMap pendingTasks = new ConcurrentSkipListMap(); + private final TaskRunnerWorkQueue pendingTasks = new TaskRunnerWorkQueue(); private final ExecutorService runPendingTasksExec = Executors.newSingleThreadExecutor(); @@ -190,20 +189,25 @@ public class RemoteTaskRunner implements TaskRunner } @Override - public Collection getRunningTasks() + public Collection getRunningTasks() { return runningTasks.values(); } @Override - public Collection getPendingTasks() + public Collection getPendingTasks() { return pendingTasks.values(); } - public boolean isTaskRunning(String taskId) + public WorkerWrapper findWorkerRunningTask(String taskId) { - return runningTasks.containsKey(taskId); + for (WorkerWrapper workerWrapper : zkWorkers.values()) { + if (workerWrapper.getRunningTasks().contains(taskId)) { + return workerWrapper; + } + } + return null; } /** @@ -217,23 +221,25 @@ public class RemoteTaskRunner implements TaskRunner public void run(Task task, TaskContext context, TaskCallback callback) { if (runningTasks.containsKey(task.getId()) || pendingTasks.containsKey(task.getId())) { - throw new ISE("Assigned a task[%s] that is already running, WTF is happening?!", task.getId()); + throw new ISE("Assigned a task[%s] that is already running or pending, WTF is happening?!", task.getId()); } - TaskWrapper taskWrapper = new TaskWrapper( - task, context, callback, retryPolicyFactory.makeRetryPolicy(), System.currentTimeMillis() + TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem( + task, context, callback, retryPolicyFactory.makeRetryPolicy(), new DateTime() ); - addPendingTask(taskWrapper); + addPendingTask(taskRunnerWorkItem); } - private void addPendingTask(final TaskWrapper taskWrapper) + private void addPendingTask(final TaskRunnerWorkItem taskRunnerWorkItem) { - pendingTasks.put(taskWrapper.getTask().getId(), taskWrapper); + log.info("Added pending task %s", taskRunnerWorkItem.getTask().getId()); + + pendingTasks.put(taskRunnerWorkItem.getTask().getId(), taskRunnerWorkItem); runPendingTasks(); } /** * This method uses a single threaded executor to extract all pending tasks and attempt to run them. Any tasks that - * are successfully started by a worker will be moved from pendingTasks to runningTasks. This method is thread-safe. + * are successfully assigned to a worker will be moved from pendingTasks to runningTasks. This method is thread-safe. * This method should be run each time there is new worker capacity or if new tasks are assigned. */ private void runPendingTasks() @@ -244,9 +250,10 @@ public class RemoteTaskRunner implements TaskRunner @Override public Void call() throws Exception { - // make a copy of the pending tasks because assignTask may delete tasks from pending and move them into moving - List copy = Lists.newArrayList(pendingTasks.values()); - for (TaskWrapper taskWrapper : copy) { + // make a copy of the pending tasks because assignTask may delete tasks from pending and move them + // into running status + List copy = Lists.newArrayList(pendingTasks.values()); + for (TaskRunnerWorkItem taskWrapper : copy) { assignTask(taskWrapper); } @@ -259,76 +266,88 @@ public class RemoteTaskRunner implements TaskRunner future.get(); } catch (InterruptedException e) { + log.error(e, "InterruptedException in runPendingTasks()"); throw Throwables.propagate(e); } catch (ExecutionException e) { + log.error(e, "ExecutionException in runPendingTasks()"); throw Throwables.propagate(e.getCause()); } } - - private void retryTask(final TaskWrapper taskWrapper, final String workerId, final String taskId) + /** + * Retries a task by inserting it back into the pending queue after a given delay. + * This method will also clean up any status paths that were associated with the task. + * + * @param taskRunnerWorkItem - the task to retry + * @param workerId - the worker that was previously running this task + */ + 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() { - runningTasks.remove(taskId); cleanup(workerId, taskId); - addPendingTask(taskWrapper); + addPendingTask(taskRunnerWorkItem); } }, - taskWrapper.getRetryPolicy().getAndIncrementRetryDelay().getMillis(), + taskRunnerWorkItem.getRetryPolicy().getAndIncrementRetryDelay().getMillis(), TimeUnit.MILLISECONDS ); } + /** + * Removes a task from the running queue and clears out the ZK status path of the task. + * + * @param workerId - the worker that was previously running the task + * @param taskId - the task to cleanup + */ private void cleanup(final String workerId, final String taskId) { + runningTasks.remove(taskId); + final String statusPath = JOINER.join(config.getStatusPath(), workerId, taskId); try { - final String statusPath = JOINER.join(config.getStatusPath(), workerId, taskId); cf.delete().guaranteed().forPath(statusPath); } catch (Exception e) { - log.warn("Tried to delete a status path that didn't exist! Must've gone away already?"); - } - - try { - final String taskPath = JOINER.join(config.getTaskPath(), workerId, taskId); - cf.delete().guaranteed().forPath(taskPath); - } - catch (Exception e) { - log.warn("Tried to delete a task path that didn't exist! Must've gone away already?"); + log.info("Tried to delete status path[%s] that didn't exist! Must've gone away already?", statusPath); } } /** * Ensures no workers are already running a task before assigning the task to a worker. - * It is possible that a worker is running a task that the RTR has no knowledge of. This is common when the RTR + * It is possible that a worker is running a task that the RTR has no knowledge of. This is possible when the RTR * needs to bootstrap after a restart. * - * @param taskWrapper - a wrapper containing task metadata + * @param taskRunnerWorkItem - the task to assign */ - private void assignTask(TaskWrapper taskWrapper) + private void assignTask(TaskRunnerWorkItem taskRunnerWorkItem) { try { - WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); + WorkerWrapper workerWrapper = findWorkerRunningTask(taskRunnerWorkItem.getTask().getId()); // If a worker is already running this task, we don't need to announce it if (workerWrapper != null) { final Worker worker = workerWrapper.getWorker(); - log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskWrapper.getTask().getId()); - runningTasks.put(taskWrapper.getTask().getId(), pendingTasks.remove(taskWrapper.getTask().getId())); + log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskRunnerWorkItem.getTask().getId()); + runningTasks.put( + taskRunnerWorkItem.getTask().getId(), + pendingTasks.remove(taskRunnerWorkItem.getTask().getId()) + ); + log.info("Task %s switched from pending to running", taskRunnerWorkItem.getTask().getId()); final ChildData statusData = workerWrapper.getStatusCache() .getCurrentData( JOINER.join( config.getStatusPath(), worker.getHost(), - taskWrapper.getTask().getId() + taskRunnerWorkItem.getTask().getId() ) ); @@ -337,19 +356,18 @@ public class RemoteTaskRunner implements TaskRunner TaskStatus.class ); - TaskCallback callback = taskWrapper.getCallback(); - if (callback != null) { - callback.notify(taskStatus); - } - if (taskStatus.isComplete()) { - cleanup(worker.getHost(), taskWrapper.getTask().getId()); + TaskCallback callback = taskRunnerWorkItem.getCallback(); + if (callback != null) { + callback.notify(taskStatus); + } + cleanup(worker.getHost(), taskRunnerWorkItem.getTask().getId()); } } else { - // Announce the task or retry if there is not enough capacity + // Nothing running this task, announce it in ZK for a worker to run it workerWrapper = findWorkerForTask(); if (workerWrapper != null) { - announceTask(workerWrapper.getWorker(), taskWrapper); + announceTask(workerWrapper.getWorker(), taskRunnerWorkItem); } } } @@ -362,14 +380,13 @@ public class RemoteTaskRunner implements TaskRunner * Creates a ZK entry under a specific path associated with a worker. The worker is responsible for * removing the task ZK entry and creating a task status ZK entry. * - * @param theWorker The worker the task is assigned to - * @param taskWrapper The task to be assigned + * @param theWorker The worker the task is assigned to + * @param taskRunnerWorkItem The task to be assigned */ - private void announceTask(Worker theWorker, TaskWrapper taskWrapper) throws Exception + private void announceTask(Worker theWorker, TaskRunnerWorkItem taskRunnerWorkItem) throws Exception { - - final Task task = taskWrapper.getTask(); - final TaskContext taskContext = taskWrapper.getTaskContext(); + final Task task = taskRunnerWorkItem.getTask(); + final TaskContext taskContext = taskRunnerWorkItem.getTaskContext(); log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId()); @@ -390,17 +407,21 @@ public class RemoteTaskRunner implements TaskRunner jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)) ); + runningTasks.put(task.getId(), pendingTasks.remove(task.getId())); + log.info("Task %s switched from pending to running", task.getId()); + + // Syncing state with Zookeeper - don't assign new tasks until the task we just assigned is actually running + // on a worker - this avoids overflowing a worker with tasks synchronized (statusLock) { - // Syncing state with Zookeeper - wait for task to go into running queue - while (pendingTasks.containsKey(task.getId())) { + while (findWorkerRunningTask(task.getId()) == null) { statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis()); } } } /** - * When a new worker appears, listeners are registered for status changes. - * Status changes indicate the creation or completion of task. + * When a new worker appears, listeners are registered for status changes associated with tasks assigned to + * the worker. Status changes indicate the creation or completion of task. * The RemoteTaskRunner updates state according to these changes. * * @param worker - contains metadata for a worker that has appeared in ZK @@ -448,7 +469,7 @@ public class RemoteTaskRunner implements TaskRunner } catch (Exception e) { log.warn(e, "Worker[%s] wrote bogus status for task: %s", worker.getHost(), taskId); - retryTask(runningTasks.get(taskId), worker.getHost(), taskId); + retryTask(runningTasks.get(taskId), worker.getHost()); throw Throwables.propagate(e); } @@ -459,16 +480,13 @@ public class RemoteTaskRunner implements TaskRunner taskId ); - if (pendingTasks.containsKey(taskId)) { - runningTasks.put(taskId, pendingTasks.remove(taskId)); - } - + // Synchronizing state with ZK synchronized (statusLock) { statusLock.notify(); } - final TaskWrapper taskWrapper = runningTasks.get(taskId); - if (taskWrapper == null) { + final TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); + if (taskRunnerWorkItem == null) { log.warn( "WTF?! Worker[%s] announcing a status for a task I didn't know about: %s", worker.getHost(), @@ -477,10 +495,8 @@ public class RemoteTaskRunner implements TaskRunner } if (taskStatus.isComplete()) { - if (taskWrapper != null) { - final TaskCallback callback = taskWrapper.getCallback(); - - // Cleanup + if (taskRunnerWorkItem != null) { + final TaskCallback callback = taskRunnerWorkItem.getCallback(); if (callback != null) { callback.notify(taskStatus); } @@ -488,10 +504,15 @@ public class RemoteTaskRunner implements TaskRunner // Worker is done with this task workerWrapper.setLastCompletedTaskTime(new DateTime()); - cf.delete().guaranteed().inBackground().forPath(event.getData().getPath()); - runningTasks.remove(taskId); + cleanup(worker.getHost(), taskId); runPendingTasks(); } + } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { + final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); + if (runningTasks.containsKey(taskId)) { + log.info("Task %s just disappeared!", taskId); + retryTask(runningTasks.get(taskId), worker.getHost()); + } } } catch (Exception e) { @@ -508,20 +529,15 @@ public class RemoteTaskRunner implements TaskRunner runPendingTasks(); } - - catch ( - Exception e - ) - - { + catch (Exception e) { throw Throwables.propagate(e); } - } /** - * When a ephemeral worker node disappears from ZK, we have to make sure there are no tasks still assigned - * to the worker. If tasks remain, they are retried. + * When a ephemeral worker node disappears from ZK, incomplete running tasks will be retried by + * the logic in the status listener. We still have to make sure there are no tasks still assigned + * to the worker. * * @param worker - the removed worker */ @@ -530,23 +546,31 @@ public class RemoteTaskRunner implements TaskRunner WorkerWrapper workerWrapper = zkWorkers.get(worker.getHost()); if (workerWrapper != null) { try { - Set tasksToRetry = Sets.newHashSet(workerWrapper.getRunningTasks()); - tasksToRetry.addAll(cf.getChildren().forPath(JOINER.join(config.getTaskPath(), worker.getHost()))); + Set tasksPending = Sets.newHashSet( + cf.getChildren() + .forPath(JOINER.join(config.getTaskPath(), worker.getHost())) + ); + log.info("%s had %d tasks pending", worker.getHost(), tasksPending.size()); - for (String taskId : tasksToRetry) { - TaskWrapper taskWrapper = runningTasks.get(taskId); - if (taskWrapper != null) { - retryTask(runningTasks.get(taskId), worker.getHost(), taskId); + for (String taskId : tasksPending) { + TaskRunnerWorkItem taskRunnerWorkItem = pendingTasks.get(taskId); + if (taskRunnerWorkItem != null) { + cf.delete().guaranteed().forPath(JOINER.join(config.getTaskPath(), worker.getHost(), taskId)); + retryTask(taskRunnerWorkItem, worker.getHost()); + } else { + log.info("RemoteTaskRunner has no knowledge of pending task %s", taskId); } } workerWrapper.getStatusCache().close(); } catch (Exception e) { - log.error(e, "Failed to cleanly remove worker[%s]"); + throw Throwables.propagate(e); + } + finally { + zkWorkers.remove(worker.getHost()); } } - zkWorkers.remove(worker.getHost()); } private WorkerWrapper findWorkerForTask() @@ -588,14 +612,4 @@ public class RemoteTaskRunner implements TaskRunner throw Throwables.propagate(e); } } - - private WorkerWrapper findWorkerRunningTask(TaskWrapper taskWrapper) - { - for (WorkerWrapper workerWrapper : zkWorkers.values()) { - if (workerWrapper.getRunningTasks().contains(taskWrapper.getTask().getId())) { - return workerWrapper; - } - } - return null; - } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java index fbb27d76c18..632a1fcc985 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java @@ -44,6 +44,11 @@ public class RetryPolicy this.retryCount = 0; } + public Duration getRetryDelay() + { + return currRetryDelay; + } + public Duration getAndIncrementRetryDelay() { Duration retVal = new Duration(currRetryDelay); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java index 0481e8f21b6..d279d310696 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java @@ -27,6 +27,8 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.exec.TaskConsumer; +import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler; +import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.netflix.curator.framework.CuratorFramework; @@ -56,6 +58,7 @@ public class TaskMaster final ServiceDiscoveryConfig serviceDiscoveryConfig, final MergerDBCoordinator mergerDBCoordinator, final TaskRunnerFactory runnerFactory, + final ResourceManagementSchedulerFactory managementSchedulerFactory, final CuratorFramework curator, final ServiceEmitter emitter ) @@ -72,11 +75,13 @@ public class TaskMaster log.info("By the power of Grayskull, I have the power!"); final TaskRunner runner = runnerFactory.build(); + final ResourceManagementScheduler scheduler = managementSchedulerFactory.build(runner); final TaskConsumer consumer = new TaskConsumer(queue, runner, mergerDBCoordinator, emitter); // Sensible order to start stuff: final Lifecycle leaderLifecycle = new Lifecycle(); leaderLifecycle.addManagedInstance(queue); + leaderLifecycle.addManagedInstance(scheduler); leaderLifecycle.addManagedInstance(runner); Initialization.makeServiceDiscoveryClient(curator, serviceDiscoveryConfig, leaderLifecycle); leaderLifecycle.addManagedInstance(consumer); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java index 4743b2bf4b4..5d57814cba1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java @@ -40,9 +40,9 @@ public interface TaskRunner */ public void run(Task task, TaskContext context, TaskCallback callback); - public Collection getRunningTasks(); + public Collection getRunningTasks(); - public Collection getPendingTasks(); + public Collection getPendingTasks(); public Collection getWorkers(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkItem.java similarity index 77% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkItem.java index 6acecf03f05..d710d76f25f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkItem.java @@ -19,26 +19,30 @@ package com.metamx.druid.merger.coordinator; -import com.google.common.primitives.Longs; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.task.Task; +import org.joda.time.DateTime; +import org.joda.time.DateTimeComparator; /** + * A holder for a task and different components associated with the task */ -public class TaskWrapper implements Comparable +public class TaskRunnerWorkItem implements Comparable { private final Task task; private final TaskContext taskContext; private final TaskCallback callback; private final RetryPolicy retryPolicy; - private final long createdTime; + private final DateTime createdTime; - public TaskWrapper( + private volatile DateTime queueInsertionTime; + + public TaskRunnerWorkItem( Task task, TaskContext taskContext, TaskCallback callback, RetryPolicy retryPolicy, - long createdTime + DateTime createdTime ) { this.task = task; @@ -68,15 +72,26 @@ public class TaskWrapper implements Comparable return retryPolicy; } - public long getCreatedTime() + public DateTime getCreatedTime() { return createdTime; } - @Override - public int compareTo(TaskWrapper taskWrapper) + public DateTime getQueueInsertionTime() { - return Longs.compare(createdTime, taskWrapper.getCreatedTime()); + return queueInsertionTime; + } + + public TaskRunnerWorkItem withQueueInsertionTime(DateTime time) + { + this.queueInsertionTime = time; + return this; + } + + @Override + public int compareTo(TaskRunnerWorkItem taskRunnerWorkItem) + { + return DateTimeComparator.getInstance().compare(createdTime, taskRunnerWorkItem.getCreatedTime()); } @Override @@ -89,7 +104,7 @@ public class TaskWrapper implements Comparable return false; } - TaskWrapper that = (TaskWrapper) o; + TaskRunnerWorkItem that = (TaskRunnerWorkItem) o; if (callback != null ? !callback.equals(that.callback) : that.callback != null) { return false; @@ -120,7 +135,7 @@ public class TaskWrapper implements Comparable @Override public String toString() { - return "TaskWrapper{" + + return "TaskRunnerWorkItem{" + "task=" + task + ", taskContext=" + taskContext + ", callback=" + callback + diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkQueue.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkQueue.java new file mode 100644 index 00000000000..f0dc161ad1a --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkQueue.java @@ -0,0 +1,35 @@ +/* + * 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.merger.coordinator; + +import org.joda.time.DateTime; + +import java.util.concurrent.ConcurrentSkipListMap; + +/** + */ +public class TaskRunnerWorkQueue extends ConcurrentSkipListMap +{ + @Override + public TaskRunnerWorkItem put(String s, TaskRunnerWorkItem taskRunnerWorkItem) + { + return super.put(s, taskRunnerWorkItem.withQueueInsertionTime(new DateTime())); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 4ba09762485..c9071fd9c95 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -48,9 +48,9 @@ import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.S3SegmentKiller; -import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.loading.S3SegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; @@ -74,11 +74,12 @@ 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.config.WorkerSetupManagerConfig; +import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.NoopAutoScalingStrategy; +import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler; import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerConfig; -import com.metamx.druid.merger.coordinator.scaling.ResourceManagmentScheduler; -import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy; +import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory; import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy; import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; @@ -149,6 +150,7 @@ public class IndexerCoordinatorNode extends RegisteringNode private IndexerZkConfig indexerZkConfig; private WorkerSetupManager workerSetupManager = null; private TaskRunnerFactory taskRunnerFactory = null; + private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null; private TaskMaster taskMaster = null; private Server server = null; @@ -211,6 +213,12 @@ public class IndexerCoordinatorNode extends RegisteringNode return this; } + public IndexerCoordinatorNode setResourceManagementSchedulerFactory(ResourceManagementSchedulerFactory resourceManagementSchedulerFactory) + { + this.resourceManagementSchedulerFactory = resourceManagementSchedulerFactory; + return this; + } + public void init() throws Exception { scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); @@ -230,6 +238,7 @@ public class IndexerCoordinatorNode extends RegisteringNode initializeIndexerZkConfig(); initializeWorkerSetupManager(); initializeTaskRunnerFactory(); + initializeResourceManagement(); initializeTaskMaster(); initializeServer(); @@ -305,6 +314,7 @@ public class IndexerCoordinatorNode extends RegisteringNode serviceDiscoveryConfig, mergerDBCoordinator, taskRunnerFactory, + resourceManagementSchedulerFactory, curatorFramework, emitter ); @@ -543,8 +553,6 @@ public class IndexerCoordinatorNode extends RegisteringNode workerSetupManager ); - initializeResourceManagement(remoteTaskRunner); - return remoteTaskRunner; } }; @@ -565,46 +573,54 @@ public class IndexerCoordinatorNode extends RegisteringNode } } - private void initializeResourceManagement(TaskRunner taskRunner) + private void initializeResourceManagement() { - final ScheduledExecutorService scalingScheduledExec = Executors.newScheduledThreadPool( - 1, - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("ScalingExec--%d") - .build() - ); + if (resourceManagementSchedulerFactory == null) { + resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory() + { + @Override + public ResourceManagementScheduler build(TaskRunner runner) + { + final ScheduledExecutorService scalingScheduledExec = Executors.newScheduledThreadPool( + 1, + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("ScalingExec--%d") + .build() + ); - AutoScalingStrategy strategy; - if (config.getStrategyImpl().equalsIgnoreCase("ec2")) { - strategy = new EC2AutoScalingStrategy( - jsonMapper, - new AmazonEC2Client( - new BasicAWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) - ), - configFactory.build(EC2AutoScalingStrategyConfig.class), - workerSetupManager - ); - } else if (config.getStrategyImpl().equalsIgnoreCase("noop")) { - strategy = new NoopAutoScalingStrategy(); - } else { - throw new ISE("Invalid strategy implementation: %s", config.getStrategyImpl()); + AutoScalingStrategy strategy; + if (config.getStrategyImpl().equalsIgnoreCase("ec2")) { + strategy = new EC2AutoScalingStrategy( + jsonMapper, + new AmazonEC2Client( + new BasicAWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ), + configFactory.build(EC2AutoScalingStrategyConfig.class), + workerSetupManager + ); + } else if (config.getStrategyImpl().equalsIgnoreCase("noop")) { + strategy = new NoopAutoScalingStrategy(); + } else { + throw new ISE("Invalid strategy implementation: %s", config.getStrategyImpl()); + } + + return new ResourceManagementScheduler( + runner, + new SimpleResourceManagementStrategy( + strategy, + configFactory.build(SimpleResourceManagmentConfig.class), + workerSetupManager + ), + configFactory.build(ResourceManagementSchedulerConfig.class), + scalingScheduledExec + ); + } + }; } - - ResourceManagmentScheduler resourceManagmentScheduler = new ResourceManagmentScheduler( - taskRunner, - new SimpleResourceManagementStrategy( - strategy, - configFactory.build(SimpleResourceManagmentConfig.class), - workerSetupManager - ), - configFactory.build(ResourceManagementSchedulerConfig.class), - scalingScheduledExec - ); - lifecycle.addManagedInstance(resourceManagmentScheduler); } public static class Builder diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index d64899a2739..23e8e0da786 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -72,7 +72,6 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy WorkerSetupData setupData = workerSetupManager.getWorkerSetupData(); EC2NodeData workerConfig = setupData.getNodeData(); - log.info("Creating new instance(s)..."); RunInstancesResult result = amazonEC2Client.runInstances( new RunInstancesRequest( workerConfig.getAmiId(), diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagmentScheduler.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java similarity index 90% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagmentScheduler.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java index 9595b026043..45c0060fda6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagmentScheduler.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java @@ -32,15 +32,15 @@ import org.joda.time.Period; import java.util.concurrent.ScheduledExecutorService; /** - * The ResourceManagmentScheduler schedules a check for when worker nodes should potentially be created or destroyed. + * The ResourceManagementScheduler schedules a check for when worker nodes should potentially be created or destroyed. * It uses a {@link TaskRunner} to return all pending tasks in the system and the status of the worker nodes in * the system. - * The ResourceManagmentScheduler does not contain the logic to decide whether provision or termination should actually + * The ResourceManagementScheduler does not contain the logic to decide whether provision or termination should actually * occur. That decision is made in the {@link ResourceManagementStrategy}. */ -public class ResourceManagmentScheduler +public class ResourceManagementScheduler { - private static final Logger log = new Logger(ResourceManagmentScheduler.class); + private static final Logger log = new Logger(ResourceManagementScheduler.class); private final TaskRunner taskRunner; private final ResourceManagementStrategy resourceManagementStrategy; @@ -50,7 +50,7 @@ public class ResourceManagmentScheduler private final Object lock = new Object(); private volatile boolean started = false; - public ResourceManagmentScheduler( + public ResourceManagementScheduler( TaskRunner taskRunner, ResourceManagementStrategy resourceManagementStrategy, ResourceManagementSchedulerConfig config, @@ -124,6 +124,7 @@ public class ResourceManagmentScheduler return; } exec.shutdown(); + started = false; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java index c732b5c6c1a..924fe897490 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java @@ -29,7 +29,7 @@ import org.skife.config.Default; public abstract class ResourceManagementSchedulerConfig { @Config("druid.indexer.provisionResources.duration") - @Default("PT1H") + @Default("PT1M") public abstract Duration getProvisionResourcesDuration(); @Config("druid.indexer.terminateResources.duration") diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerFactory.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerFactory.java new file mode 100644 index 00000000000..8d33c8be220 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerFactory.java @@ -0,0 +1,29 @@ +/* + * 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.merger.coordinator.scaling; + +import com.metamx.druid.merger.coordinator.TaskRunner; + +/** + */ +public interface ResourceManagementSchedulerFactory +{ + public ResourceManagementScheduler build(TaskRunner runner); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java index 92599ae00a2..ae48084a1b9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java @@ -19,7 +19,7 @@ package com.metamx.druid.merger.coordinator.scaling; -import com.metamx.druid.merger.coordinator.TaskWrapper; +import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; import com.metamx.druid.merger.coordinator.WorkerWrapper; import java.util.Collection; @@ -30,9 +30,9 @@ import java.util.Collection; */ public interface ResourceManagementStrategy { - public boolean doProvision(Collection runningTasks, Collection workerWrappers); + public boolean doProvision(Collection runningTasks, Collection workerWrappers); - public boolean doTerminate(Collection runningTasks, Collection workerWrappers); + public boolean doTerminate(Collection runningTasks, Collection workerWrappers); public ScalingStats getStats(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java index cf464253d54..ad739da5f49 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -24,7 +24,7 @@ import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.druid.merger.coordinator.TaskWrapper; +import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; import com.metamx.druid.merger.coordinator.WorkerWrapper; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.EmittingLogger; @@ -65,7 +65,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat } @Override - public boolean doProvision(Collection pendingTasks, Collection workerWrappers) + public boolean doProvision(Collection pendingTasks, Collection workerWrappers) { List workerNodeIds = autoScalingStrategy.ipToIdLookup( Lists.newArrayList( @@ -89,15 +89,17 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat boolean nothingProvisioning = currentlyProvisioning.isEmpty(); - if (nothingProvisioning && hasTaskPendingBeyondThreshold(pendingTasks)) { - AutoScalingData provisioned = autoScalingStrategy.provision(); + if (nothingProvisioning) { + if (hasTaskPendingBeyondThreshold(pendingTasks)) { + AutoScalingData provisioned = autoScalingStrategy.provision(); - if (provisioned != null) { - currentlyProvisioning.addAll(provisioned.getNodeIds()); - lastProvisionTime = new DateTime(); - scalingStats.addProvisionEvent(provisioned); + if (provisioned != null) { + currentlyProvisioning.addAll(provisioned.getNodeIds()); + lastProvisionTime = new DateTime(); + scalingStats.addProvisionEvent(provisioned); - return true; + return true; + } } } else { Duration durSinceLastProvision = new Duration(new DateTime(), lastProvisionTime); @@ -118,7 +120,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat } @Override - public boolean doTerminate(Collection pendingTasks, Collection workerWrappers) + public boolean doTerminate(Collection pendingTasks, Collection workerWrappers) { List workerNodeIds = autoScalingStrategy.ipToIdLookup( Lists.newArrayList( @@ -214,12 +216,15 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat return scalingStats; } - private boolean hasTaskPendingBeyondThreshold(Collection pendingTasks) + private boolean hasTaskPendingBeyondThreshold(Collection pendingTasks) { long now = System.currentTimeMillis(); - for (TaskWrapper pendingTask : pendingTasks) { - if (new Duration(pendingTask.getCreatedTime(), now).isEqual(config.getMaxPendingTaskDuration()) || - new Duration(pendingTask.getCreatedTime(), now).isLongerThan(config.getMaxPendingTaskDuration())) { + for (TaskRunnerWorkItem pendingTask : pendingTasks) { + if (new Duration(pendingTask.getQueueInsertionTime().getMillis(), now).isEqual(config.getMaxPendingTaskDuration()) + || + new Duration( + pendingTask.getQueueInsertionTime().getMillis(), now + ).isLongerThan(config.getMaxPendingTaskDuration())) { return true; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java index 79f6b7f5db9..19fc2fdcedb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java @@ -32,14 +32,14 @@ public abstract class SimpleResourceManagmentConfig public abstract int getMaxWorkerIdleTimeMillisBeforeDeletion(); @Config("druid.indexer.maxScalingDuration") - @Default("PT1H") + @Default("PT15M") public abstract Duration getMaxScalingDuration(); @Config("druid.indexer.numEventsToTrack") - @Default("20") + @Default("50") public abstract int getNumEventsToTrack(); @Config("druid.indexer.maxPendingTaskDuration") - @Default("20") + @Default("PT30S") public abstract Duration getMaxPendingTaskDuration(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index 506b33e1416..94c948094a9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -93,6 +93,8 @@ public class TaskMonitor final TaskContext taskContext = taskHolder.getTaskContext(); if (workerCuratorCoordinator.statusExists(task.getId())) { + log.warn("Got task %s that I am already running...", task.getId()); + workerCuratorCoordinator.unannounceTask(task.getId()); return; } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java index 5b5f3a0a6e7..f668533a86b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java @@ -39,6 +39,7 @@ public abstract class WorkerConfig @Config("druid.worker.version") public abstract String getVersion(); + @Config("druid.worker.capacity") public int getCapacity() { return Runtime.getRuntime().availableProcessors() - 1; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index 9d6b6ca4e76..b27696d0046 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -164,11 +164,6 @@ public class RemoteTaskRunnerTest } catch (ISE expected) { } - finally { - cf.delete().guaranteed().forPath( - String.format("%s/worker1/task1", statusPath) - ); - } } @Test @@ -231,7 +226,7 @@ public class RemoteTaskRunnerTest // Really don't like this way of waiting for the task to appear int count = 0; - while (!remoteTaskRunner.isTaskRunning(task1.getId())) { + while (remoteTaskRunner.findWorkerRunningTask(task1.getId()) == null) { Thread.sleep(500); if (count > 10) { throw new ISE("WTF?! Task still not announced in ZK?"); @@ -249,7 +244,7 @@ public class RemoteTaskRunnerTest // Really don't like this way of waiting for the task to disappear count = 0; - while (remoteTaskRunner.isTaskRunning(task1.getId())) { + while (remoteTaskRunner.findWorkerRunningTask(task1.getId()) != null) { Thread.sleep(500); if (count > 10) { throw new ISE("WTF?! Task still not announced in ZK?"); diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java index 5755a2b991e..f133c70b64d 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -26,7 +26,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.TestTask; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskWrapper; +import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; import com.metamx.druid.merger.coordinator.WorkerWrapper; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; @@ -120,8 +120,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestWorkerWrapper(testTask) @@ -148,8 +148,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestWorkerWrapper(testTask) @@ -164,8 +164,8 @@ public class SimpleResourceManagementStrategyTest ); provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestWorkerWrapper(testTask) @@ -198,8 +198,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( - Arrays.asList( - new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestWorkerWrapper(null) @@ -230,8 +230,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( - Arrays.asList( - new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestWorkerWrapper(null) @@ -245,8 +245,8 @@ public class SimpleResourceManagementStrategyTest ); terminatedSomething = simpleResourceManagementStrategy.doTerminate( - Arrays.asList( - new TaskWrapper(testTask, null, null, null, System.currentTimeMillis()) + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestWorkerWrapper(null) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index ef91bcdcc7a..c106ee71127 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -142,44 +142,47 @@ public class DruidMaster public Map getLoadStatus() { - Map availableSegmentMap = Maps.newHashMap(); - - for (DataSegment segment : getAvailableDataSegments()) { - Integer count = availableSegmentMap.get(segment.getDataSource()); - int newCount = (count == null) ? 0 : count.intValue(); - availableSegmentMap.put(segment.getDataSource(), ++newCount); + // find available segments + Map> availableSegments = Maps.newHashMap(); + for (DataSegment dataSegment : getAvailableDataSegments()) { + Set segments = availableSegments.get(dataSegment.getDataSource()); + if (segments == null) { + segments = Sets.newHashSet(); + availableSegments.put(dataSegment.getDataSource(), segments); + } + segments.add(dataSegment); } - Map> loadedDataSources = Maps.newHashMap(); - for (DruidServer server : serverInventoryManager.getInventory()) { - for (DruidDataSource dataSource : server.getDataSources()) { - if (!loadedDataSources.containsKey(dataSource.getName())) { - TreeSet setToAdd = Sets.newTreeSet(DataSegment.bucketMonthComparator()); - setToAdd.addAll(dataSource.getSegments()); - loadedDataSources.put(dataSource.getName(), setToAdd); - } else { - loadedDataSources.get(dataSource.getName()).addAll(dataSource.getSegments()); + // find segments currently loaded + Map> segmentsInCluster = Maps.newHashMap(); + for (DruidServer druidServer : serverInventoryManager.getInventory()) { + for (DruidDataSource druidDataSource : druidServer.getDataSources()) { + Set segments = segmentsInCluster.get(druidDataSource.getName()); + if (segments == null) { + segments = Sets.newHashSet(); + segmentsInCluster.put(druidDataSource.getName(), segments); } + segments.addAll(druidDataSource.getSegments()); } } - Map loadedSegmentMap = Maps.newHashMap(); - for (Map.Entry> entry : loadedDataSources.entrySet()) { - loadedSegmentMap.put(entry.getKey(), entry.getValue().size()); - } - - Map retVal = Maps.newHashMap(); - - for (Map.Entry entry : availableSegmentMap.entrySet()) { - String key = entry.getKey(); - if (!loadedSegmentMap.containsKey(key) || entry.getValue().doubleValue() == 0.0) { - retVal.put(key, 0.0); - } else { - retVal.put(key, 100 * loadedSegmentMap.get(key).doubleValue() / entry.getValue().doubleValue()); + // compare available segments with currently loaded + Map loadStatus = Maps.newHashMap(); + for (Map.Entry> entry : availableSegments.entrySet()) { + String dataSource = entry.getKey(); + Set segmentsAvailable = entry.getValue(); + Set loadedSegments = segmentsInCluster.get(dataSource); + if (loadedSegments == null) { + loadedSegments = Sets.newHashSet(); } + Set unloadedSegments = Sets.difference(segmentsAvailable, loadedSegments); + loadStatus.put( + dataSource, + 100 * ((double) (segmentsAvailable.size() - unloadedSegments.size()) / (double) segmentsAvailable.size()) + ); } - return retVal; + return loadStatus; } public int lookupSegmentLifetime(DataSegment segment) From 1963bfe83f37046a19b0c103a7843b0501562f1a Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 13 Feb 2013 16:04:43 -0800 Subject: [PATCH 36/63] cleanup --- .../merger/coordinator/LocalTaskRunner.java | 2 +- .../merger/coordinator/RemoteTaskRunner.java | 96 +++++++------------ .../druid/merger/coordinator/TaskRunner.java | 2 +- .../{WorkerWrapper.java => ZkWorker.java} | 7 +- .../scaling/ResourceManagementStrategy.java | 6 +- .../SimpleResourceManagementStrategy.java | 32 +++---- .../coordinator/RemoteTaskRunnerTest.java | 2 +- .../SimpleResourceManagementStrategyTest.java | 30 +++--- 8 files changed, 76 insertions(+), 101 deletions(-) rename merger/src/main/java/com/metamx/druid/merger/coordinator/{WorkerWrapper.java => ZkWorker.java} (92%) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index 49f9dcbb237..54f0c271005 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -124,7 +124,7 @@ public class LocalTaskRunner implements TaskRunner } @Override - public Collection getWorkers() + public Collection getWorkers() { return Lists.newArrayList(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index d532b9b492b..f4ee1e2a557 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -89,7 +89,7 @@ public class RemoteTaskRunner implements TaskRunner private final WorkerSetupManager workerSetupManager; // all workers that exist in ZK - private final Map zkWorkers = new ConcurrentHashMap(); + private final Map zkWorkers = new ConcurrentHashMap(); // all tasks that have been assigned to a worker private final TaskRunnerWorkQueue runningTasks = new TaskRunnerWorkQueue(); // tasks that have not yet run @@ -170,8 +170,8 @@ public class RemoteTaskRunner implements TaskRunner return; } - for (WorkerWrapper workerWrapper : zkWorkers.values()) { - workerWrapper.close(); + for (ZkWorker zkWorker : zkWorkers.values()) { + zkWorker.close(); } } catch (Exception e) { @@ -183,7 +183,7 @@ public class RemoteTaskRunner implements TaskRunner } @Override - public Collection getWorkers() + public Collection getWorkers() { return zkWorkers.values(); } @@ -200,11 +200,11 @@ public class RemoteTaskRunner implements TaskRunner return pendingTasks.values(); } - public WorkerWrapper findWorkerRunningTask(String taskId) + public ZkWorker findWorkerRunningTask(String taskId) { - for (WorkerWrapper workerWrapper : zkWorkers.values()) { - if (workerWrapper.getRunningTasks().contains(taskId)) { - return workerWrapper; + for (ZkWorker zkWorker : zkWorkers.values()) { + if (zkWorker.getRunningTasks().contains(taskId)) { + return zkWorker; } } return null; @@ -321,7 +321,7 @@ public class RemoteTaskRunner implements TaskRunner /** * Ensures no workers are already running a task before assigning the task to a worker. - * It is possible that a worker is running a task that the RTR has no knowledge of. This is possible when the RTR + * It is possible that a worker is running a task that the RTR has no knowledge of. This occurs when the RTR * needs to bootstrap after a restart. * * @param taskRunnerWorkItem - the task to assign @@ -329,45 +329,20 @@ public class RemoteTaskRunner implements TaskRunner private void assignTask(TaskRunnerWorkItem taskRunnerWorkItem) { try { - WorkerWrapper workerWrapper = findWorkerRunningTask(taskRunnerWorkItem.getTask().getId()); + final String taskId = taskRunnerWorkItem.getTask().getId(); + ZkWorker zkWorker = findWorkerRunningTask(taskId); // If a worker is already running this task, we don't need to announce it - if (workerWrapper != null) { - final Worker worker = workerWrapper.getWorker(); - - log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskRunnerWorkItem.getTask().getId()); - runningTasks.put( - taskRunnerWorkItem.getTask().getId(), - pendingTasks.remove(taskRunnerWorkItem.getTask().getId()) - ); - log.info("Task %s switched from pending to running", taskRunnerWorkItem.getTask().getId()); - - final ChildData statusData = workerWrapper.getStatusCache() - .getCurrentData( - JOINER.join( - config.getStatusPath(), - worker.getHost(), - taskRunnerWorkItem.getTask().getId() - ) - ); - - final TaskStatus taskStatus = jsonMapper.readValue( - statusData.getData(), - TaskStatus.class - ); - - if (taskStatus.isComplete()) { - TaskCallback callback = taskRunnerWorkItem.getCallback(); - if (callback != null) { - callback.notify(taskStatus); - } - cleanup(worker.getHost(), taskRunnerWorkItem.getTask().getId()); - } + if (zkWorker != null) { + final Worker worker = zkWorker.getWorker(); + log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskId); + runningTasks.put(taskId, pendingTasks.remove(taskId)); + log.info("Task %s switched from pending to running", taskId); } else { // Nothing running this task, announce it in ZK for a worker to run it - workerWrapper = findWorkerForTask(); - if (workerWrapper != null) { - announceTask(workerWrapper.getWorker(), taskRunnerWorkItem); + zkWorker = findWorkerForTask(); + if (zkWorker != null) { + announceTask(zkWorker.getWorker(), taskRunnerWorkItem); } } } @@ -391,7 +366,6 @@ public class RemoteTaskRunner implements TaskRunner log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId()); byte[] rawBytes = jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)); - if (rawBytes.length > config.getMaxNumBytes()) { throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes()); } @@ -421,7 +395,7 @@ public class RemoteTaskRunner implements TaskRunner /** * When a new worker appears, listeners are registered for status changes associated with tasks assigned to - * the worker. Status changes indicate the creation or completion of task. + * the worker. Status changes indicate the creation or completion of a task. * The RemoteTaskRunner updates state according to these changes. * * @param worker - contains metadata for a worker that has appeared in ZK @@ -431,7 +405,7 @@ public class RemoteTaskRunner implements TaskRunner try { final String workerStatusPath = JOINER.join(config.getStatusPath(), worker.getHost()); final PathChildrenCache statusCache = new PathChildrenCache(cf, workerStatusPath, true); - final WorkerWrapper workerWrapper = new WorkerWrapper( + final ZkWorker zkWorker = new ZkWorker( worker, statusCache, jsonMapper @@ -503,7 +477,7 @@ public class RemoteTaskRunner implements TaskRunner } // Worker is done with this task - workerWrapper.setLastCompletedTaskTime(new DateTime()); + zkWorker.setLastCompletedTaskTime(new DateTime()); cleanup(worker.getHost(), taskId); runPendingTasks(); } @@ -524,7 +498,7 @@ public class RemoteTaskRunner implements TaskRunner } } ); - zkWorkers.put(worker.getHost(), workerWrapper); + zkWorkers.put(worker.getHost(), zkWorker); statusCache.start(); runPendingTasks(); @@ -536,15 +510,15 @@ public class RemoteTaskRunner implements TaskRunner /** * When a ephemeral worker node disappears from ZK, incomplete running tasks will be retried by - * the logic in the status listener. We still have to make sure there are no tasks still assigned - * to the worker. + * the logic in the status listener. We still have to make sure there are no tasks assigned + * to the worker but not yet running. * * @param worker - the removed worker */ private void removeWorker(final Worker worker) { - WorkerWrapper workerWrapper = zkWorkers.get(worker.getHost()); - if (workerWrapper != null) { + ZkWorker zkWorker = zkWorkers.get(worker.getHost()); + if (zkWorker != null) { try { Set tasksPending = Sets.newHashSet( cf.getChildren() @@ -558,11 +532,11 @@ public class RemoteTaskRunner implements TaskRunner cf.delete().guaranteed().forPath(JOINER.join(config.getTaskPath(), worker.getHost(), taskId)); retryTask(taskRunnerWorkItem, worker.getHost()); } else { - log.info("RemoteTaskRunner has no knowledge of pending task %s", taskId); + log.warn("RemoteTaskRunner has no knowledge of pending task %s", taskId); } } - workerWrapper.getStatusCache().close(); + zkWorker.getStatusCache().close(); } catch (Exception e) { throw Throwables.propagate(e); @@ -573,24 +547,24 @@ public class RemoteTaskRunner implements TaskRunner } } - private WorkerWrapper findWorkerForTask() + private ZkWorker findWorkerForTask() { try { - final MinMaxPriorityQueue workerQueue = MinMaxPriorityQueue.orderedBy( - new Comparator() + final MinMaxPriorityQueue workerQueue = MinMaxPriorityQueue.orderedBy( + new Comparator() { @Override - public int compare(WorkerWrapper w1, WorkerWrapper w2) + public int compare(ZkWorker w1, ZkWorker w2) { return -Ints.compare(w1.getRunningTasks().size(), w2.getRunningTasks().size()); } } ).create( FunctionalIterable.create(zkWorkers.values()).filter( - new Predicate() + new Predicate() { @Override - public boolean apply(WorkerWrapper input) + public boolean apply(ZkWorker input) { return (!input.isAtCapacity() && input.getWorker() diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java index 5d57814cba1..ea1b58fd98e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java @@ -44,5 +44,5 @@ public interface TaskRunner public Collection getPendingTasks(); - public Collection getWorkers(); + public Collection getWorkers(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java similarity index 92% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java index 2e164e9de75..c700fa3f047 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java @@ -36,8 +36,9 @@ import java.io.IOException; import java.util.Set; /** + * Holds information about a worker and a listener for task status changes associated with the worker. */ -public class WorkerWrapper implements Closeable +public class ZkWorker implements Closeable { private final Worker worker; private final PathChildrenCache statusCache; @@ -45,7 +46,7 @@ public class WorkerWrapper implements Closeable private volatile DateTime lastCompletedTaskTime = new DateTime(); - public WorkerWrapper(Worker worker, PathChildrenCache statusCache, final ObjectMapper jsonMapper) + public ZkWorker(Worker worker, PathChildrenCache statusCache, final ObjectMapper jsonMapper) { this.worker = worker; this.statusCache = statusCache; @@ -108,7 +109,7 @@ public class WorkerWrapper implements Closeable @Override public String toString() { - return "WorkerWrapper{" + + return "ZkWorker{" + "worker=" + worker + ", statusCache=" + statusCache + ", cacheConverter=" + cacheConverter + diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java index ae48084a1b9..7e2d666c3bd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java @@ -20,7 +20,7 @@ package com.metamx.druid.merger.coordinator.scaling; import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; -import com.metamx.druid.merger.coordinator.WorkerWrapper; +import com.metamx.druid.merger.coordinator.ZkWorker; import java.util.Collection; @@ -30,9 +30,9 @@ import java.util.Collection; */ public interface ResourceManagementStrategy { - public boolean doProvision(Collection runningTasks, Collection workerWrappers); + public boolean doProvision(Collection runningTasks, Collection zkWorkers); - public boolean doTerminate(Collection runningTasks, Collection workerWrappers); + public boolean doTerminate(Collection runningTasks, Collection zkWorkers); public ScalingStats getStats(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java index ad739da5f49..1cd3f8e46e6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -25,7 +25,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; -import com.metamx.druid.merger.coordinator.WorkerWrapper; +import com.metamx.druid.merger.coordinator.ZkWorker; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; @@ -65,16 +65,16 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat } @Override - public boolean doProvision(Collection pendingTasks, Collection workerWrappers) + public boolean doProvision(Collection pendingTasks, Collection zkWorkers) { List workerNodeIds = autoScalingStrategy.ipToIdLookup( Lists.newArrayList( Iterables.transform( - workerWrappers, - new Function() + zkWorkers, + new Function() { @Override - public String apply(WorkerWrapper input) + public String apply(ZkWorker input) { return input.getWorker().getIp(); } @@ -120,16 +120,16 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat } @Override - public boolean doTerminate(Collection pendingTasks, Collection workerWrappers) + public boolean doTerminate(Collection pendingTasks, Collection zkWorkers) { List workerNodeIds = autoScalingStrategy.ipToIdLookup( Lists.newArrayList( Iterables.transform( - workerWrappers, - new Function() + zkWorkers, + new Function() { @Override - public String apply(WorkerWrapper input) + public String apply(ZkWorker input) { return input.getWorker().getIp(); } @@ -146,18 +146,18 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat if (nothingTerminating) { final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers(); - if (workerWrappers.size() <= minNumWorkers) { + if (zkWorkers.size() <= minNumWorkers) { return false; } - List thoseLazyWorkers = Lists.newArrayList( + List thoseLazyWorkers = Lists.newArrayList( FunctionalIterable - .create(workerWrappers) + .create(zkWorkers) .filter( - new Predicate() + new Predicate() { @Override - public boolean apply(WorkerWrapper input) + public boolean apply(ZkWorker input) { return input.getRunningTasks().isEmpty() && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() @@ -174,10 +174,10 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat AutoScalingData terminated = autoScalingStrategy.terminate( Lists.transform( thoseLazyWorkers.subList(minNumWorkers, thoseLazyWorkers.size() - 1), - new Function() + new Function() { @Override - public String apply(WorkerWrapper input) + public String apply(ZkWorker input) { return input.getWorker().getIp(); } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index b27696d0046..2f85f644288 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -247,7 +247,7 @@ public class RemoteTaskRunnerTest while (remoteTaskRunner.findWorkerRunningTask(task1.getId()) != null) { Thread.sleep(500); if (count > 10) { - throw new ISE("WTF?! Task still not announced in ZK?"); + throw new ISE("WTF?! Task still exists in ZK?"); } count++; } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java index f133c70b64d..31111cfe685 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -27,7 +27,7 @@ import com.metamx.druid.merger.TestTask; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; -import com.metamx.druid.merger.coordinator.WorkerWrapper; +import com.metamx.druid.merger.coordinator.ZkWorker; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.Worker; @@ -123,8 +123,8 @@ public class SimpleResourceManagementStrategyTest Arrays.asList( new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), - Arrays.asList( - new TestWorkerWrapper(testTask) + Arrays.asList( + new TestZkWorker(testTask) ) ); @@ -151,8 +151,8 @@ public class SimpleResourceManagementStrategyTest Arrays.asList( new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), - Arrays.asList( - new TestWorkerWrapper(testTask) + Arrays.asList( + new TestZkWorker(testTask) ) ); @@ -167,8 +167,8 @@ public class SimpleResourceManagementStrategyTest Arrays.asList( new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), - Arrays.asList( - new TestWorkerWrapper(testTask) + Arrays.asList( + new TestZkWorker(testTask) ) ); @@ -201,8 +201,8 @@ public class SimpleResourceManagementStrategyTest Arrays.asList( new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), - Arrays.asList( - new TestWorkerWrapper(null) + Arrays.asList( + new TestZkWorker(null) ) ); @@ -233,8 +233,8 @@ public class SimpleResourceManagementStrategyTest Arrays.asList( new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), - Arrays.asList( - new TestWorkerWrapper(null) + Arrays.asList( + new TestZkWorker(null) ) ); @@ -248,8 +248,8 @@ public class SimpleResourceManagementStrategyTest Arrays.asList( new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime()) ), - Arrays.asList( - new TestWorkerWrapper(null) + Arrays.asList( + new TestZkWorker(null) ) ); @@ -263,11 +263,11 @@ public class SimpleResourceManagementStrategyTest EasyMock.verify(autoScalingStrategy); } - private static class TestWorkerWrapper extends WorkerWrapper + private static class TestZkWorker extends ZkWorker { private final Task testTask; - private TestWorkerWrapper( + private TestZkWorker( Task testTask ) { From c5b78e75cfe80b594ad5eb09db94ec830fa1e72b Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Tue, 12 Feb 2013 20:58:12 -0600 Subject: [PATCH 37/63] 1) Move up to Jackson 2.1 2) Some bugs were fixed, but they ended up getting mixed in with the Jackson upgrade :( --- client/pom.xml | 19 +++--- .../main/java/com/metamx/druid/BaseNode.java | 3 +- .../main/java/com/metamx/druid/BaseQuery.java | 3 +- .../src/main/java/com/metamx/druid/Query.java | 6 +- .../java/com/metamx/druid/QueryableNode.java | 9 ++- .../com/metamx/druid/RegisteringNode.java | 3 +- .../metamx/druid/client/BrokerServerView.java | 3 +- .../druid/client/CachingClusteredClient.java | 6 +- .../com/metamx/druid/client/DataSegment.java | 12 ++-- .../druid/client/DirectDruidClient.java | 20 ++++--- .../metamx/druid/client/DruidDataSource.java | 3 +- .../com/metamx/druid/client/DruidServer.java | 6 +- .../com/metamx/druid/client/ZKPhoneBook.java | 3 +- .../com/metamx/druid/http/BrokerNode.java | 6 +- .../druid/http/ClientServletModule.java | 6 +- .../com/metamx/druid/http/QueryServlet.java | 6 +- .../druid/initialization/Initialization.java | 3 +- .../metamx/druid/merge/ClientAppendQuery.java | 6 +- .../druid/merge/ClientDefaultMergeQuery.java | 6 +- .../metamx/druid/merge/ClientDeleteQuery.java | 6 +- .../metamx/druid/merge/ClientMergeQuery.java | 8 ++- .../com/metamx/druid/query/CacheStrategy.java | 3 +- .../metamx/druid/query/QueryToolChest.java | 3 +- .../query/dimension/DefaultDimensionSpec.java | 6 +- .../druid/query/dimension/DimensionSpec.java | 6 +- .../dimension/ExtractionDimensionSpec.java | 6 +- .../query/dimension/LegacyDimensionSpec.java | 3 +- .../query/extraction/DimExtractionFn.java | 5 +- .../extraction/PartialDimExtractionFn.java | 7 ++- .../extraction/RegexDimExtractionFn.java | 7 ++- .../SearchQuerySpecDimExtractionFn.java | 6 +- .../query/extraction/TimeDimExtractionFn.java | 6 +- .../druid/query/filter/AndDimFilter.java | 6 +- .../metamx/druid/query/filter/DimFilter.java | 5 +- .../query/filter/ExtractionDimFilter.java | 6 +- .../druid/query/filter/NotDimFilter.java | 7 ++- .../druid/query/filter/OrDimFilter.java | 6 +- .../druid/query/filter/RegexDimFilter.java | 6 +- .../query/filter/SearchQueryDimFilter.java | 3 +- .../druid/query/filter/SelectorDimFilter.java | 7 ++- .../druid/query/group/GroupByQuery.java | 6 +- .../group/GroupByQueryQueryToolChest.java | 3 +- .../druid/query/metadata/ColumnAnalysis.java | 6 +- .../query/metadata/ColumnIncluderator.java | 5 +- .../metadata/ListColumnIncluderator.java | 6 +- .../druid/query/metadata/SegmentAnalysis.java | 6 +- .../query/metadata/SegmentMetadataQuery.java | 3 +- .../SegmentMetadataQueryQueryToolChest.java | 3 +- .../query/search/FragmentSearchQuerySpec.java | 6 +- .../InsensitiveContainsSearchQuerySpec.java | 7 ++- .../search/LexicographicSearchSortSpec.java | 4 +- .../metamx/druid/query/search/SearchHit.java | 7 ++- .../druid/query/search/SearchQuery.java | 6 +- .../search/SearchQueryQueryToolChest.java | 3 +- .../druid/query/search/SearchQuerySpec.java | 6 +- .../druid/query/search/SearchSortSpec.java | 7 ++- .../query/segment/LegacySegmentSpec.java | 3 +- .../segment/MultipleIntervalSegmentSpec.java | 6 +- .../segment/MultipleSpecificSegmentSpec.java | 6 +- .../druid/query/segment/QuerySegmentSpec.java | 8 ++- .../query/segment/SegmentDescriptor.java | 6 +- .../query/timeboundary/TimeBoundaryQuery.java | 6 +- .../TimeBoundaryQueryQueryToolChest.java | 3 +- .../query/timeseries/TimeseriesQuery.java | 11 ++-- .../TimeseriesQueryQueryToolChest.java | 3 +- .../result/BySegmentResultValueClass.java | 3 +- .../result/BySegmentSearchResultValue.java | 6 +- .../druid/result/MetricValueExtractor.java | 7 ++- .../java/com/metamx/druid/result/Result.java | 8 ++- .../druid/result/SearchResultValue.java | 6 +- .../druid/result/TimeBoundaryResultValue.java | 6 +- .../druid/result/TimeseriesResultValue.java | 4 +- .../metamx/druid/shard/LinearShardSpec.java | 3 +- .../com/metamx/druid/shard/NoneShardSpec.java | 2 +- .../com/metamx/druid/shard/ShardSpec.java | 8 ++- .../druid/shard/SingleDimensionShardSpec.java | 5 +- .../metamx/druid/client/DataSegmentTest.java | 6 +- .../druid/query/QuerySegmentSpecTest.java | 3 +- common/pom.xml | 15 +++-- .../com/metamx/druid/DurationGranularity.java | 6 +- .../com/metamx/druid/PeriodGranularity.java | 6 +- .../com/metamx/druid/QueryGranularity.java | 11 ++-- .../druid/aggregation/AggregatorFactory.java | 8 ++- .../aggregation/CountAggregatorFactory.java | 6 +- .../DoubleSumAggregatorFactory.java | 6 +- .../metamx/druid/aggregation/Histogram.java | 3 +- .../HistogramAggregatorFactory.java | 6 +- .../druid/aggregation/HistogramVisual.java | 9 ++- .../JavaScriptAggregatorFactory.java | 6 +- .../aggregation/LongSumAggregatorFactory.java | 6 +- .../aggregation/MaxAggregatorFactory.java | 6 +- .../aggregation/MinAggregatorFactory.java | 6 +- .../post/ArithmeticPostAggregator.java | 6 +- .../post/ConstantPostAggregator.java | 6 +- .../post/FieldAccessPostAggregator.java | 6 +- .../aggregation/post/PostAggregator.java | 7 ++- .../metamx/druid/db/DbConnectorConfig.java | 3 +- .../com/metamx/druid/input/MapBasedRow.java | 6 +- .../main/java/com/metamx/druid/input/Row.java | 7 ++- .../jackson/CommaListJoinDeserializer.java | 13 ++-- .../jackson/CommaListJoinSerializer.java | 14 +++-- .../druid/jackson/DefaultObjectMapper.java | 60 +++++++++++++------ .../com/metamx/druid/jackson/JodaStuff.java | 28 +++++---- .../metamx/druid/QueryGranularityTest.java | 3 +- .../metamx/druid/histogram/HistogramTest.java | 3 +- .../jackson/DefaultObjectMapperTest.java | 40 +++++++++++++ examples/rand/pom.xml | 19 +++--- .../druid/examples/RandomFirehoseFactory.java | 9 ++- .../examples/RealtimeStandaloneMain.java | 3 +- examples/twitter/pom.xml | 19 +++--- .../examples/RealtimeStandaloneMain.java | 3 +- .../TwitterSpritzerFirehoseFactory.java | 9 ++- index-common/pom.xml | 7 ++- .../index/column/ColumnCapabilitiesImpl.java | 3 +- .../druid/index/column/ColumnDescriptor.java | 6 +- .../druid/index/serde/ColumnPartSerde.java | 6 +- .../index/serde/ComplexColumnPartSerde.java | 6 +- .../DictionaryEncodedColumnPartSerde.java | 6 +- .../serde/FloatGenericColumnPartSerde.java | 6 +- .../serde/LongGenericColumnPartSerde.java | 6 +- .../com/metamx/druid/index/v1/IndexIO.java | 3 +- .../druid/index/v1/serde/Registererer.java | 3 +- .../druid/indexer/data/CSVDataSpec.java | 6 +- .../metamx/druid/indexer/data/DataSpec.java | 6 +- .../druid/indexer/data/DelimitedDataSpec.java | 6 +- .../druid/indexer/data/JSONDataSpec.java | 3 +- .../indexer/data/StringInputRowParser.java | 6 +- .../druid/indexer/data/TimestampSpec.java | 6 +- .../indexer/data/ToLowercaseDataSpec.java | 3 +- indexer/pom.xml | 11 ++-- .../metamx/druid/indexer/DbUpdaterJob.java | 3 +- .../druid/indexer/DeterminePartitionsJob.java | 3 +- .../druid/indexer/HadoopDruidIndexer.java | 2 +- .../indexer/HadoopDruidIndexerConfig.java | 59 +++++++++--------- .../druid/indexer/HadoopDruidIndexerNode.java | 4 +- .../druid/indexer/HadoopyShardSpec.java | 6 +- .../java/com/metamx/druid/indexer/Utils.java | 6 +- .../granularity/ArbitraryGranularitySpec.java | 6 +- .../indexer/granularity/GranularitySpec.java | 6 +- .../granularity/UniformGranularitySpec.java | 6 +- .../indexer/partitions/PartitionsSpec.java | 9 ++- .../path/GranularUnprocessedPathSpec.java | 3 +- .../indexer/path/GranularityPathSpec.java | 3 +- .../metamx/druid/indexer/path/PathSpec.java | 8 ++- .../druid/indexer/path/StaticPathSpec.java | 3 +- .../druid/indexer/rollup/DataRollupSpec.java | 3 +- .../indexer/updater/DbUpdaterJobSpec.java | 3 +- .../druid/indexer/updater/UpdaterJobSpec.java | 5 +- .../indexer/updater/ZkUpdaterJobSpec.java | 3 +- .../indexer/HadoopDruidIndexerConfigTest.java | 7 ++- .../granularity/ArbitraryGranularityTest.java | 3 +- .../granularity/UniformGranularityTest.java | 3 +- merger/pom.xml | 15 ++--- .../druid/merger/common/TaskHolder.java | 6 +- .../druid/merger/common/TaskStatus.java | 6 +- .../druid/merger/common/TaskToolbox.java | 3 +- .../common/index/StaticS3FirehoseFactory.java | 12 ++-- .../common/index/YeOldePlumberSchool.java | 12 ++-- .../merger/common/task/AbstractTask.java | 6 +- .../druid/merger/common/task/AppendTask.java | 6 +- .../merger/common/task/DefaultMergeTask.java | 6 +- .../druid/merger/common/task/DeleteTask.java | 8 ++- .../task/IndexDeterminePartitionsTask.java | 9 ++- .../common/task/IndexGeneratorTask.java | 6 +- .../druid/merger/common/task/IndexTask.java | 6 +- .../druid/merger/common/task/MergeTask.java | 9 ++- .../metamx/druid/merger/common/task/Task.java | 6 +- .../merger/common/task/V8toV9UpgradeTask.java | 3 +- .../merger/coordinator/DbTaskStorage.java | 3 +- .../coordinator/MergerDBCoordinator.java | 3 +- .../merger/coordinator/RemoteTaskRunner.java | 3 +- .../druid/merger/coordinator/TaskContext.java | 6 +- .../merger/coordinator/WorkerWrapper.java | 3 +- .../config/IndexerDbConnectorConfig.java | 3 +- .../http/IndexerCoordinatorNode.java | 6 +- .../http/IndexerCoordinatorServletModule.java | 6 +- .../scaling/EC2AutoScalingStrategy.java | 3 +- .../merger/coordinator/setup/EC2NodeData.java | 7 ++- .../coordinator/setup/GalaxyUserData.java | 5 +- .../coordinator/setup/WorkerSetupData.java | 7 ++- .../coordinator/setup/WorkerSetupManager.java | 3 +- .../druid/merger/worker/TaskMonitor.java | 2 +- .../metamx/druid/merger/worker/Worker.java | 6 +- .../worker/WorkerCuratorCoordinator.java | 3 +- .../druid/merger/worker/http/WorkerNode.java | 6 +- .../coordinator/RemoteTaskRunnerTest.java | 14 +++-- pom.xml | 43 ++++++++----- realtime/pom.xml | 15 ++--- .../metamx/druid/realtime/FireDepartment.java | 7 ++- .../druid/realtime/FireDepartmentConfig.java | 3 +- .../druid/realtime/FirehoseFactory.java | 7 ++- .../druid/realtime/KafkaFirehoseFactory.java | 6 +- .../druid/realtime/MetadataUpdater.java | 3 +- .../metamx/druid/realtime/PlumberSchool.java | 5 +- .../metamx/druid/realtime/RealtimeNode.java | 18 ++++-- .../druid/realtime/RealtimePlumberSchool.java | 15 +++-- .../druid/realtime/S3SegmentPusher.java | 3 +- .../com/metamx/druid/realtime/Schema.java | 6 +- server/pom.xml | 27 +++++---- .../src/main/java/com/metamx/TsvToJson.java | 3 +- .../java/com/metamx/druid/BaseServerNode.java | 3 +- .../DataSegmentChangeRequest.java | 8 ++- .../SegmentChangeRequestDrop.java | 9 ++- .../SegmentChangeRequestLoad.java | 9 ++- .../druid/coordination/ZkCoordinator.java | 3 +- .../metamx/druid/db/DatabaseRuleManager.java | 6 +- .../druid/db/DatabaseSegmentManager.java | 3 +- .../com/metamx/druid/http/ComputeNode.java | 6 +- .../com/metamx/druid/http/MasterMain.java | 3 +- .../druid/http/MasterServletModule.java | 6 +- .../com/metamx/druid/http/SegmentToDrop.java | 5 +- .../com/metamx/druid/http/SegmentToMove.java | 5 +- .../druid/index/brita/SearchQueryFilter.java | 6 +- .../druid/index/v1/IndexGranularity.java | 3 +- .../metamx/druid/loading/S3SegmentPusher.java | 3 +- .../com/metamx/druid/master/DruidMaster.java | 3 +- .../master/DruidMasterSegmentMerger.java | 3 +- .../metamx/druid/master/HttpMergerClient.java | 3 +- .../druid/master/rules/IntervalDropRule.java | 6 +- .../druid/master/rules/IntervalLoadRule.java | 6 +- .../druid/master/rules/PeriodDropRule.java | 6 +- .../druid/master/rules/PeriodLoadRule.java | 6 +- .../com/metamx/druid/master/rules/Rule.java | 6 +- .../SegmentChangeRequestDropTest.java | 6 +- .../SegmentChangeRequestLoadTest.java | 6 +- .../druid/coordination/ServerManagerTest.java | 3 +- .../druid/coordination/ZkCoordinatorTest.java | 3 +- 227 files changed, 1029 insertions(+), 542 deletions(-) create mode 100644 common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java diff --git a/client/pom.xml b/client/pom.xml index 36bb84de0d4..91674a214cc 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-client @@ -106,20 +107,20 @@ icu4j - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile javax.inject diff --git a/client/src/main/java/com/metamx/druid/BaseNode.java b/client/src/main/java/com/metamx/druid/BaseNode.java index 7c7ed5f0984..8902d14a65f 100644 --- a/client/src/main/java/com/metamx/druid/BaseNode.java +++ b/client/src/main/java/com/metamx/druid/BaseNode.java @@ -19,9 +19,10 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; -import org.codehaus.jackson.map.ObjectMapper; + import org.skife.config.ConfigurationObjectFactory; import java.util.Properties; diff --git a/client/src/main/java/com/metamx/druid/BaseQuery.java b/client/src/main/java/com/metamx/druid/BaseQuery.java index 76448ed9f17..9f3ae8d30ed 100644 --- a/client/src/main/java/com/metamx/druid/BaseQuery.java +++ b/client/src/main/java/com/metamx/druid/BaseQuery.java @@ -19,13 +19,14 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.metamx.common.guava.Sequence; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentWalker; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/Query.java b/client/src/main/java/com/metamx/druid/Query.java index 4c4e7f715b4..818400fc262 100644 --- a/client/src/main/java/com/metamx/druid/Query.java +++ b/client/src/main/java/com/metamx/druid/Query.java @@ -19,6 +19,8 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.guava.Sequence; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.group.GroupByQuery; @@ -28,8 +30,8 @@ import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; import com.metamx.druid.query.timeseries.TimeseriesQuery; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index 76e835d4ffe..749b2bd8b40 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -19,6 +19,9 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -47,9 +50,9 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import com.metamx.phonebook.PhoneBook; import org.I0Itec.zkclient.ZkClient; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.jsontype.NamedType; -import org.codehaus.jackson.smile.SmileFactory; + + + import org.mortbay.jetty.Server; import org.skife.config.ConfigurationObjectFactory; diff --git a/client/src/main/java/com/metamx/druid/RegisteringNode.java b/client/src/main/java/com/metamx/druid/RegisteringNode.java index 93d8fd7c8cd..1332bbd2a4b 100644 --- a/client/src/main/java/com/metamx/druid/RegisteringNode.java +++ b/client/src/main/java/com/metamx/druid/RegisteringNode.java @@ -19,8 +19,9 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.index.v1.serde.Registererer; -import org.codehaus.jackson.map.ObjectMapper; + import java.util.Arrays; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java b/client/src/main/java/com/metamx/druid/client/BrokerServerView.java index e37e0bfeb01..98900e9de41 100644 --- a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java +++ b/client/src/main/java/com/metamx/druid/client/BrokerServerView.java @@ -19,6 +19,7 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -28,7 +29,7 @@ import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.http.client.HttpClient; -import org.codehaus.jackson.map.ObjectMapper; + import javax.annotation.Nullable; import java.util.Iterator; diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index ad08ede04eb..6d896d8bf0f 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -19,6 +19,8 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.base.Throwables; @@ -53,8 +55,8 @@ import com.metamx.druid.query.segment.MultipleSpecificSegmentSpec; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.BySegmentResultValueClass; import com.metamx.druid.result.Result; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/client/DataSegment.java b/client/src/main/java/com/metamx/druid/client/DataSegment.java index 9b29f3507ee..b915f7680cd 100644 --- a/client/src/main/java/com/metamx/druid/client/DataSegment.java +++ b/client/src/main/java/com/metamx/druid/client/DataSegment.java @@ -19,6 +19,10 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; @@ -31,10 +35,10 @@ import com.metamx.druid.jackson.CommaListJoinSerializer; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.annotate.JsonDeserialize; -import org.codehaus.jackson.map.annotate.JsonSerialize; + + + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java b/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java index 62782a41892..7c3e23c1395 100644 --- a/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java +++ b/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java @@ -19,6 +19,13 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.core.ObjectCodec; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.type.TypeFactory; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.google.common.io.Closeables; @@ -40,13 +47,12 @@ import com.metamx.http.client.HttpClient; import com.metamx.http.client.io.AppendableByteArrayInputStream; import com.metamx.http.client.response.ClientResponse; import com.metamx.http.client.response.InputStreamResponseHandler; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonToken; -import org.codehaus.jackson.ObjectCodec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.type.TypeFactory; -import org.codehaus.jackson.smile.SmileFactory; -import org.codehaus.jackson.type.JavaType; + + + + + + import org.jboss.netty.handler.codec.http.HttpChunk; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpResponse; diff --git a/client/src/main/java/com/metamx/druid/client/DruidDataSource.java b/client/src/main/java/com/metamx/druid/client/DruidDataSource.java index 6f96e8e8672..e112bbf6234 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidDataSource.java +++ b/client/src/main/java/com/metamx/druid/client/DruidDataSource.java @@ -19,8 +19,9 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.Collections; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/client/DruidServer.java b/client/src/main/java/com/metamx/druid/client/DruidServer.java index 6ecfdb16a64..a71d8a15ac7 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServer.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServer.java @@ -19,10 +19,12 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.metamx.common.logger.Logger; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.HashMap; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java b/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java index af9d552d009..1b8fa9df876 100644 --- a/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java +++ b/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java @@ -19,6 +19,7 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.HashBasedTable; import com.google.common.collect.Maps; @@ -33,7 +34,7 @@ import org.I0Itec.zkclient.IZkChildListener; import org.I0Itec.zkclient.ZkClient; import org.I0Itec.zkclient.exception.ZkNoNodeException; import org.I0Itec.zkclient.exception.ZkNodeExistsException; -import org.codehaus.jackson.map.ObjectMapper; + import java.io.IOException; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/http/BrokerNode.java b/client/src/main/java/com/metamx/druid/http/BrokerNode.java index 2a94a00d76d..76e487efb56 100644 --- a/client/src/main/java/com/metamx/druid/http/BrokerNode.java +++ b/client/src/main/java/com/metamx/druid/http/BrokerNode.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +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.Guice; @@ -53,8 +55,8 @@ import com.metamx.http.client.HttpClientInit; import com.metamx.metrics.Monitor; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.x.discovery.ServiceDiscovery; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.smile.SmileFactory; + + import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; diff --git a/client/src/main/java/com/metamx/druid/http/ClientServletModule.java b/client/src/main/java/com/metamx/druid/http/ClientServletModule.java index 7ecc5d7ad81..c5b89f37004 100644 --- a/client/src/main/java/com/metamx/druid/http/ClientServletModule.java +++ b/client/src/main/java/com/metamx/druid/http/ClientServletModule.java @@ -19,13 +19,15 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; import com.metamx.druid.client.ClientInventoryManager; import com.metamx.druid.query.segment.QuerySegmentWalker; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import org.codehaus.jackson.jaxrs.JacksonJsonProvider; -import org.codehaus.jackson.map.ObjectMapper; + + import javax.inject.Singleton; diff --git a/client/src/main/java/com/metamx/druid/http/QueryServlet.java b/client/src/main/java/com/metamx/druid/http/QueryServlet.java index 108863bf7af..9a13c50f45c 100644 --- a/client/src/main/java/com/metamx/druid/http/QueryServlet.java +++ b/client/src/main/java/com/metamx/druid/http/QueryServlet.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; @@ -31,8 +33,8 @@ import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.ObjectWriter; + + import org.joda.time.DateTime; import org.mortbay.jetty.Request; diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index cfb2d9302a2..4865208e88e 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -19,6 +19,7 @@ package com.metamx.druid.initialization; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.io.Closeables; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -41,7 +42,7 @@ import com.netflix.curator.x.discovery.ServiceInstance; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.ZkClient; import org.I0Itec.zkclient.ZkConnection; -import org.codehaus.jackson.map.ObjectMapper; + import org.mortbay.jetty.Connector; import org.mortbay.jetty.Server; import org.mortbay.jetty.nio.SelectChannelConnector; diff --git a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java index 441adcffc3b..5a8e3bdb50f 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java @@ -19,9 +19,11 @@ package com.metamx.druid.merge; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java index c5cc8045492..4286cd211cb 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java @@ -19,10 +19,12 @@ package com.metamx.druid.merge; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java index a0733db0a2a..3acf20a0a43 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java @@ -19,8 +19,10 @@ package com.metamx.druid.merge; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; public class ClientDeleteQuery diff --git a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java index 83f269d0db3..3956991ed6b 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java @@ -19,15 +19,17 @@ package com.metamx.druid.merge; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.List; /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class) @JsonSubTypes(value={ @JsonSubTypes.Type(name="append", value=ClientAppendQuery.class) }) diff --git a/client/src/main/java/com/metamx/druid/query/CacheStrategy.java b/client/src/main/java/com/metamx/druid/query/CacheStrategy.java index f8f5098f6ca..b17e95b86e7 100644 --- a/client/src/main/java/com/metamx/druid/query/CacheStrategy.java +++ b/client/src/main/java/com/metamx/druid/query/CacheStrategy.java @@ -19,10 +19,11 @@ package com.metamx.druid.query; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; -import org.codehaus.jackson.type.TypeReference; + /** */ diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 701a4512403..f729016032e 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -19,13 +19,14 @@ package com.metamx.druid.query; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.client.selector.ServerSelector; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java index a936023550f..a3c36476253 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java index 34310ef4c09..bcf5fabf62b 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + /** */ diff --git a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java index fe80c24c94f..6feb011dcd7 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java index 9a4e11a9a5f..b753b04c46b 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonCreator; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java index 0c1ef8b159c..8fe869cb4af 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.extraction; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java index 28ea1ebc37b..f30c4578706 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.extraction; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; import java.util.regex.Matcher; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java index da3a878aa2f..c722b0c52b7 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.extraction; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; import java.util.regex.Matcher; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java index 57e3bfde16b..7dbc2f3c65d 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.extraction; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.search.SearchQuerySpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java index dc2c2eb36cf..c02797cf088 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.extraction; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.ibm.icu.text.SimpleDateFormat; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.text.ParseException; diff --git a/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java index 92696b29c26..6b557fdba27 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Joiner; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java index 9dc819cb8ec..8da47da465c 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.filter; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java index db3376b9de8..21785f06b0c 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java index 8b474d301f3..0753d9f64de 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.filter; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java index 2a056c8dfd6..9105cfe0412 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Joiner; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java index 519814d8262..6ee15d4b0c9 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java @@ -19,9 +19,11 @@ 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 org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java index b579acd87c5..7b1b6b0fd15 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java @@ -19,9 +19,10 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.metamx.druid.query.search.SearchQuerySpec; -import org.codehaus.jackson.annotate.JsonProperty; + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java index c778027d48c..11deaeb28e2 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.filter; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java index d30af309473..4c1ac740dca 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java @@ -19,6 +19,8 @@ package com.metamx.druid.query.group; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -33,8 +35,8 @@ import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.segment.LegacySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index 1c66774d3bc..51570e30955 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.group; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; @@ -43,7 +44,7 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java b/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java index 33b894ca07a..9eb8a46bcdb 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java @@ -19,10 +19,12 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.druid.index.column.ValueType; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** */ diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java index 90533c4eaca..c6cfb081835 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.metadata; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java index e74661d6822..a7de6b1920e 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java @@ -19,12 +19,14 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.ByteArrayOutputStream; import java.io.IOException; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java index 1182bfeb9fa..22657e45301 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java @@ -19,8 +19,10 @@ package com.metamx.druid.query.metadata; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java index 7e0d04c0739..707a0a285b2 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java @@ -19,10 +19,11 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.BaseQuery; import com.metamx.druid.Query; import com.metamx.druid.query.segment.QuerySegmentSpec; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 2b979117800..6ede9c7f247 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; @@ -39,7 +40,7 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.ResultMergeQueryRunner; import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java index e0a73b4bc3c..b813b7febbf 100644 --- a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java @@ -19,10 +19,12 @@ package com.metamx.druid.query.search; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Lists; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java index 87a6246a5b2..1d7a836a3ab 100644 --- a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java b/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java index b7dfddef413..83f81bfc349 100644 --- a/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java @@ -19,7 +19,9 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonCreator; + + +import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Comparator; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchHit.java b/client/src/main/java/com/metamx/druid/query/search/SearchHit.java index c1c0727067a..ba896912197 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchHit.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchHit.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import static com.google.common.base.Preconditions.checkNotNull; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java index 4ce3bd72e7f..124d9c5fa62 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java @@ -19,6 +19,8 @@ package com.metamx.druid.query.search; +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.collect.Lists; @@ -29,8 +31,8 @@ import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import javax.annotation.Nullable; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index cefc32b5e3d..f58f31ebad7 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.search; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; @@ -49,7 +50,7 @@ import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java index 3f634bd3530..dbd0e91c48d 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.search; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Predicate; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java b/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java index b68f074e2ce..f98cf41f999 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.util.Comparator; diff --git a/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java index 6d25c152ab4..f4e62ff77c2 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java @@ -19,10 +19,11 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; + import org.joda.time.Interval; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java index f81126857bc..5432d468ed4 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java @@ -19,11 +19,13 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.Query; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.utils.JodaUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import java.util.Collections; diff --git a/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java index 4193934fbc0..fb259c5e5d5 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java @@ -19,13 +19,15 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.metamx.druid.Query; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.utils.JodaUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java index ca42ce8bc8a..98fc8e7b327 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java @@ -19,17 +19,19 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.Query; import com.metamx.druid.query.QueryRunner; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.Interval; import java.util.List; /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type", defaultImpl = LegacySegmentSpec.class) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = LegacySegmentSpec.class) @JsonSubTypes(value={ @JsonSubTypes.Type(name="intervals", value=MultipleIntervalSegmentSpec.class), @JsonSubTypes.Type(name="segments", value=MultipleSpecificSegmentSpec.class) diff --git a/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java b/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java index d918309baa6..90d433c1e5d 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java +++ b/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java @@ -19,8 +19,10 @@ package com.metamx.druid.query.segment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; /** diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java index 071d7695c0f..07fd071d6db 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java @@ -19,6 +19,8 @@ package com.metamx.druid.query.timeboundary; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -28,8 +30,8 @@ import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 8e41f1f6eb8..86e618bcee8 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.timeboundary; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.collect.Lists; @@ -40,7 +41,7 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java index 490e2531edd..a67caf19924 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java @@ -19,6 +19,9 @@ package com.metamx.druid.query.timeseries; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.metamx.druid.BaseQuery; @@ -31,10 +34,10 @@ import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeName; + + + + import java.util.List; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 538897480b3..bef34d9a2da 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.timeseries; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.collect.Lists; @@ -45,7 +46,7 @@ import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java b/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java index fb1e0df8e22..dfaacb572cc 100644 --- a/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java +++ b/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java @@ -19,7 +19,8 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java b/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java index b324ccbab37..e5a069f17a1 100644 --- a/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java @@ -19,9 +19,11 @@ package com.metamx.druid.result; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.druid.query.search.SearchHit; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonValue; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java b/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java index 3baf45c6a67..c99ac0f3ed3 100644 --- a/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java +++ b/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java @@ -19,8 +19,11 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonValue; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import java.lang.Double;import java.lang.Float;import java.lang.Long;import java.lang.Number;import java.lang.Object;import java.lang.Override;import java.lang.String;import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/result/Result.java b/client/src/main/java/com/metamx/druid/result/Result.java index 9b055cd4800..f9c30fc8319 100644 --- a/client/src/main/java/com/metamx/druid/result/Result.java +++ b/client/src/main/java/com/metamx/druid/result/Result.java @@ -19,9 +19,11 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonUnwrapped; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.DateTime; /** diff --git a/client/src/main/java/com/metamx/druid/result/SearchResultValue.java b/client/src/main/java/com/metamx/druid/result/SearchResultValue.java index 40f10ed85ec..c9dd74fc4ec 100644 --- a/client/src/main/java/com/metamx/druid/result/SearchResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/SearchResultValue.java @@ -19,9 +19,11 @@ package com.metamx.druid.result; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.druid.query.search.SearchHit; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonValue; + + import java.util.Iterator; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java b/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java index 74daa14f376..1b1b4c45547 100644 --- a/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java @@ -19,10 +19,12 @@ package com.metamx.druid.result; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.common.IAE; import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonValue; + + import org.joda.time.DateTime; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java b/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java index d68a9855395..55892023def 100644 --- a/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java @@ -19,7 +19,9 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonCreator; + + +import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java b/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java index 97f22ae6660..ae83287f290 100644 --- a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java @@ -1,9 +1,10 @@ package com.metamx.druid.shard; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.LinearPartitionChunk; import com.metamx.druid.partition.PartitionChunk; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java b/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java index 4ed756e1daf..9e40f9f703e 100644 --- a/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java @@ -22,7 +22,7 @@ package com.metamx.druid.shard; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.SingleElementPartitionChunk; -import org.codehaus.jackson.annotate.JsonTypeName; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/shard/ShardSpec.java b/client/src/main/java/com/metamx/druid/shard/ShardSpec.java index df3eb29004a..2db8f15ed5f 100644 --- a/client/src/main/java/com/metamx/druid/shard/ShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/ShardSpec.java @@ -19,17 +19,19 @@ package com.metamx.druid.shard; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.PartitionChunk; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.Map; /** * A Marker interface that exists to combine ShardSpec objects together for Jackson */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type", include=JsonTypeInfo.As.PROPERTY) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", include=JsonTypeInfo.As.PROPERTY) @JsonSubTypes({ @JsonSubTypes.Type(name="single", value=SingleDimensionShardSpec.class), @JsonSubTypes.Type(name="none", value=NoneShardSpec.class), diff --git a/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java b/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java index 7580673b071..0843ec7c61d 100644 --- a/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java @@ -19,11 +19,12 @@ package com.metamx.druid.shard; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.StringPartitionChunk; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; + + import java.util.List; import java.util.Map; diff --git a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java b/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java index 24d5986a8f8..082f66e8aac 100644 --- a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java +++ b/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java @@ -19,6 +19,8 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -27,8 +29,8 @@ import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java b/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java index a05973af729..654ea2e6b79 100644 --- a/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java +++ b/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.metamx.druid.jackson.DefaultObjectMapper; @@ -27,7 +28,7 @@ import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; import com.metamx.druid.query.segment.MultipleSpecificSegmentSpec; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.SegmentDescriptor; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/common/pom.xml b/common/pom.xml index 350828b9316..f334b521775 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-common @@ -72,12 +73,16 @@ guava - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.datatype + jackson-datatype-joda org.jdbi diff --git a/common/src/main/java/com/metamx/druid/DurationGranularity.java b/common/src/main/java/com/metamx/druid/DurationGranularity.java index da485b62e97..b2d5e55be42 100644 --- a/common/src/main/java/com/metamx/druid/DurationGranularity.java +++ b/common/src/main/java/com/metamx/druid/DurationGranularity.java @@ -19,9 +19,11 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Longs; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import java.nio.ByteBuffer; diff --git a/common/src/main/java/com/metamx/druid/PeriodGranularity.java b/common/src/main/java/com/metamx/druid/PeriodGranularity.java index 6fb461a5bc9..3c6ca5fd23e 100644 --- a/common/src/main/java/com/metamx/druid/PeriodGranularity.java +++ b/common/src/main/java/com/metamx/druid/PeriodGranularity.java @@ -19,8 +19,10 @@ package com.metamx.druid; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Chronology; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; diff --git a/common/src/main/java/com/metamx/druid/QueryGranularity.java b/common/src/main/java/com/metamx/druid/QueryGranularity.java index 36a4dd01714..5b94c5e1ae2 100644 --- a/common/src/main/java/com/metamx/druid/QueryGranularity.java +++ b/common/src/main/java/com/metamx/druid/QueryGranularity.java @@ -19,14 +19,17 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + import org.joda.time.DateTime; import org.joda.time.ReadableDuration; -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property = "type", defaultImpl = QueryGranularity.class) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property = "type", defaultImpl = QueryGranularity.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "period", value = PeriodGranularity.class), @JsonSubTypes.Type(name = "duration", value = DurationGranularity.class), diff --git a/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java index 00722fcf5eb..c12793972f2 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java @@ -19,9 +19,11 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.Comparator; import java.util.List; @@ -36,7 +38,7 @@ import java.util.List; * provided to the Aggregator through the MetricSelector object, so whatever creates that object gets to choose how * the data is actually stored and accessed. */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type") +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type") @JsonSubTypes(value={ @JsonSubTypes.Type(name="count", value=CountAggregatorFactory.class), @JsonSubTypes.Type(name="longSum", value=LongSumAggregatorFactory.class), diff --git a/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java index 61dcde40449..a901c3f54e2 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java @@ -19,11 +19,13 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Longs; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.List; diff --git a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java index dc8bc9a9842..92a972d0658 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Doubles; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java b/common/src/main/java/com/metamx/druid/aggregation/Histogram.java index bb27e317eb4..a3ec32d4a15 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java +++ b/common/src/main/java/com/metamx/druid/aggregation/Histogram.java @@ -19,11 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.base.Preconditions; import com.google.common.primitives.Floats; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; -import org.codehaus.jackson.annotate.JsonValue; + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java index 8902ae56bef..af6d23c32ac 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java @@ -19,13 +19,15 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.google.common.primitives.Floats; import com.google.common.primitives.Longs; import com.metamx.druid.processing.MetricSelectorFactory; import org.apache.commons.codec.binary.Base64; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java b/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java index ab4e8836fe1..01c01719854 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java +++ b/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java @@ -19,16 +19,19 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Arrays; public class HistogramVisual { @JsonProperty final public double[] breaks; - @JsonProperty final public double[] counts; + @JsonProperty + final public double[] counts; // an array of the quantiles including the min. and max. @JsonProperty final public double[] quantiles; diff --git a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java index 48e5cf776f5..5c257f8cc74 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java @@ -19,14 +19,16 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Joiner; import com.google.common.collect.Lists; import com.google.common.primitives.Doubles; import com.metamx.common.IAE; import com.metamx.druid.processing.FloatMetricSelector; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.mozilla.javascript.Context; import org.mozilla.javascript.Function; import org.mozilla.javascript.Script; diff --git a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java index 75e59b39e2c..61247e40add 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Longs; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java index 85a5466ec53..76a57f23066 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Doubles; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java index 49b2ad154c9..2720fa2a9f1 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Doubles; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java index cec19d80d78..1166413a3e7 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java @@ -19,11 +19,13 @@ package com.metamx.druid.aggregation.post; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.Iterator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java index f1bbb0d8392..f36761f962f 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java @@ -19,9 +19,11 @@ package com.metamx.druid.aggregation.post; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.Map; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java index 780c720103d..eb3a531e2d1 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation.post; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; import com.metamx.common.ISE; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.Map; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java index 5b1ebc60528..66427785fbc 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java @@ -19,8 +19,11 @@ package com.metamx.druid.aggregation.post; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.util.Comparator; import java.util.Map; diff --git a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java index b6e195cd5b7..808be63267c 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java @@ -19,7 +19,8 @@ package com.metamx.druid.db; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; import org.skife.config.Config; /** diff --git a/common/src/main/java/com/metamx/druid/input/MapBasedRow.java b/common/src/main/java/com/metamx/druid/input/MapBasedRow.java index d823762eddd..5449ff2f865 100644 --- a/common/src/main/java/com/metamx/druid/input/MapBasedRow.java +++ b/common/src/main/java/com/metamx/druid/input/MapBasedRow.java @@ -19,13 +19,15 @@ package com.metamx.druid.input; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.IAE; import com.metamx.common.exception.FormattedException; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/common/src/main/java/com/metamx/druid/input/Row.java b/common/src/main/java/com/metamx/druid/input/Row.java index add4623d498..353d3e52f06 100644 --- a/common/src/main/java/com/metamx/druid/input/Row.java +++ b/common/src/main/java/com/metamx/druid/input/Row.java @@ -19,8 +19,11 @@ package com.metamx.druid.input; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.util.List; diff --git a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java index 1932746b3d0..dd18713d797 100644 --- a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java +++ b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java @@ -19,10 +19,15 @@ package com.metamx.druid.jackson; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonProcessingException; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.deser.std.StdScalarDeserializer; + + + + + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer; import java.io.IOException; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java index 3b270ad4aa1..a837c88bbe7 100644 --- a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java +++ b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java @@ -19,18 +19,22 @@ package com.metamx.druid.jackson; +import com.fasterxml.jackson.core.JsonGenerationException; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer; import com.google.common.base.Joiner; -import org.codehaus.jackson.JsonGenerationException; -import org.codehaus.jackson.JsonGenerator; -import org.codehaus.jackson.map.SerializerProvider; -import org.codehaus.jackson.map.ser.std.ScalarSerializerBase; + + + + import java.io.IOException; import java.util.List; /** */ -public class CommaListJoinSerializer extends ScalarSerializerBase> +public class CommaListJoinSerializer extends StdScalarSerializer> { private static final Joiner joiner = Joiner.on(","); diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index c80958eff88..84514bd6c4c 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -19,26 +19,45 @@ package com.metamx.druid.jackson; +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.Version; +import com.fasterxml.jackson.databind.DeserializationConfig; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationConfig; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.fasterxml.jackson.datatype.joda.JodaModule; import com.google.common.base.Throwables; import com.metamx.common.Granularity; import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; -import org.codehaus.jackson.JsonFactory; -import org.codehaus.jackson.JsonGenerator; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonProcessingException; -import org.codehaus.jackson.Version; -import org.codehaus.jackson.map.DeserializationConfig; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.JsonDeserializer; -import org.codehaus.jackson.map.JsonSerializer; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.SerializationConfig; -import org.codehaus.jackson.map.SerializerProvider; -import org.codehaus.jackson.map.Serializers; -import org.codehaus.jackson.map.module.SimpleModule; -import org.codehaus.jackson.map.ser.std.ToStringSerializer; + + + + + + + + + + + + + + + import org.joda.time.DateTimeZone; +import org.skife.jdbi.v2.sqlobject.customizers.Mapper; import java.io.IOException; import java.nio.ByteOrder; @@ -153,9 +172,12 @@ public class DefaultObjectMapper extends ObjectMapper ); registerModule(serializerModule); - configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); - configure(SerializationConfig.Feature.AUTO_DETECT_GETTERS, false); - configure(SerializationConfig.Feature.AUTO_DETECT_FIELDS, false); - configure(SerializationConfig.Feature.INDENT_OUTPUT, false); + configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + configure(MapperFeature.AUTO_DETECT_GETTERS, false); + configure(MapperFeature.AUTO_DETECT_CREATORS, false); + configure(MapperFeature.AUTO_DETECT_FIELDS, false); + configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false); + configure(MapperFeature.AUTO_DETECT_SETTERS, false); + configure(SerializationFeature.INDENT_OUTPUT, false); } } diff --git a/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java b/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java index 50d39378390..f06bee8c76d 100644 --- a/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java +++ b/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java @@ -19,19 +19,21 @@ package com.metamx.druid.jackson; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonProcessingException; -import org.codehaus.jackson.JsonToken; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.KeyDeserializer; -import org.codehaus.jackson.map.deser.std.StdDeserializer; -import org.codehaus.jackson.map.ext.JodaDeserializers; -import org.codehaus.jackson.map.module.SimpleModule; -import org.codehaus.jackson.map.ser.std.ToStringSerializer; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.KeyDeserializer; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.fasterxml.jackson.datatype.joda.deser.DurationDeserializer; +import com.fasterxml.jackson.datatype.joda.deser.PeriodDeserializer; import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; +import org.joda.time.Duration; import org.joda.time.Interval; -import org.joda.time.ReadableInstant; +import org.joda.time.Period; import org.joda.time.format.ISODateTimeFormat; import java.io.IOException; @@ -47,6 +49,10 @@ public class JodaStuff module.addSerializer(DateTime.class, ToStringSerializer.instance); module.addDeserializer(Interval.class, new JodaStuff.IntervalDeserializer()); module.addSerializer(Interval.class, ToStringSerializer.instance); + module.addDeserializer(Period.class, new PeriodDeserializer()); + module.addSerializer(Period.class, ToStringSerializer.instance); + module.addDeserializer(Duration.class, new DurationDeserializer()); + module.addSerializer(Duration.class, ToStringSerializer.instance); return module; } diff --git a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java index de0a297c569..ec95cb6b586 100644 --- a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java +++ b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java @@ -19,10 +19,11 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Days; diff --git a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java index c280507e207..f221b0e26df 100644 --- a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java +++ b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java @@ -19,10 +19,11 @@ package com.metamx.druid.histogram; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; import com.metamx.druid.aggregation.Histogram; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java b/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java new file mode 100644 index 00000000000..aff4ad208c6 --- /dev/null +++ b/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java @@ -0,0 +1,40 @@ +/* + * 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.jackson; + +import com.fasterxml.jackson.databind.ObjectMapper; +import junit.framework.Assert; +import org.joda.time.DateTime; +import org.junit.Test; + +/** + */ +public class DefaultObjectMapperTest +{ + ObjectMapper mapper = new DefaultObjectMapper(); + + @Test + public void testDateTime() throws Exception + { + final DateTime time = new DateTime(); + + Assert.assertEquals(String.format("\"%s\"", time), mapper.writeValueAsString(time)); + } +} diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index f431915bbac..592a713cf3a 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -1,5 +1,6 @@ - + 4.0.0 com.metamx.druid druid-examples-rand @@ -75,20 +76,20 @@ icu4j - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile org.jdbi diff --git a/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java b/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java index 66c071a79fe..520b0188171 100644 --- a/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java +++ b/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java @@ -1,5 +1,8 @@ package druid.examples; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.metamx.druid.guava.Runnables; @@ -7,9 +10,9 @@ import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; + + + import java.io.IOException; import java.util.HashMap; diff --git a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java index 4ffd7119431..ecdf2606a3b 100644 --- a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -1,5 +1,6 @@ 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; @@ -13,7 +14,7 @@ import com.metamx.druid.realtime.MetadataUpdaterConfig; import com.metamx.druid.realtime.RealtimeNode; import com.metamx.druid.loading.SegmentPusher; import com.metamx.phonebook.PhoneBook; -import org.codehaus.jackson.map.jsontype.NamedType; + import java.io.File; import java.io.IOException; diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 9ab850948d9..a96af1c08a0 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -1,5 +1,6 @@ - + 4.0.0 com.metamx.druid druid-examples-twitter @@ -75,20 +76,20 @@ icu4j - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile org.jdbi diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index fd072a427f3..e936d481489 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -1,5 +1,6 @@ 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; @@ -14,7 +15,7 @@ import com.metamx.druid.realtime.RealtimeNode; import com.metamx.druid.loading.SegmentPusher; import com.metamx.phonebook.PhoneBook; import druid.examples.twitter.TwitterSpritzerFirehoseFactory; -import org.codehaus.jackson.map.jsontype.NamedType; + import java.io.File; import java.io.IOException; diff --git a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index 992cd239487..1bef29dfc6c 100644 --- a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -1,14 +1,17 @@ package druid.examples.twitter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; + + + import twitter4j.ConnectionLifeCycleListener; import twitter4j.HashtagEntity; import twitter4j.Status; diff --git a/index-common/pom.xml b/index-common/pom.xml index 9d17c6b2c53..fee0b62da1f 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-index-common @@ -52,8 +53,8 @@ extendedset - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core net.java.dev.jets3t diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java b/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java index 2be3a50a595..69cb7ed628a 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java @@ -19,7 +19,8 @@ package com.metamx.druid.index.column; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java b/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java index 7f9b7ac7ee2..587c2455652 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java @@ -19,12 +19,14 @@ package com.metamx.druid.index.column; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.IAE; import com.metamx.druid.index.serde.ColumnPartSerde; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java index d60428fbe3e..d9e53011429 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java @@ -19,9 +19,11 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.index.column.ColumnBuilder; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java index 760fcbb3f20..0dc17bdb0dd 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java @@ -19,12 +19,14 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; import com.metamx.druid.index.v1.serde.ComplexMetrics; import com.metamx.druid.kv.GenericIndexed; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java index d30c6c6bbc0..5b542803c6c 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java @@ -19,6 +19,8 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.IAE; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.column.ValueType; @@ -27,8 +29,8 @@ import com.metamx.druid.kv.GenericIndexed; import com.metamx.druid.kv.VSizeIndexed; import com.metamx.druid.kv.VSizeIndexedInts; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java index 87cb95758db..2d0d5a0dc09 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java @@ -19,11 +19,13 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.column.ValueType; import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java index 0d140751e4c..8304062b05b 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java @@ -19,11 +19,13 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.column.ValueType; import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index 34a8e06962d..6d3badd4c35 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -19,6 +19,7 @@ package com.metamx.druid.index.v1; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -61,7 +62,7 @@ import com.metamx.druid.kv.VSizeIndexedInts; import com.metamx.druid.utils.SerializerUtils; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Interval; import java.io.ByteArrayOutputStream; diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java b/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java index 4a90f35a288..066d6bf47ed 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java @@ -19,7 +19,8 @@ package com.metamx.druid.index.v1.serde; -import org.codehaus.jackson.map.ObjectMapper; + +import com.fasterxml.jackson.databind.ObjectMapper; /** * This is a "factory" interface for registering handlers in the system. It exists because I'm unaware of diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java index 05613567b87..cf2a2d56d0d 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java @@ -19,11 +19,13 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.common.parsers.CSVParser; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java index 668dbc00fff..149a1946c42 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java index d372d46696f..177f0a2c358 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java @@ -19,11 +19,13 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.common.parsers.DelimitedParser; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java index 11228bc132f..74255c2b261 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java @@ -19,9 +19,10 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.parsers.JSONParser; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java b/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java index 3721f2c14f7..e04bd3d1d7c 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java @@ -19,6 +19,8 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -27,8 +29,8 @@ import com.metamx.common.parsers.Parser; import com.metamx.common.parsers.ToLowerCaseParser; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java index 4905f268ed4..0f52bae8bd2 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java @@ -19,10 +19,12 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.metamx.common.parsers.ParserUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import java.util.Map; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java index 2156314b4c6..2882d1a813a 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java @@ -19,9 +19,10 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.common.parsers.Parser; import com.metamx.common.parsers.ToLowerCaseParser; -import org.codehaus.jackson.annotate.JsonValue; + import java.util.List; diff --git a/indexer/pom.xml b/indexer/pom.xml index d38a3a17769..2af06cfbb0c 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-indexer @@ -79,12 +80,12 @@ - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind org.jdbi diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java index 720242466e4..a7dbee1f4e2 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.metamx.common.logger.Logger; @@ -30,7 +31,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index 13490e3ef00..d5bb8a4925a 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Joiner; @@ -63,7 +64,7 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.DateTimeComparator; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java index b020a99fa4e..f9749b79861 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java @@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableList; import com.metamx.common.Pair; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import java.util.List; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index 3d682dadce0..eed2339114b 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -19,6 +19,11 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; @@ -54,11 +59,11 @@ import com.metamx.druid.utils.JodaUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; -import org.codehaus.jackson.JsonGenerator; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + + + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.format.ISODateTimeFormat; @@ -97,24 +102,24 @@ public class HadoopDruidIndexerConfig public static HadoopDruidIndexerConfig fromMap(Map argSpec) { - if (argSpec.containsKey("registererers")) { - List registererers = Lists.transform( - MapUtils.getList(argSpec, "registererers"), - new Function() + List registererers = Lists.transform( + MapUtils.getList(argSpec, "registererers", ImmutableList.of()), + new Function() + { + @Override + public Registererer apply(@Nullable Object input) { - @Override - public Registererer apply(@Nullable Object input) - { - try { - return (Registererer) Class.forName((String) input).newInstance(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } + try { + return (Registererer) Class.forName((String) input).newInstance(); + } + catch (Exception e) { + throw Throwables.propagate(e); } } - ); + } + ); + if (!registererers.isEmpty()) { RegisteringNode.registerHandlers(registererers, Arrays.asList(jsonMapper)); } @@ -125,13 +130,7 @@ public class HadoopDruidIndexerConfig public static HadoopDruidIndexerConfig fromFile(File file) { try { - return fromMap( - (Map) jsonMapper.readValue( - file, new TypeReference>() - { - } - ) - ); + return fromMap((Map) jsonMapper.readValue(file, new TypeReference>(){})); } catch (IOException e) { throw Throwables.propagate(e); @@ -191,14 +190,14 @@ public class HadoopDruidIndexerConfig public HadoopDruidIndexerConfig( final @JsonProperty("intervals") List intervals, final @JsonProperty("dataSource") String dataSource, - final @JsonProperty("timestampColumnName") String timestampColumnName, + final @JsonProperty("timestampColumn") String timestampColumnName, final @JsonProperty("timestampFormat") String timestampFormat, final @JsonProperty("dataSpec") DataSpec dataSpec, final @JsonProperty("segmentGranularity") Granularity segmentGranularity, final @JsonProperty("granularitySpec") GranularitySpec granularitySpec, final @JsonProperty("pathSpec") PathSpec pathSpec, - final @JsonProperty("jobOutputDir") String jobOutputDir, - final @JsonProperty("segmentOutputDir") String segmentOutputDir, + final @JsonProperty("workingPath") String jobOutputDir, + final @JsonProperty("segmentOutputPath") String segmentOutputDir, final @JsonProperty("version") DateTime version, final @JsonProperty("partitionDimension") String partitionDimension, final @JsonProperty("targetPartitionSize") Long targetPartitionSize, @@ -221,7 +220,7 @@ public class HadoopDruidIndexerConfig this.pathSpec = pathSpec; this.jobOutputDir = jobOutputDir; this.segmentOutputDir = segmentOutputDir; - this.version = version; + this.version = version == null ? new DateTime() : version; this.partitionsSpec = partitionsSpec; this.leaveIntermediate = leaveIntermediate; this.cleanupOnFailure = cleanupOnFailure; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java index 15354d0be67..62441560573 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java @@ -19,11 +19,13 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import org.codehaus.jackson.map.jsontype.NamedType; + +import org.joda.time.DateTime; import org.joda.time.Interval; import java.io.File; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java index d7f40087ba8..c0d692c7c5f 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.shard.ShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** * ShardSpec + a shard ID that is unique across this run. The shard ID is used for grouping and partitioning. diff --git a/indexer/src/main/java/com/metamx/druid/indexer/Utils.java b/indexer/src/main/java/com/metamx/druid/indexer/Utils.java index aa1c3ed2f4b..97b046076e3 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/Utils.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/Utils.java @@ -19,6 +19,8 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; import com.metamx.common.ISE; @@ -30,8 +32,8 @@ import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.ReflectionUtils; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import java.io.IOException; import java.io.InputStream; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java b/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java index 151055271ec..e79af1e1df5 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java @@ -19,13 +19,15 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.collect.Iterators; import com.google.common.collect.PeekingIterator; import com.google.common.collect.Sets; import com.metamx.common.guava.Comparators; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java b/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java index 30ce2df1171..ee73f4fdb25 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Optional; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java b/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java index 51d2f37d437..22696a69c5d 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java @@ -19,6 +19,8 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; @@ -26,8 +28,8 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.Granularity; import com.metamx.common.guava.Comparators; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java index 2d00cf71f06..7988fb49c5b 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java @@ -1,7 +1,11 @@ package com.metamx.druid.indexer.partitions; -import org.codehaus.jackson.annotate.JsonIgnore; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; import javax.annotation.Nullable; @@ -14,6 +18,7 @@ public class PartitionsSpec private final boolean assumeGrouped; + @JsonCreator public PartitionsSpec( @JsonProperty("partitionDimension") @Nullable String partitionDimension, @JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize, diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java index 47e1676f848..3bcfb30c31d 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -32,7 +33,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; -import org.codehaus.jackson.annotate.JsonProperty; + import org.jets3t.service.model.S3Object; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java index aeca725bd1c..fb5b9e47ffe 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; import com.metamx.common.Granularity; import com.metamx.common.guava.Comparators; @@ -30,7 +31,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.format.DateTimeFormat; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java index 776f51c99bd..4a0b7278ad7 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java @@ -19,16 +19,18 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.indexer.HadoopDruidIndexerConfig; import org.apache.hadoop.mapreduce.Job; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.io.IOException; /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type") +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type") @JsonSubTypes(value={ @JsonSubTypes.Type(name="granular_unprocessed", value=GranularUnprocessedPathSpec.class), @JsonSubTypes.Type(name="granularity", value=GranularityPathSpec.class), diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java index f46135c32cc..9a244c2f74a 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java @@ -19,11 +19,12 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.logger.Logger; import com.metamx.druid.indexer.HadoopDruidIndexerConfig; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.codehaus.jackson.annotate.JsonProperty; + import java.io.IOException; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java index e2e1f9ce032..a223589db08 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java @@ -19,10 +19,11 @@ package com.metamx.druid.indexer.rollup; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.Granularity; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.List; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java index 9bf40d86657..7a8892e9160 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java @@ -19,8 +19,9 @@ package com.metamx.druid.indexer.updater; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.db.DbConnectorConfig; -import org.codehaus.jackson.annotate.JsonProperty; + /** */ diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java index 3523a198ba6..2957c1b3bf7 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java @@ -19,8 +19,9 @@ package com.metamx.druid.indexer.updater; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java index be1668afa93..1199e1ddf44 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java @@ -19,7 +19,8 @@ package com.metamx.druid.indexer.updater; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java index f4db1148327..5fdff8ce8b8 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -19,12 +19,13 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.druid.indexer.granularity.UniformGranularitySpec; import com.metamx.druid.indexer.partitions.PartitionsSpec; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -39,8 +40,8 @@ public class HadoopDruidIndexerConfigTest try { cfg = jsonMapper.readValue( - "{" - + "\"granularitySpec\":{" + "{" + + " \"granularitySpec\":{" + " \"type\":\"uniform\"," + " \"gran\":\"hour\"," + " \"intervals\":[\"2012-01-01/P1D\"]" diff --git a/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java b/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java index 0044d7d13e6..ecf6918b289 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java @@ -19,11 +19,12 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java b/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java index ab21be5f9f5..ea29d5ae727 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java @@ -19,12 +19,13 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.common.Granularity; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/merger/pom.xml b/merger/pom.xml index 980aa96d5f0..7f91b1e8b92 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-merger @@ -113,16 +114,16 @@ guice-servlet - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind javax.inject diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java index d9479a7dbbf..9abb60d063a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java @@ -19,10 +19,12 @@ package com.metamx.druid.merger.common; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskContext; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** */ diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java index 39b38a18a93..d6cb18093b8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java @@ -19,13 +19,15 @@ package com.metamx.druid.merger.common; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.task.Task; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Collections; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index 74a546cf696..0cebe1fc91c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.common; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.druid.loading.S3SegmentPuller; import com.metamx.druid.loading.S3SegmentGetterConfig; @@ -28,7 +29,7 @@ import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.loading.SegmentPusher; import com.metamx.emitter.service.ServiceEmitter; -import org.codehaus.jackson.map.ObjectMapper; + import org.jets3t.service.impl.rest.httpclient.RestS3Service; import java.io.File; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java index 28b78f78ad6..1ea40a95d2a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java @@ -19,6 +19,10 @@ package com.metamx.druid.merger.common.index; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; @@ -31,10 +35,10 @@ import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; import org.apache.commons.io.IOUtils; import org.apache.commons.io.LineIterator; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; -import org.codehaus.jackson.map.annotate.JacksonInject; + + + + import org.jets3t.service.S3Service; import org.jets3t.service.model.S3Object; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index 634fe65ebaf..7d456d29e01 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -19,6 +19,10 @@ package com.metamx.druid.merger.common.index; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -39,10 +43,10 @@ import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.PlumberSchool; import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.Sink; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; -import org.codehaus.jackson.map.annotate.JacksonInject; + + + + import org.joda.time.Interval; import java.io.File; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index 6de3eb0d73f..ae383f49f60 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -19,12 +19,14 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.coordinator.TaskContext; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; public abstract class AbstractTask implements Task diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java index f1153e5c43c..b3c90fa1330 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +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.Predicate; @@ -33,8 +35,8 @@ import com.metamx.druid.index.v1.IndexableAdapter; import com.metamx.druid.index.v1.QueryableIndexIndexableAdapter; import com.metamx.druid.index.v1.Rowboat; import com.metamx.druid.index.v1.RowboatFilteringIndexAdapter; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import javax.annotation.Nullable; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java index e17db4b980e..4ba3ab38030 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -28,8 +30,8 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import javax.annotation.Nullable; import java.io.File; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 5f08298cd2d..5f37ad2853e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import com.metamx.druid.QueryGranularity; @@ -33,9 +35,9 @@ import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.ObjectMapper; + + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 55edfdbc3bc..110fdbf86f6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -38,8 +40,8 @@ import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import javax.annotation.Nullable; @@ -49,7 +51,8 @@ import java.util.Set; public class IndexDeterminePartitionsTask extends AbstractTask { - @JsonProperty private final FirehoseFactory firehoseFactory; + @JsonProperty + private final FirehoseFactory firehoseFactory; @JsonProperty private final Schema schema; @JsonProperty private final long targetPartitionSize; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index b89142ef19a..b3da8978bd3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; @@ -36,8 +38,8 @@ import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.Schema; import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.realtime.Sink; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 6074765ddbf..ca514cec52b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; @@ -31,8 +33,8 @@ import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.realtime.FirehoseFactory; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 20fefa0014f..21f6c1e6416 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -19,6 +19,9 @@ package com.metamx.druid.merger.common.task; +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.Joiner; import com.google.common.base.Objects; @@ -43,9 +46,9 @@ import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import org.apache.commons.codec.digest.DigestUtils; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 807172d11ae..60d1cf21e39 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -19,14 +19,16 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.merger.common.task.IndexDeterminePartitionsTask; import com.metamx.druid.merger.common.task.IndexGeneratorTask; import com.metamx.druid.merger.common.task.IndexTask; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java index 1a718fdaf20..32e8bfd71dd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java @@ -1,9 +1,10 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java index c1dbce4fc61..41032cc831f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; @@ -29,7 +30,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java index bc53ef0d4f7..f226a19be4f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; @@ -27,7 +28,7 @@ import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.db.DbConnectorConfig; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.skife.jdbi.v2.DBI; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index a95f64cb623..4157d6a66ca 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Predicate; @@ -48,7 +49,7 @@ import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; import com.netflix.curator.utils.ZKPaths; import org.apache.zookeeper.CreateMode; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java index fcbb2450cbc..d3a32aca462 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java @@ -19,9 +19,11 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Set; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index c6353698cbd..8203781bbf9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -19,6 +19,7 @@ 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.Lists; @@ -27,7 +28,7 @@ import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.worker.Worker; import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java index 26f0105771d..5b342883bb2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java @@ -19,8 +19,9 @@ package com.metamx.druid.merger.coordinator.config; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.db.DbConnectorConfig; -import org.codehaus.jackson.annotate.JsonProperty; + import org.skife.config.Config; public abstract class IndexerDbConnectorConfig extends DbConnectorConfig diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 3dae4046764..7d075cbe9e8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -21,6 +21,8 @@ package com.metamx.druid.merger.coordinator.http; 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.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -89,8 +91,8 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; -import org.codehaus.jackson.map.InjectableValues; -import org.codehaus.jackson.map.ObjectMapper; + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java index 4cc1df9fa6f..1012beb830b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.coordinator.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; @@ -26,8 +28,8 @@ import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import org.codehaus.jackson.jaxrs.JacksonJsonProvider; -import org.codehaus.jackson.map.ObjectMapper; + + import javax.inject.Singleton; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index 8d51da61afd..16b629d9abb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -28,6 +28,7 @@ import com.amazonaws.services.ec2.model.Reservation; import com.amazonaws.services.ec2.model.RunInstancesRequest; import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; @@ -36,7 +37,7 @@ import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.EmittingLogger; import org.apache.commons.codec.binary.Base64; -import org.codehaus.jackson.map.ObjectMapper; + import javax.annotation.Nullable; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java index 8d302df25f6..4c2b86f4f6f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java @@ -19,8 +19,11 @@ package com.metamx.druid.merger.coordinator.setup; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java index 876a2635273..76061637312 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java @@ -19,8 +19,9 @@ package com.metamx.druid.merger.coordinator.setup; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java index 8395fa2d6c8..fada73cb40e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java @@ -19,8 +19,11 @@ package com.metamx.druid.merger.coordinator.setup; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java index 5e43e68ae66..89a0dd2d5c1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator.setup; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.common.ISE; @@ -28,7 +29,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; import org.apache.commons.collections.MapUtils; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Duration; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.FoldController; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index 5fc49788fcd..c41f4ac8be4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -32,7 +32,7 @@ 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.codehaus.jackson.map.ObjectMapper; + import java.io.File; import java.util.concurrent.ExecutorService; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java b/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java index a1ebf273521..6b349de38f7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java @@ -19,9 +19,11 @@ package com.metamx.druid.merger.worker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.merger.worker.config.WorkerConfig; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** * A container for worker metadata. diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java index d4237da7a9e..ab3a35d53de 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.worker; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -32,7 +33,7 @@ import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.state.ConnectionState; import com.netflix.curator.framework.state.ConnectionStateListener; import org.apache.zookeeper.CreateMode; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import java.util.Arrays; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 0799a8de37c..237ae09d854 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.worker.http; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.concurrent.ScheduledExecutorFactory; @@ -59,8 +61,8 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; -import org.codehaus.jackson.map.InjectableValues; -import org.codehaus.jackson.map.ObjectMapper; + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index b7107ed72d2..ded6b55d64d 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -1,5 +1,10 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -29,10 +34,10 @@ import com.netflix.curator.retry.ExponentialBackoffRetry; import com.netflix.curator.test.TestingCluster; import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.zookeeper.CreateMode; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.jsontype.NamedType; + + + + import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -495,6 +500,7 @@ public class RemoteTaskRunnerTest private final List segments; private final List aggregators; + @JsonCreator public TestTask( @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, diff --git a/pom.xml b/pom.xml index aa77ea6894b..7b1e3f9e062 100644 --- a/pom.xml +++ b/pom.xml @@ -38,6 +38,7 @@ UTF-8 + 0.20.0 @@ -58,17 +59,17 @@ com.metamx emitter - 0.0.7 + 0.2.0 com.metamx http-client - 0.6.1 + 0.7.0 com.metamx java-util - 0.19.1 + ${metamx.java-util.version} com.metamx @@ -162,24 +163,34 @@ 4.8.1 - org.codehaus.jackson - jackson-core-asl - 1.9.9 + com.fasterxml.jackson.core + jackson-annotations + 2.1.2 - org.codehaus.jackson - jackson-jaxrs - 1.9.9 + com.fasterxml.jackson.core + jackson-core + 2.1.3 - org.codehaus.jackson - jackson-mapper-asl - 1.9.9 + com.fasterxml.jackson.core + jackson-databind + 2.1.3 - org.codehaus.jackson - jackson-smile - 1.9.9 + com.fasterxml.jackson.datatype + jackson-datatype-joda + 2.1.2 + + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + 2.1.3 + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + 2.1.3 javax.inject @@ -269,7 +280,7 @@ java-util test-jar test - 0.16.0 + ${metamx.java-util.version} diff --git a/realtime/pom.xml b/realtime/pom.xml index a8c50128f2b..26c23ad9133 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-realtime @@ -82,16 +83,16 @@ guava - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile org.jdbi diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java index b5709de1d6b..aab4509bbe5 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java @@ -19,8 +19,11 @@ package com.metamx.druid.realtime; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.io.IOException; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java index b2afbbc4884..d98997b5051 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java @@ -19,8 +19,9 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.Period; /** diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java index e090e088d08..40d0e662b49 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java @@ -19,8 +19,11 @@ package com.metamx.druid.realtime; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.io.IOException; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java index 05714ddfdbd..12c74ad6b16 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java @@ -19,6 +19,8 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import com.metamx.common.exception.FormattedException; @@ -30,8 +32,8 @@ import kafka.consumer.ConsumerConfig; import kafka.consumer.KafkaMessageStream; import kafka.javaapi.consumer.ConsumerConnector; import kafka.message.Message; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.CharBuffer; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java b/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java index 5bb10b36753..2d377124cc3 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java @@ -19,13 +19,14 @@ 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.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java index 0f0d87b3490..5fcc1f29f7d 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java @@ -19,8 +19,9 @@ package com.metamx.druid.realtime; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index f503e80ade3..d8fc7ebeea7 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -19,6 +19,12 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Maps; @@ -47,12 +53,12 @@ import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.Monitor; -import org.codehaus.jackson.map.BeanProperty; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.InjectableValues; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.smile.SmileFactory; -import org.codehaus.jackson.type.TypeReference; + + + + + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java index f4df5e054f8..47a4ef0f40f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -19,6 +19,11 @@ 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; @@ -53,11 +58,11 @@ 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.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; -import org.codehaus.jackson.map.annotate.JacksonInject; + + + + + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java index 1b22f1e3a78..5a74b17e223 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java @@ -19,9 +19,10 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.loading.S3SegmentPusherConfig; import com.metamx.druid.loading.SegmentPusher; -import org.codehaus.jackson.map.ObjectMapper; + import org.jets3t.service.impl.rest.httpclient.RestS3Service; /** diff --git a/realtime/src/main/java/com/metamx/druid/realtime/Schema.java b/realtime/src/main/java/com/metamx/druid/realtime/Schema.java index b54c8efbbf6..ee3d63a14f8 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/Schema.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/Schema.java @@ -19,14 +19,16 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Arrays; diff --git a/server/pom.xml b/server/pom.xml index 544108cef5f..66309b73c30 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-server @@ -107,20 +108,20 @@ guice-servlet - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile javax.inject @@ -154,10 +155,10 @@ joda-time joda-time - - com.google.code.findbugs - jsr305 - + + com.google.code.findbugs + jsr305 + log4j log4j diff --git a/server/src/main/java/com/metamx/TsvToJson.java b/server/src/main/java/com/metamx/TsvToJson.java index 9d4417fe597..65effd5077e 100644 --- a/server/src/main/java/com/metamx/TsvToJson.java +++ b/server/src/main/java/com/metamx/TsvToJson.java @@ -19,11 +19,12 @@ package com.metamx; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.collect.Maps; import com.metamx.common.IAE; import com.metamx.common.ISE; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.DateTimeZone; diff --git a/server/src/main/java/com/metamx/druid/BaseServerNode.java b/server/src/main/java/com/metamx/druid/BaseServerNode.java index cdc7a68e4f6..dbbe286e270 100644 --- a/server/src/main/java/com/metamx/druid/BaseServerNode.java +++ b/server/src/main/java/com/metamx/druid/BaseServerNode.java @@ -19,6 +19,7 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; @@ -29,7 +30,7 @@ import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import org.codehaus.jackson.map.ObjectMapper; + import org.skife.config.ConfigurationObjectFactory; import java.nio.ByteBuffer; diff --git a/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java b/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java index 4e68ad2995e..a6d342d8419 100644 --- a/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java +++ b/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java @@ -19,13 +19,15 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="action") +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="action") @JsonSubTypes(value={ @JsonSubTypes.Type(name="load", value=SegmentChangeRequestLoad.class), @JsonSubTypes.Type(name="drop", value=SegmentChangeRequestDrop.class) diff --git a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java index 0093907d26e..b231781e394 100644 --- a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java +++ b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java @@ -19,10 +19,13 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonUnwrapped; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonUnwrapped; + + + /** */ diff --git a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java index c98106efdfb..ecf35513af9 100644 --- a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java @@ -19,10 +19,13 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonUnwrapped; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonUnwrapped; + + + /** */ diff --git a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java index 1fb756daafe..57d64e0ba32 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java +++ b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java @@ -19,6 +19,7 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; @@ -33,7 +34,7 @@ import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import java.io.File; diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index 2840eb037a8..e76b372978f 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -19,6 +19,8 @@ package com.metamx.druid.db; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -29,8 +31,8 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.master.rules.PeriodLoadRule; import com.metamx.druid.master.rules.Rule; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index ef620cd1e38..b17a2f63299 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -19,6 +19,7 @@ package com.metamx.druid.db; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -31,7 +32,7 @@ import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index c6d284403c0..2230932d9a1 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -46,8 +48,8 @@ import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.metrics.Monitor; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.smile.SmileFactory; + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index 51df502fa64..5ba559d4656 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -19,6 +19,7 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -67,7 +68,7 @@ import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.x.discovery.ServiceDiscovery; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.ZkClient; -import org.codehaus.jackson.map.ObjectMapper; + import org.mortbay.jetty.Server; import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.DefaultServlet; diff --git a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java index 1d33550635c..47395f73eeb 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java +++ b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.coordination.DruidClusterInfo; @@ -27,8 +29,8 @@ import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.master.DruidMaster; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import org.codehaus.jackson.jaxrs.JacksonJsonProvider; -import org.codehaus.jackson.map.ObjectMapper; + + import javax.inject.Singleton; diff --git a/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java b/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java index 3fdcd5dbdc5..a609329b0fd 100644 --- a/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java +++ b/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java @@ -19,8 +19,9 @@ package com.metamx.druid.http; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/server/src/main/java/com/metamx/druid/http/SegmentToMove.java b/server/src/main/java/com/metamx/druid/http/SegmentToMove.java index b35c3ce4d7c..d54a78b8bba 100644 --- a/server/src/main/java/com/metamx/druid/http/SegmentToMove.java +++ b/server/src/main/java/com/metamx/druid/http/SegmentToMove.java @@ -19,8 +19,9 @@ package com.metamx.druid.http; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java b/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java index f97d6dde379..2bf090850b6 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java @@ -19,10 +19,12 @@ package com.metamx.druid.index.brita; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Predicate; import com.metamx.druid.query.search.SearchQuerySpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java b/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java index 3069984739b..4628028ee0b 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java @@ -19,8 +19,9 @@ package com.metamx.druid.index.v1; +import com.fasterxml.jackson.annotation.JsonCreator; import com.metamx.druid.QueryGranularity; -import org.codehaus.jackson.annotate.JsonCreator; + import org.joda.time.DateTime; import org.joda.time.Days; import org.joda.time.Hours; diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java index 4d64c3ed464..5af4b905719 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java @@ -19,6 +19,7 @@ package com.metamx.druid.loading; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.common.io.Closeables; @@ -29,7 +30,7 @@ import com.metamx.druid.index.v1.IndexIO; import com.metamx.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; -import org.codehaus.jackson.map.ObjectMapper; + import org.jets3t.service.S3ServiceException; import org.jets3t.service.acl.gs.GSAccessControlList; import org.jets3t.service.impl.rest.httpclient.RestS3Service; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 3d9c8ecffe0..281c4cf5c3d 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -19,6 +19,7 @@ package com.metamx.druid.master; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; @@ -49,7 +50,7 @@ import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.exception.ZkNodeExistsException; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Duration; import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java index e7f3d5f41ed..f3d93d0e6fd 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java @@ -19,6 +19,7 @@ package com.metamx.druid.master; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -41,7 +42,7 @@ import com.metamx.http.client.HttpClientConfig; import com.metamx.http.client.HttpClientInit; import com.metamx.http.client.response.ToStringResponseHandler; import com.netflix.curator.x.discovery.ServiceProvider; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java b/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java index 32c77fdff3d..07a0f8dba6e 100644 --- a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java +++ b/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java @@ -19,13 +19,14 @@ package com.metamx.druid.master; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merge.ClientAppendQuery; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.HttpResponseHandler; import com.netflix.curator.x.discovery.ServiceProvider; -import org.codehaus.jackson.map.ObjectMapper; + import java.net.URL; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java index 0acdd8bc2f8..1e9bef477b5 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java @@ -19,9 +19,11 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java index 5aa984ccba8..bb50ef352fa 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java @@ -19,10 +19,12 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java index 152f074dc3a..22f1794189d 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java @@ -19,9 +19,11 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java index 051967e65ab..a332c302dd7 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java @@ -19,10 +19,12 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/master/rules/Rule.java b/server/src/main/java/com/metamx/druid/master/rules/Rule.java index a6eced93c68..3d2ea2f59c1 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/Rule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/Rule.java @@ -19,12 +19,14 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.client.DataSegment; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterRuntimeParams; import com.metamx.druid.master.MasterStats; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.DateTime; /** diff --git a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java index 62cb939e0ed..38a63d57f0c 100644 --- a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java @@ -19,13 +19,15 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java index 122b779e922..41ed201242c 100644 --- a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java @@ -19,13 +19,15 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index de148a9219c..762662741a6 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.collect.ImmutableList; @@ -60,7 +61,7 @@ import com.metamx.druid.result.SearchResultValue; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java index 5165647b24f..99019ef5568 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.util.concurrent.MoreExecutors; @@ -33,7 +34,7 @@ import com.metamx.druid.loading.NoopSegmentLoader; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.NoopQueryRunnerFactoryConglomerate; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.map.ObjectMapper; + import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Assert; From ee193f0ca7e8b9f7676b7388114470c5beb67135 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 13 Feb 2013 09:50:30 -0800 Subject: [PATCH 38/63] DeterminePartitionsJob: Select partition dimension to minimize segment size variance when cardinality is low --- .../druid/indexer/DeterminePartitionsJob.java | 36 ++++++++++++++++--- 1 file changed, 32 insertions(+), 4 deletions(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index d5bb8a4925a..4b4a5c8b995 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -500,6 +500,7 @@ public class DeterminePartitionsJob implements Jobby { private static final double SHARD_COMBINE_THRESHOLD = 0.25; private static final double SHARD_OVERSIZE_THRESHOLD = 1.5; + private static final int HIGH_CARDINALITY_THRESHOLD = 3000000; @Override protected void innerReduce( @@ -634,7 +635,9 @@ public class DeterminePartitionsJob implements Jobby final int totalRows = dimPartitionss.values().iterator().next().getRows(); - int maxCardinality = -1; + int maxCardinality = Integer.MIN_VALUE; + long minVariance = Long.MAX_VALUE; + DimPartitions minVariancePartitions = null; DimPartitions maxCardinalityPartitions = null; for(final DimPartitions dimPartitions : dimPartitionss.values()) { @@ -660,10 +663,18 @@ public class DeterminePartitionsJob implements Jobby continue; } - if(dimPartitions.getCardinality() > maxCardinality) { - maxCardinality = dimPartitions.getCardinality(); + final int cardinality = dimPartitions.getCardinality(); + final long variance = dimPartitions.getVariance(); + + if(cardinality > maxCardinality) { + maxCardinality = cardinality; maxCardinalityPartitions = dimPartitions; } + + if(variance < minVariance) { + minVariance = variance; + minVariancePartitions = dimPartitions; + } } if(maxCardinalityPartitions == null) { @@ -675,8 +686,12 @@ public class DeterminePartitionsJob implements Jobby context, config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)), config.isOverwriteFiles() ); + final DimPartitions chosenPartitions = maxCardinality > HIGH_CARDINALITY_THRESHOLD + ? maxCardinalityPartitions + : minVariancePartitions; + final List chosenShardSpecs = Lists.transform( - maxCardinalityPartitions.partitions, new Function() + chosenPartitions.partitions, new Function() { @Override public ShardSpec apply(DimPartition dimPartition) @@ -752,6 +767,19 @@ public class DeterminePartitionsJob implements Jobby return sum; } + public long getVariance() + { + final long meanRows = getRows() / partitions.size(); + + long variance = 0; + for(final DimPartition dimPartition : partitions) { + variance += (dimPartition.rows - meanRows) * (dimPartition.rows - meanRows); + } + + variance /= partitions.size(); + return variance; + } + public int getRows() { int sum = 0; From e3822f6ca795cb7fc8385b97147959d8b16ce582 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 13 Feb 2013 10:16:03 -0800 Subject: [PATCH 39/63] DeterminePartitionsJob: Fix docs --- .../com/metamx/druid/indexer/DeterminePartitionsJob.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index 4b4a5c8b995..d4ee1941396 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -79,13 +79,16 @@ import java.util.Set; /** * Determines appropriate ShardSpecs for a job by determining whether or not partitioning is necessary, and if so, - * choosing the highest cardinality dimension that satisfies the criteria: + * choosing the best dimension that satisfies the criteria: * *

      *
    • Must have exactly one value per row.
    • *
    • Must not generate oversized partitions. A dimension with N rows having the same value will necessarily * put all those rows in the same partition, and that partition may be much larger than the target size.
    • *
    + * + * "Best" means a very high cardinality dimension, or, if none exist, the dimension that minimizes segment size + * variance. */ public class DeterminePartitionsJob implements Jobby { From 1b85eaad3e39aea91461b0406a6a70b112ef09dd Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 8 Feb 2013 16:50:23 -0800 Subject: [PATCH 40/63] simplify query toolchest interface --- .../metamx/druid/query/QueryToolChest.java | 12 ++---- .../TimeBoundaryQueryQueryToolChest.java | 39 +++++-------------- .../java/com/metamx/druid/LogicalSegment.java | 8 ++++ .../metamx/druid/TimelineObjectHolder.java | 3 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- 6 files changed, 25 insertions(+), 41 deletions(-) create mode 100644 common/src/main/java/com/metamx/druid/LogicalSegment.java diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index f729016032e..e2d227729d3 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -22,12 +22,9 @@ package com.metamx.druid.query; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; +import com.metamx.druid.LogicalSegment; import com.metamx.druid.Query; -import com.metamx.druid.TimelineObjectHolder; -import com.metamx.druid.client.selector.ServerSelector; import com.metamx.emitter.service.ServiceMetricEvent; - - import java.util.List; /** @@ -50,7 +47,7 @@ public abstract class QueryToolChest makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); public abstract TypeReference getResultTypeReference(); - public CacheStrategy getCacheStrategy(QueryType query) { + public CacheStrategy getCacheStrategy(QueryType query) { return null; } @@ -62,10 +59,7 @@ public abstract class QueryToolChest> filterSegments( - QueryType query, - List> segments - ) { + public List filterSegments(QueryType query, List segments) { return segments; } } diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 86e618bcee8..4ff1ffdf533 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -22,16 +22,14 @@ package com.metamx.druid.query.timeboundary; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Ordering; -import com.google.common.collect.Sets; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import com.metamx.druid.LogicalSegment; import com.metamx.druid.Query; -import com.metamx.druid.TimelineObjectHolder; -import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.collect.OrderedMergeSequence; import com.metamx.druid.query.BySegmentSkippingQueryRunner; import com.metamx.druid.query.CacheStrategy; @@ -47,8 +45,6 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.List; -import java.util.Map; -import java.util.Set; /** */ @@ -64,32 +60,17 @@ public class TimeBoundaryQueryQueryToolChest { }; - @Override - public List> filterSegments( - TimeBoundaryQuery query, - List> input - ) { - long minMillis = Long.MAX_VALUE; - long maxMillis = Long.MIN_VALUE; - TimelineObjectHolder min = null; - TimelineObjectHolder max = null; - - for(TimelineObjectHolder e : input) { - final long start = e.getInterval().getStartMillis(); - final long end = e.getInterval().getEndMillis(); - - if(min == null || start < minMillis) { - min = e; - minMillis = start; - } - if(max == null || end > maxMillis) { - max = e; - maxMillis = end; - } + public List filterSegments(TimeBoundaryQuery query, List input) + { + if(input.size() <= 1) { + return input; } - return min == max ? Lists.newArrayList(min) : Lists.newArrayList(min , max); + return Lists.newArrayList( + Iterables.getFirst(input, null), + Iterables.getLast(input) + ); } @Override diff --git a/common/src/main/java/com/metamx/druid/LogicalSegment.java b/common/src/main/java/com/metamx/druid/LogicalSegment.java new file mode 100644 index 00000000000..5550fb44966 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/LogicalSegment.java @@ -0,0 +1,8 @@ +package com.metamx.druid; + +import org.joda.time.Interval; + +public interface LogicalSegment +{ + public Interval getInterval(); +} diff --git a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java index 403fe8bb2cb..d93e5ed8f6b 100644 --- a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java +++ b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java @@ -24,7 +24,7 @@ import org.joda.time.Interval; /** */ -public class TimelineObjectHolder +public class TimelineObjectHolder implements LogicalSegment { private final Interval interval; private final VersionType version; @@ -41,6 +41,7 @@ public class TimelineObjectHolder this.object = object; } + @Override public Interval getInterval() { return interval; diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 592a713cf3a..c496170be99 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -10,7 +10,7 @@ com.metamx druid-examples - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a96af1c08a0..a327c94881d 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -10,7 +10,7 @@ com.metamx druid-examples - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT From 6998d604a293f752af8649c7f12b31f2801decaa Mon Sep 17 00:00:00 2001 From: xvrl Date: Thu, 14 Feb 2013 12:41:34 -0800 Subject: [PATCH 41/63] fixes #80 --- .../query/timeboundary/TimeBoundaryQueryQueryToolChest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 4ff1ffdf533..9701d9eee76 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -41,7 +41,6 @@ import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; import org.joda.time.DateTime; - import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.List; From ff325b1f65728509fa890e499dfc4ecce3e42cb9 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 12:46:40 -0800 Subject: [PATCH 42/63] Merger: Changes to support rolling locks and a task-action API Generally, TaskStatus is no longer used as the sole way for tasks to communicate status and metadata update requests back to the coordinator. It is still used to communicate success or failure, but a TaskActionClient is used for all other communication. In remote mode this uses an http api. TaskContext no longer exists, as its purpose is now served by the TaskActionClient. The TaskQueue and TaskConsumer are no longer in charge of committing stuff for a task. Instead, it happens when an action is sent in. Tasks are expected to ensure that this is OK by making their behavior idempotent (since they may be retried after partially completing). One new kind of request afforded to tasks is the ability to release and acquire arbitrary locks. This will be useful for long-running tasks. Short-lived tasks can still be given a lock automatically upon startup by providing a "fixed interval". New tables when using db storage: - tasklocks, to support persistence of arbitrary locks. - tasklogs, to log actions taken by tasks --- .../druid/merger/common/TaskCallback.java | 2 - .../druid/merger/common/TaskHolder.java | 55 -- .../TaskGroup.java => common/TaskLock.java} | 82 +-- .../druid/merger/common/TaskStatus.java | 101 +-- .../druid/merger/common/TaskToolbox.java | 19 +- .../common/actions/LocalTaskActionClient.java | 37 + .../common/actions/LockAcquireAction.java | 53 ++ .../merger/common/actions/LockListAction.java | 45 ++ .../common/actions/LockReleaseAction.java | 55 ++ .../actions/RemoteTaskActionClient.java | 53 ++ .../common/actions/SegmentInsertAction.java | 95 +++ .../actions/SegmentListUnusedAction.java | 63 ++ .../common/actions/SegmentListUsedAction.java | 63 ++ .../common/actions/SegmentNukeAction.java | 95 +++ .../common/actions/SpawnTasksAction.java | 57 ++ .../merger/common/actions/TaskAction.java | 24 + .../common/actions/TaskActionClient.java | 6 + .../common/actions/TaskActionToolbox.java | 47 ++ .../merger/common/task/AbstractTask.java | 44 +- .../druid/merger/common/task/AppendTask.java | 6 +- .../merger/common/task/DefaultMergeTask.java | 4 +- .../druid/merger/common/task/DeleteTask.java | 27 +- .../task/IndexDeterminePartitionsTask.java | 67 +- .../common/task/IndexGeneratorTask.java | 40 +- .../druid/merger/common/task/IndexTask.java | 14 +- .../druid/merger/common/task/KillTask.java | 61 +- .../druid/merger/common/task/MergeTask.java | 29 +- .../metamx/druid/merger/common/task/Task.java | 45 +- .../merger/coordinator/DbTaskStorage.java | 263 +++++-- .../merger/coordinator/LocalTaskRunner.java | 4 +- .../merger/coordinator/LocalTaskStorage.java | 213 ++++-- .../coordinator/MergerDBCoordinator.java | 59 +- .../merger/coordinator/RemoteTaskRunner.java | 12 +- .../druid/merger/coordinator/TaskContext.java | 66 -- .../druid/merger/coordinator/TaskLockbox.java | 405 +++++++++++ ...skMaster.java => TaskMasterLifecycle.java} | 46 +- .../druid/merger/coordinator/TaskQueue.java | 551 ++++----------- .../druid/merger/coordinator/TaskRunner.java | 7 +- .../druid/merger/coordinator/TaskStorage.java | 33 +- .../coordinator/TaskStorageQueryAdapter.java | 84 ++- .../druid/merger/coordinator/TaskWrapper.java | 9 +- .../coordinator/VersionedTaskWrapper.java | 44 -- .../coordinator/commit/CommitStyle.java | 32 - .../commit/ImmediateCommitStyle.java | 33 - .../coordinator/commit/TaskCommitStyle.java | 32 - .../config/IndexerDbConnectorConfig.java | 8 + .../merger/coordinator/exec/TaskConsumer.java | 191 +----- .../http/IndexerCoordinatorNode.java | 93 ++- .../http/IndexerCoordinatorResource.java | 71 +- .../http/IndexerCoordinatorServletModule.java | 9 +- .../druid/merger/worker/TaskMonitor.java | 19 +- .../druid/merger/worker/http/WorkerNode.java | 65 +- .../merger/common/task/MergeTaskTest.java | 8 +- .../merger/common/task/TaskSerdeTest.java | 108 +++ .../coordinator/RemoteTaskRunnerTest.java | 25 +- .../merger/coordinator/TaskConsumerTest.java | 230 ------- .../merger/coordinator/TaskLifecycleTest.java | 644 ++++++++++++++++++ .../merger/coordinator/TaskQueueTest.java | 308 +++------ 58 files changed, 3032 insertions(+), 1929 deletions(-) delete mode 100644 merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java rename merger/src/main/java/com/metamx/druid/merger/{coordinator/TaskGroup.java => common/TaskLock.java} (50%) create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClient.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java rename merger/src/main/java/com/metamx/druid/merger/coordinator/{TaskMaster.java => TaskMasterLifecycle.java} (82%) delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/VersionedTaskWrapper.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/commit/CommitStyle.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/commit/ImmediateCommitStyle.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/commit/TaskCommitStyle.java create mode 100644 merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java delete mode 100644 merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java create mode 100644 merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java index a134ad2455e..edf3c9bae86 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java @@ -19,8 +19,6 @@ package com.metamx.druid.merger.common; -import com.metamx.druid.merger.common.TaskStatus; - public interface TaskCallback { public void notify(TaskStatus status); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java deleted file mode 100644 index d9479a7dbbf..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java +++ /dev/null @@ -1,55 +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.merger.common; - -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskContext; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; - -/** - */ -public class TaskHolder -{ - private final Task task; - private final TaskContext taskContext; - - @JsonCreator - public TaskHolder( - @JsonProperty("task") Task task, - @JsonProperty("taskContext") TaskContext taskContext - ) - { - this.task = task; - this.taskContext = taskContext; - } - - @JsonProperty - public Task getTask() - { - return task; - } - - @JsonProperty - public TaskContext getTaskContext() - { - return taskContext; - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskLock.java similarity index 50% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java rename to merger/src/main/java/com/metamx/druid/merger/common/TaskLock.java index 853e18c918c..6dd891ff500 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskLock.java @@ -17,30 +17,30 @@ * 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.google.common.base.Objects; -import com.google.common.base.Preconditions; -import com.google.common.collect.Maps; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.commit.CommitStyle; -import com.metamx.druid.merger.coordinator.commit.ImmediateCommitStyle; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; import org.joda.time.Interval; -import java.util.Map; - /** - * Represents a transaction as well as the lock it holds. Not immutable: the task set can change. + * Represents a lock held by some task. Immutable. */ -public class TaskGroup +public class TaskLock { private final String groupId; private final String dataSource; private final Interval interval; private final String version; - private final Map taskMap = Maps.newHashMap(); - public TaskGroup(String groupId, String dataSource, Interval interval, String version) + @JsonCreator + public TaskLock( + @JsonProperty("groupId") String groupId, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval, + @JsonProperty("version") String version + ) { this.groupId = groupId; this.dataSource = dataSource; @@ -48,75 +48,48 @@ public class TaskGroup this.version = version; } + @JsonProperty public String getGroupId() { return groupId; } + @JsonProperty public String getDataSource() { return dataSource; } + @JsonProperty public Interval getInterval() { return interval; } + @JsonProperty public String getVersion() { return version; } - public CommitStyle getCommitStyle() + @Override + public boolean equals(Object o) { - // TODO -- should be configurable - return new ImmediateCommitStyle(); - } - - /** - * Returns number of tasks in this group. - */ - public int size() { - return taskMap.size(); - } - - /** - * Adds a task to this group. - * @param task task to add - * @return true iff this group did not already contain the task - */ - public boolean add(final Task task) { - Preconditions.checkArgument( - task.getGroupId().equals(groupId), - "Task group id[%s] != TaskGroup group id[%s]", - task.getGroupId(), - groupId - ); - - if(taskMap.containsKey(task.getId())) { + if (!(o instanceof TaskLock)) { return false; } else { - taskMap.put(task.getId(), task); - return true; + final TaskLock x = (TaskLock) o; + return Objects.equal(this.groupId, x.groupId) && + Objects.equal(this.dataSource, x.dataSource) && + Objects.equal(this.interval, x.interval) && + Objects.equal(this.version, x.version); } } - /** - * Returns true if this group contains a particular task. - */ - public boolean contains(final String taskId) { - return taskMap.containsKey(taskId); - } - - /** - * Removes a task from this group. - * @param taskId task ID to remove - * @return the removed task, or null if the task was not in this group - */ - public Task remove(final String taskId) + @Override + public int hashCode() { - return taskMap.remove(taskId); + return Objects.hashCode(groupId, dataSource, interval, version); } @Override @@ -127,7 +100,6 @@ public class TaskGroup .add("dataSource", dataSource) .add("interval", interval) .add("version", version) - .add("tasks", taskMap.keySet()) .toString(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java index fe79d4f2e5e..90a0b9a3be4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java @@ -35,7 +35,7 @@ import java.util.Set; /** * Represents the status of a task. The task may be ongoing ({@link #isComplete()} false) or it may be * complete ({@link #isComplete()} true). - * + *

    * TaskStatus objects are immutable. */ public class TaskStatus @@ -49,49 +49,20 @@ public class TaskStatus public static TaskStatus running(String taskId) { - return new TaskStatus( - taskId, - Status.RUNNING, - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableList.of(), - -1 - ); + return new TaskStatus(taskId, Status.RUNNING, -1); } public static TaskStatus success(String taskId) { - return success(taskId, ImmutableSet.of()); - } - - public static TaskStatus success(String taskId, Set segments) - { - return new TaskStatus( - taskId, - Status.SUCCESS, - segments, - ImmutableSet.of(), - ImmutableList.of(), - -1 - ); + return new TaskStatus(taskId, Status.SUCCESS, -1); } public static TaskStatus failure(String taskId) { - return new TaskStatus( - taskId, - Status.FAILED, - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableList.of(), - -1 - ); + return new TaskStatus(taskId, Status.FAILED, -1); } private final String id; - private final ImmutableSet segments; - private final ImmutableSet segmentsNuked; - private final ImmutableList nextTasks; private final Status status; private final long duration; @@ -99,42 +70,16 @@ public class TaskStatus private TaskStatus( @JsonProperty("id") String id, @JsonProperty("status") Status status, - @JsonProperty("segments") Set segments, - @JsonProperty("segmentsNuked") Set segmentsNuked, - @JsonProperty("nextTasks") List nextTasks, @JsonProperty("duration") long duration ) { this.id = id; - this.segments = ImmutableSet.copyOf(segments); - this.segmentsNuked = ImmutableSet.copyOf(segmentsNuked); - this.nextTasks = ImmutableList.copyOf(nextTasks); this.status = status; this.duration = duration; // Check class invariants. Preconditions.checkNotNull(id, "id"); Preconditions.checkNotNull(status, "status"); - - if (this.segments.size() > 0) { - Preconditions.checkArgument( - status == Status.RUNNING || status == Status.SUCCESS, - "segments not allowed for %s tasks", - status - ); - } - - if (this.segmentsNuked.size() > 0) { - Preconditions.checkArgument(status == Status.SUCCESS, "segmentsNuked not allowed for %s tasks", status); - } - - if (this.nextTasks.size() > 0) { - Preconditions.checkArgument( - status == Status.SUCCESS || status == Status.RUNNING, - "nextTasks not allowed for %s tasks", - status - ); - } } @JsonProperty("id") @@ -149,24 +94,6 @@ public class TaskStatus return status; } - @JsonProperty("segments") - public Set getSegments() - { - return segments; - } - - @JsonProperty("segmentsNuked") - public Set getSegmentsNuked() - { - return segmentsNuked; - } - - @JsonProperty("nextTasks") - public List getNextTasks() - { - return nextTasks; - } - @JsonProperty("duration") public long getDuration() { @@ -212,25 +139,9 @@ public class TaskStatus return status == Status.FAILED; } - public TaskStatus withSegments(Set _segments) - { - return new TaskStatus(id, status, _segments, segmentsNuked, nextTasks, duration); - } - - - public TaskStatus withSegmentsNuked(Set _segmentsNuked) - { - return new TaskStatus(id, status, segments, _segmentsNuked, nextTasks, duration); - } - - public TaskStatus withNextTasks(List _nextTasks) - { - return new TaskStatus(id, status, segments, segmentsNuked, _nextTasks, duration); - } - public TaskStatus withDuration(long _duration) { - return new TaskStatus(id, status, segments, segmentsNuked, nextTasks, _duration); + return new TaskStatus(id, status, _duration); } @Override @@ -238,8 +149,6 @@ public class TaskStatus { return Objects.toStringHelper(this) .add("id", id) - .add("segments", segments) - .add("nextTasks", nextTasks) .add("status", status) .add("duration", duration) .toString(); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index e01dcb0c020..0b5ac1d6366 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -20,15 +20,15 @@ package com.metamx.druid.merger.common; import com.google.common.collect.ImmutableMap; -import com.metamx.druid.loading.S3SegmentPuller; import com.metamx.druid.loading.S3SegmentGetterConfig; +import com.metamx.druid.loading.S3SegmentPuller; import com.metamx.druid.loading.S3ZippedSegmentPuller; import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.loading.SegmentPuller; +import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.merger.common.actions.TaskActionClient; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.emitter.service.ServiceEmitter; import org.codehaus.jackson.map.ObjectMapper; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -42,6 +42,7 @@ import java.util.Map; public class TaskToolbox { private final TaskConfig config; + private final TaskActionClient taskActionClient; private final ServiceEmitter emitter; private final RestS3Service s3Client; private final SegmentPusher segmentPusher; @@ -50,6 +51,7 @@ public class TaskToolbox public TaskToolbox( TaskConfig config, + TaskActionClient taskActionClient, ServiceEmitter emitter, RestS3Service s3Client, SegmentPusher segmentPusher, @@ -58,6 +60,7 @@ public class TaskToolbox ) { this.config = config; + this.taskActionClient = taskActionClient; this.emitter = emitter; this.s3Client = s3Client; this.segmentPusher = segmentPusher; @@ -70,16 +73,16 @@ public class TaskToolbox return config; } + public TaskActionClient getTaskActionClient() + { + return taskActionClient; + } + public ServiceEmitter getEmitter() { return emitter; } - public RestS3Service getS3Client() - { - return s3Client; - } - public SegmentPusher getSegmentPusher() { return segmentPusher; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java new file mode 100644 index 00000000000..f6740064f52 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java @@ -0,0 +1,37 @@ +package com.metamx.druid.merger.common.actions; + +import com.metamx.druid.merger.coordinator.TaskStorage; +import com.metamx.emitter.EmittingLogger; + +public class LocalTaskActionClient implements TaskActionClient +{ + private final TaskStorage storage; + private final TaskActionToolbox toolbox; + + private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class); + + public LocalTaskActionClient(TaskStorage storage, TaskActionToolbox toolbox) + { + this.storage = storage; + this.toolbox = toolbox; + } + + @Override + public RetType submit(TaskAction taskAction) + { + final RetType ret = taskAction.perform(toolbox); + + // Add audit log + try { + storage.addAuditLog(taskAction); + } + catch (Exception e) { + log.makeAlert(e, "Failed to record action in audit log") + .addData("task", taskAction.getTask().getId()) + .addData("actionClass", taskAction.getClass().getName()) + .emit(); + } + + return ret; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java new file mode 100644 index 00000000000..f2d558ff1b7 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java @@ -0,0 +1,53 @@ +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 org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.type.TypeReference; +import org.joda.time.Interval; + +public class LockAcquireAction implements TaskAction> +{ + private final Task task; + private final Interval interval; + + @JsonCreator + public LockAcquireAction( + @JsonProperty("task") Task task, + @JsonProperty("interval") Interval interval + ) + { + this.task = task; + this.interval = interval; + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + public TypeReference> getReturnTypeReference() + { + return new TypeReference>() {}; + } + + @Override + public Optional perform(TaskActionToolbox toolbox) + { + try { + return toolbox.getTaskLockbox().tryLock(task, interval); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java new file mode 100644 index 00000000000..69454f358e8 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java @@ -0,0 +1,45 @@ +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 org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.type.TypeReference; + +import java.util.List; + +public class LockListAction implements TaskAction> +{ + private final Task task; + + @JsonCreator + public LockListAction( + @JsonProperty("task") Task task + ) + { + this.task = task; + } + + @JsonProperty + public Task getTask() + { + return task; + } + + public TypeReference> getReturnTypeReference() + { + return new TypeReference>() {}; + } + + @Override + public List perform(TaskActionToolbox toolbox) + { + try { + return toolbox.getTaskLockbox().findLocksForTask(task); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java new file mode 100644 index 00000000000..afbcb84cc6f --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java @@ -0,0 +1,55 @@ +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 org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.type.TypeReference; +import org.joda.time.Interval; + +import java.util.List; + +public class LockReleaseAction implements TaskAction +{ + private final Task task; + private final Interval interval; + + @JsonCreator + public LockReleaseAction( + @JsonProperty("task") Task task, + @JsonProperty("interval") Interval interval + ) + { + this.task = task; + this.interval = interval; + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + public TypeReference getReturnTypeReference() + { + return new TypeReference() {}; + } + + @Override + public Void perform(TaskActionToolbox toolbox) + { + try { + toolbox.getTaskLockbox().unlock(task, interval); + return null; + } catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java new file mode 100644 index 00000000000..670e07759a5 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java @@ -0,0 +1,53 @@ +package com.metamx.druid.merger.common.actions; + +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.ToStringResponseHandler; +import org.codehaus.jackson.map.ObjectMapper; + +import java.net.URI; +import java.net.URISyntaxException; + +public class RemoteTaskActionClient implements TaskActionClient +{ + private final HttpClient httpClient; + private final ObjectMapper jsonMapper; + + private static final Logger log = new Logger(RemoteTaskActionClient.class); + + public RemoteTaskActionClient(HttpClient httpClient, ObjectMapper jsonMapper) + { + this.httpClient = httpClient; + this.jsonMapper = jsonMapper; + } + + @Override + public RetType submit(TaskAction taskAction) + { + try { + byte[] dataToSend = jsonMapper.writeValueAsBytes(taskAction); + + final String response = httpClient.post(getServiceUri().toURL()) + .setContent("application/json", dataToSend) + .go(new ToStringResponseHandler(Charsets.UTF_8)) + .get(); + + // TODO Figure out how to check HTTP status code + if(response.equals("")) { + return null; + } else { + return jsonMapper.readValue(response, taskAction.getReturnTypeReference()); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public URI getServiceUri() throws URISyntaxException + { + return new URI("http://localhost:8087/mmx/merger/v1/action"); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java new file mode 100644 index 00000000000..8f11014650b --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java @@ -0,0 +1,95 @@ +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 org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.type.TypeReference; + +import java.util.List; +import java.util.Set; + +public class SegmentInsertAction implements TaskAction +{ + private final Task task; + private final Set segments; + + @JsonCreator + public SegmentInsertAction( + @JsonProperty("task") Task task, + @JsonProperty("segments") Set segments + ) + { + this.task = task; + this.segments = ImmutableSet.copyOf(segments); + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public Set getSegments() + { + return segments; + } + + public TypeReference getReturnTypeReference() + { + return new TypeReference() {}; + } + + @Override + public Void perform(TaskActionToolbox toolbox) + { + // Verify that each of these segments-to-insert falls under some lock + // TODO: Should this be done while holding the giant lock on TaskLockbox? (To prevent anyone from grabbing + // TODO: these locks out from under us while the operation is ongoing.) Probably not necessary. + final List taskLocks = toolbox.getTaskLockbox().findLocksForTask(task); + for(final DataSegment segment : segments) { + final boolean ok = Iterables.any( + taskLocks, new Predicate() + { + @Override + public boolean apply(TaskLock taskLock) + { + return taskLock.getVersion().equals(segment.getVersion()) + && taskLock.getDataSource().equals(segment.getDataSource()) + && taskLock.getInterval().contains(segment.getInterval()); + } + } + ); + + if(!ok) { + throw new ISE("No currently-held lock covers segment: %s", segment); + } + } + + try { + toolbox.getMergerDBCoordinator().announceHistoricalSegments(segments); + + // 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); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java new file mode 100644 index 00000000000..81b97de7ca7 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java @@ -0,0 +1,63 @@ +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 org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.type.TypeReference; +import org.joda.time.Interval; + +import java.util.List; + +public class SegmentListUnusedAction implements TaskAction> +{ + private final Task task; + private final String dataSource; + private final Interval interval; + + @JsonCreator + public SegmentListUnusedAction( + @JsonProperty("task") Task task, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval + ) + { + this.task = task; + this.dataSource = dataSource; + this.interval = interval; + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + public TypeReference> getReturnTypeReference() + { + return new TypeReference>() {}; + } + + @Override + public List perform(TaskActionToolbox toolbox) + { + try { + return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java new file mode 100644 index 00000000000..e1fa72ccf76 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java @@ -0,0 +1,63 @@ +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 org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.type.TypeReference; +import org.joda.time.Interval; + +import java.util.List; + +public class SegmentListUsedAction implements TaskAction> +{ + private final Task task; + private final String dataSource; + private final Interval interval; + + @JsonCreator + public SegmentListUsedAction( + @JsonProperty("task") Task task, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval + ) + { + this.task = task; + this.dataSource = dataSource; + this.interval = interval; + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + public TypeReference> getReturnTypeReference() + { + return new TypeReference>() {}; + } + + @Override + public List perform(TaskActionToolbox toolbox) + { + try { + return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java new file mode 100644 index 00000000000..53f764118d4 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java @@ -0,0 +1,95 @@ +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 org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.type.TypeReference; + +import java.util.List; +import java.util.Set; + +public class SegmentNukeAction implements TaskAction +{ + private final Task task; + private final Set segments; + + @JsonCreator + public SegmentNukeAction( + @JsonProperty("task") Task task, + @JsonProperty("segments") Set segments + ) + { + this.task = task; + this.segments = ImmutableSet.copyOf(segments); + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public Set getSegments() + { + return segments; + } + + public TypeReference getReturnTypeReference() + { + return new TypeReference() {}; + } + + @Override + public Void perform(TaskActionToolbox toolbox) + { + // Verify that each of these segments-to-nuke falls under some lock + // TODO: Should this be done while holding the giant lock on TaskLockbox? (To prevent anyone from grabbing + // TODO: these locks out from under us while the operation is ongoing.) Probably not necessary. + final List taskLocks = toolbox.getTaskLockbox().findLocksForTask(task); + for(final DataSegment segment : segments) { + final boolean ok = Iterables.any( + taskLocks, new Predicate() + { + @Override + public boolean apply(TaskLock taskLock) + { + return taskLock.getVersion().compareTo(segment.getVersion()) >= 0 + && taskLock.getDataSource().equals(segment.getDataSource()) + && taskLock.getInterval().contains(segment.getInterval()); + } + } + ); + + if(!ok) { + throw new ISE("No currently-held lock covers segment: %s", segment); + } + } + + try { + toolbox.getMergerDBCoordinator().deleteSegments(segments); + + // 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); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java new file mode 100644 index 00000000000..66f9b83fcb6 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java @@ -0,0 +1,57 @@ +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 org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.type.TypeReference; + +import java.util.List; + +public class SpawnTasksAction implements TaskAction +{ + private final Task task; + private final List newTasks; + + @JsonCreator + public SpawnTasksAction( + @JsonProperty("task") Task task, + @JsonProperty("newTasks") List newTasks + ) + { + this.task = task; + this.newTasks = ImmutableList.copyOf(newTasks); + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public List getNewTasks() + { + return newTasks; + } + + public TypeReference getReturnTypeReference() + { + return new TypeReference() {}; + } + + @Override + public Void perform(TaskActionToolbox toolbox) + { + try { + for(final Task newTask : newTasks) { + toolbox.getTaskQueue().add(newTask); + } + + return null; + } catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java new file mode 100644 index 00000000000..532771fe4db --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java @@ -0,0 +1,24 @@ +package com.metamx.druid.merger.common.actions; + +import com.metamx.druid.merger.common.task.Task; +import org.codehaus.jackson.annotate.JsonSubTypes; +import org.codehaus.jackson.annotate.JsonTypeInfo; +import org.codehaus.jackson.type.TypeReference; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "lockAcquire", value = LockAcquireAction.class), + @JsonSubTypes.Type(name = "lockList", value = LockListAction.class), + @JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class), + @JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class), + @JsonSubTypes.Type(name = "segmentListUsed", value = SegmentListUsedAction.class), + @JsonSubTypes.Type(name = "segmentListUnused", value = SegmentListUnusedAction.class), + @JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class), + @JsonSubTypes.Type(name = "spawnTasks", value = SpawnTasksAction.class) +}) +public interface TaskAction +{ + public Task getTask(); // TODO Look into replacing this with task ID so stuff serializes smaller + public TypeReference getReturnTypeReference(); // T_T + public RetType perform(TaskActionToolbox toolbox); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClient.java new file mode 100644 index 00000000000..7baa08fe788 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClient.java @@ -0,0 +1,6 @@ +package com.metamx.druid.merger.common.actions; + +public interface TaskActionClient +{ + public RetType submit(TaskAction taskAction); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java new file mode 100644 index 00000000000..30f87a1f6c3 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java @@ -0,0 +1,47 @@ +package com.metamx.druid.merger.common.actions; + +import com.metamx.druid.merger.coordinator.MergerDBCoordinator; +import com.metamx.druid.merger.coordinator.TaskLockbox; +import com.metamx.druid.merger.coordinator.TaskQueue; +import com.metamx.emitter.service.ServiceEmitter; + +public class TaskActionToolbox +{ + private final TaskQueue taskQueue; + private final TaskLockbox taskLockbox; + private final MergerDBCoordinator mergerDBCoordinator; + private final ServiceEmitter emitter; + + public TaskActionToolbox( + TaskQueue taskQueue, + TaskLockbox taskLockbox, + MergerDBCoordinator mergerDBCoordinator, + ServiceEmitter emitter + ) + { + this.taskQueue = taskQueue; + this.taskLockbox = taskLockbox; + this.mergerDBCoordinator = mergerDBCoordinator; + this.emitter = emitter; + } + + public TaskQueue getTaskQueue() + { + return taskQueue; + } + + public TaskLockbox getTaskLockbox() + { + return taskLockbox; + } + + public MergerDBCoordinator getMergerDBCoordinator() + { + return mergerDBCoordinator; + } + + public ServiceEmitter getEmitter() + { + return emitter; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index 6de3eb0d73f..95d7094317e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -20,10 +20,10 @@ package com.metamx.druid.merger.common.task; import com.google.common.base.Objects; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.coordinator.TaskContext; -import org.codehaus.jackson.annotate.JsonCreator; +import com.metamx.druid.merger.common.TaskToolbox; import org.codehaus.jackson.annotate.JsonProperty; import org.joda.time.Interval; @@ -32,30 +32,19 @@ public abstract class AbstractTask implements Task private final String id; private final String groupId; private final String dataSource; - private final Interval interval; + private final Optional interval; - public AbstractTask(String id, String dataSource, Interval interval) + protected AbstractTask(String id, String dataSource, Interval interval) { this(id, id, dataSource, interval); } - @JsonCreator - public AbstractTask( - @JsonProperty("id") String id, - @JsonProperty("groupId") String groupId, - @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval - ) + protected AbstractTask(String id, String groupId, String dataSource, Interval interval) { - Preconditions.checkNotNull(id, "id"); - Preconditions.checkNotNull(groupId, "groupId"); - Preconditions.checkNotNull(dataSource, "dataSource"); - Preconditions.checkNotNull(interval, "interval"); - - this.id = id; - this.groupId = groupId; - this.dataSource = dataSource; - this.interval = interval; + this.id = Preconditions.checkNotNull(id, "id"); + this.groupId = Preconditions.checkNotNull(groupId, "groupId"); + this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); + this.interval = Optional.fromNullable(interval); } @JsonProperty @@ -79,15 +68,22 @@ public abstract class AbstractTask implements Task return dataSource; } - @JsonProperty @Override - public Interval getInterval() + public Optional getFixedInterval() { return interval; } + // Awesome hack to get around lack of serde for Optional + // TODO Look into jackson-datatype-guava + @JsonProperty("interval") + private Interval getNullableIntervalForJackson() + { + return interval.orNull(); + } + @Override - public TaskStatus preflight(TaskContext context) throws Exception + public TaskStatus preflight(TaskToolbox toolbox) throws Exception { return TaskStatus.running(id); } @@ -99,7 +95,7 @@ public abstract class AbstractTask implements Task .add("id", id) .add("type", getType()) .add("dataSource", dataSource) - .add("interval", getInterval()) + .add("interval", getFixedInterval()) .toString(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java index f1153e5c43c..0d947514cda 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java @@ -60,7 +60,7 @@ public class AppendTask extends MergeTask throws Exception { VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( - Ordering.natural().nullsFirst() + Ordering.natural().nullsFirst() ); for (DataSegment segment : segments.keySet()) { @@ -109,9 +109,9 @@ public class AppendTask extends MergeTask } @Override - public Type getType() + public String getType() { - return Task.Type.APPEND; + return "append"; } private class SegmentToMergeHolder diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java index e17db4b980e..8d9a593ab09 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java @@ -80,8 +80,8 @@ public class DefaultMergeTask extends MergeTask } @Override - public Type getType() + public String getType() { - return Task.Type.MERGE; + return "merge"; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index cf261e2af62..29a1da13266 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -19,7 +19,9 @@ package com.metamx.druid.merger.common.task; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import com.metamx.druid.QueryGranularity; @@ -29,10 +31,11 @@ import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IncrementalIndexAdapter; import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.index.v1.IndexableAdapter; -import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.coordinator.TaskContext; +import com.metamx.druid.merger.common.actions.LockListAction; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.shard.NoneShardSpec; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; @@ -60,29 +63,31 @@ public class DeleteTask extends AbstractTask new DateTime().toString() ), dataSource, - interval + Preconditions.checkNotNull(interval, "interval") ); } @Override - public Type getType() + public String getType() { - return Task.Type.DELETE; + return "delete"; } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { // Strategy: Create an empty segment covering the interval to be deleted + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final Interval interval = this.getFixedInterval().get(); final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); - final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(this.getInterval(), empty); + final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty); // Create DataSegment final DataSegment segment = DataSegment.builder() .dataSource(this.getDataSource()) - .interval(this.getInterval()) - .version(context.getVersion()) + .interval(interval) + .version(myLock.getVersion()) .shardSpec(new NoneShardSpec()) .build(); @@ -99,6 +104,8 @@ public class DeleteTask extends AbstractTask segment.getVersion() ); - return TaskStatus.success(getId(), ImmutableSet.of(uploadedSegment)); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment))); + + return TaskStatus.success(getId()); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 3255fb61277..50b073665d2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -20,6 +20,7 @@ package com.metamx.druid.merger.common.task; import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -28,10 +29,9 @@ import com.google.common.collect.TreeMultiset; import com.google.common.primitives.Ints; import com.metamx.common.logger.Logger; import com.metamx.druid.input.InputRow; -import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.coordinator.TaskContext; +import com.metamx.druid.merger.common.actions.SpawnTasksAction; import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; import com.metamx.druid.realtime.Schema; @@ -75,7 +75,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask ), groupId, schema.getDataSource(), - interval + Preconditions.checkNotNull(interval, "interval") ); this.firehoseFactory = firehoseFactory; @@ -84,21 +84,20 @@ public class IndexDeterminePartitionsTask extends AbstractTask } @Override - public Type getType() + public String getType() { - return Type.INDEX; + return "index_partitions"; } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { log.info("Running with targetPartitionSize[%d]", targetPartitionSize); - // This is similar to what DeterminePartitionsJob does in the hadoop indexer, but we don't require - // a preconfigured partition dimension (we'll just pick the one with highest cardinality). + // TODO: Replace/merge/whatever with hadoop determine-partitions code - // NOTE: Space-efficiency (stores all unique dimension values, although at least not all combinations) - // NOTE: Time-efficiency (runs all this on one single node instead of through map/reduce) + // We know this exists + final Interval interval = getFixedInterval().get(); // Blacklist dimensions that have multiple values per row final Set unusableDimensions = Sets.newHashSet(); @@ -114,7 +113,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask final InputRow inputRow = firehose.nextRow(); - if (getInterval().contains(inputRow.getTimestampFromEpoch())) { + if (interval.contains(inputRow.getTimestampFromEpoch())) { // Extract dimensions from event for (final String dim : inputRow.getDimensions()) { @@ -229,28 +228,30 @@ public class IndexDeterminePartitionsTask extends AbstractTask } } - return TaskStatus.success(getId()).withNextTasks( - Lists.transform( - shardSpecs, - new Function() - { - @Override - public Task apply(ShardSpec shardSpec) - { - return new IndexGeneratorTask( - getGroupId(), - getInterval(), - firehoseFactory, - new Schema( - schema.getDataSource(), - schema.getAggregators(), - schema.getIndexGranularity(), - shardSpec - ) - ); - } - } - ) + List nextTasks = Lists.transform( + shardSpecs, + new Function() + { + @Override + public Task apply(ShardSpec shardSpec) + { + return new IndexGeneratorTask( + getGroupId(), + getFixedInterval().get(), + firehoseFactory, + new Schema( + schema.getDataSource(), + schema.getAggregators(), + schema.getIndexGranularity(), + shardSpec + ) + ); + } + } ); + + toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks)); + + return TaskStatus.success(getId()); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index 9b70cbf5952..c9ce684edea 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -19,24 +19,25 @@ package com.metamx.druid.merger.common.task; -import com.google.common.collect.ImmutableList; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.input.InputRow; -import com.metamx.druid.merger.common.TaskCallback; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.LockListAction; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; import com.metamx.druid.merger.common.index.YeOldePlumberSchool; -import com.metamx.druid.merger.coordinator.TaskContext; +import com.metamx.druid.merger.common.TaskLock; 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.Schema; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.realtime.Sink; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; @@ -77,7 +78,7 @@ public class IndexGeneratorTask extends AbstractTask ), groupId, schema.getDataSource(), - interval + Preconditions.checkNotNull(interval, "interval") ); this.firehoseFactory = firehoseFactory; @@ -85,14 +86,20 @@ public class IndexGeneratorTask extends AbstractTask } @Override - public Type getType() + public String getType() { - return Type.INDEX; + return "index_generator"; } @Override - public TaskStatus run(final TaskContext context, final TaskToolbox toolbox, TaskCallback callback) throws Exception + public TaskStatus run(final TaskToolbox toolbox) throws Exception { + // We should have a lock from before we started running + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + + // We know this exists + final Interval interval = getFixedInterval().get(); + // Set up temporary directory for indexing final File tmpDir = new File( String.format( @@ -101,9 +108,9 @@ public class IndexGeneratorTask extends AbstractTask String.format( "%s_%s_%s_%s_%s", this.getDataSource(), - this.getInterval().getStart(), - this.getInterval().getEnd(), - context.getVersion(), + interval.getStart(), + interval.getEnd(), + myLock.getVersion(), schema.getShardSpec().getPartitionNum() ) ) @@ -126,8 +133,8 @@ public class IndexGeneratorTask extends AbstractTask final FireDepartmentMetrics metrics = new FireDepartmentMetrics(); final Firehose firehose = firehoseFactory.connect(); final Plumber plumber = new YeOldePlumberSchool( - getInterval(), - context.getVersion(), + interval, + myLock.getVersion(), wrappedSegmentPusher, tmpDir ).findPlumber(schema, metrics); @@ -175,8 +182,11 @@ public class IndexGeneratorTask extends AbstractTask metrics.rowOutput() ); + // Request segment pushes + toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(pushedSegments))); + // Done - return TaskStatus.success(getId(), ImmutableSet.copyOf(pushedSegments)); + return TaskStatus.success(getId()); } /** @@ -185,7 +195,7 @@ public class IndexGeneratorTask extends AbstractTask * @return true or false */ private boolean shouldIndex(InputRow inputRow) { - if(!getInterval().contains(inputRow.getTimestampFromEpoch())) { + if(!getFixedInterval().get().contains(inputRow.getTimestampFromEpoch())) { return false; } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 4d5e5624ccb..b16b9d9acbc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -25,10 +25,9 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.indexer.granularity.GranularitySpec; -import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.coordinator.TaskContext; +import com.metamx.druid.merger.common.actions.SpawnTasksAction; import com.metamx.druid.realtime.FirehoseFactory; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; @@ -119,19 +118,20 @@ public class IndexTask extends AbstractTask } @Override - public Type getType() + public String getType() { - return Type.INDEX; + return "index"; } @Override - public TaskStatus preflight(TaskContext context) throws Exception + public TaskStatus preflight(TaskToolbox toolbox) throws Exception { - return TaskStatus.success(getId()).withNextTasks(toSubtasks()); + toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, toSubtasks())); + return TaskStatus.success(getId()); } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { throw new IllegalStateException("IndexTasks should not be run!"); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index eb5992925f0..05a46e2a62c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -1,20 +1,22 @@ package com.metamx.druid.merger.common.task; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; +import com.google.common.collect.Iterables; +import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.coordinator.TaskContext; +import com.metamx.druid.merger.common.actions.LockListAction; +import com.metamx.druid.merger.common.actions.SegmentListUnusedAction; +import com.metamx.druid.merger.common.actions.SegmentNukeAction; +import com.metamx.druid.merger.common.TaskLock; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.List; -import java.util.Set; /** */ @@ -42,18 +44,55 @@ public class KillTask extends AbstractTask } @Override - public Type getType() + public String getType() { - return Task.Type.KILL; + return "kill"; } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { - // Kill segments - toolbox.getSegmentKiller() - .kill(context.getUnusedSegments()); + // Confirm we have a lock (will throw if there isn't exactly one element) + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); - return TaskStatus.success(getId()).withSegmentsNuked(context.getUnusedSegments()); + if(!myLock.getDataSource().equals(getDataSource())) { + throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); + } + + if(!myLock.getInterval().equals(getFixedInterval().get())) { + throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getFixedInterval().get()); + } + + // List unused segments + final List unusedSegments = toolbox.getTaskActionClient() + .submit( + new SegmentListUnusedAction( + this, + myLock.getDataSource(), + myLock.getInterval() + ) + ); + + // Verify none of these segments have versions > lock version + for(final DataSegment unusedSegment : unusedSegments) { + if(unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) { + throw new ISE( + "WTF?! Unused segment[%s] has version[%s] > task version[%s]", + unusedSegment.getIdentifier(), + unusedSegment.getVersion(), + myLock.getVersion() + ); + } + + log.info("OK to kill segment: %s", unusedSegment.getIdentifier()); + } + + // Kill segments + toolbox.getSegmentKiller().kill(unusedSegments); + + // Remove metadata for these segments + toolbox.getTaskActionClient().submit(new SegmentNukeAction(this, ImmutableSet.copyOf(unusedSegments))); + + return TaskStatus.success(getId()); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 34419009c88..ce60352423e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -35,10 +35,12 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.SegmentPuller; -import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.coordinator.TaskContext; +import com.metamx.druid.merger.common.actions.LockListAction; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import com.metamx.druid.merger.common.actions.SegmentListUsedAction; +import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; @@ -116,11 +118,12 @@ public abstract class MergeTask extends AbstractTask } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); final ServiceEmitter emitter = toolbox.getEmitter(); final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); - final DataSegment mergedSegment = computeMergedSegment(getDataSource(), context.getVersion(), segments); + final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments); final File taskDir = toolbox.getConfig().getTaskDir(this); try { @@ -172,7 +175,9 @@ public abstract class MergeTask extends AbstractTask emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - return TaskStatus.success(getId(), ImmutableSet.of(uploadedSegment)); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment))); + + return TaskStatus.success(getId()); } catch (Exception e) { log.error( @@ -201,7 +206,7 @@ public abstract class MergeTask extends AbstractTask * we are operating on every segment that overlaps the chosen interval. */ @Override - public TaskStatus preflight(TaskContext context) + public TaskStatus preflight(TaskToolbox toolbox) { final Function toIdentifier = new Function() { @@ -212,7 +217,13 @@ public abstract class MergeTask extends AbstractTask } }; - final Set current = ImmutableSet.copyOf(Iterables.transform(context.getCurrentSegments(), toIdentifier)); + final Set current = ImmutableSet.copyOf( + Iterables.transform( + toolbox.getTaskActionClient() + .submit(new SegmentListUsedAction(this, getDataSource(), getFixedInterval().get())), + toIdentifier + ) + ); final Set requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier)); final Set missingFromRequested = Sets.difference(current, requested); @@ -250,7 +261,7 @@ public abstract class MergeTask extends AbstractTask return Objects.toStringHelper(this) .add("id", getId()) .add("dataSource", getDataSource()) - .add("interval", getInterval()) + .add("interval", getFixedInterval()) .add("segments", segments) .toString(); } @@ -276,7 +287,7 @@ public abstract class MergeTask extends AbstractTask ) ); - return String.format("%s_%s", dataSource, DigestUtils.shaHex(segmentIDs).toLowerCase()); + return String.format("%s_%s", dataSource, DigestUtils.sha1Hex(segmentIDs).toLowerCase()); } private static Interval computeMergedInterval(final List segments) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 57984c09dc4..695fa99977e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -19,10 +19,10 @@ package com.metamx.druid.merger.common.task; +import com.google.common.base.Optional; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.TaskCallback; -import com.metamx.druid.merger.coordinator.TaskContext; import org.codehaus.jackson.annotate.JsonSubTypes; import org.codehaus.jackson.annotate.JsonTypeInfo; import org.joda.time.Interval; @@ -33,6 +33,7 @@ import org.joda.time.Interval; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "append", value = AppendTask.class), + @JsonSubTypes.Type(name = "merge", value = DefaultMergeTask.class), @JsonSubTypes.Type(name = "delete", value = DeleteTask.class), @JsonSubTypes.Type(name = "kill", value = KillTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), @@ -41,49 +42,53 @@ import org.joda.time.Interval; }) public interface Task { - enum Type - { - INDEX, - MERGE, - APPEND, - DELETE, - TEST, - KILL - } - + /** + * Returns ID of this task. Must be unique across all tasks ever created. + */ public String getId(); + /** + * Returns group ID of this task. Tasks with the same group ID can share locks. If tasks do not need to share locks, + * a common convention is to set group ID equal to task ID. + */ public String getGroupId(); - public Type getType(); + /** + * Returns a descriptive label for this task type. Used for metrics emission and logging. + */ + public String getType(); + /** + * Returns the datasource this task operates on. Each task can operate on only one datasource. + */ public String getDataSource(); - public Interval getInterval(); + /** + * Returns fixed interval for this task, if any. Tasks without fixed intervals are not granted locks when started + * and must explicitly request them. + */ + public Optional getFixedInterval(); /** * Execute preflight checks for a task. This typically runs on the coordinator, and will be run while * holding a lock on our dataSouce and interval. If this method throws an exception, the task should be * considered a failure. * - * @param context Context for this task, gathered under indexer lock + * @param toolbox Toolbox for this task * @return Some kind of status (runnable means continue on to a worker, non-runnable means we completed without * using a worker). * @throws Exception */ - public TaskStatus preflight(TaskContext context) throws Exception; + public TaskStatus preflight(TaskToolbox toolbox) throws Exception; /** * Execute a task. This typically runs on a worker as determined by a TaskRunner, and will be run while * holding a lock on our dataSource and interval. If this method throws an exception, the task should be * considered a failure. * - * @param context Context for this task, gathered under indexer lock * @param toolbox Toolbox for this task - * @param callback Callback for "early returns". Statuses returned to this callback must not be - * complete (isRunnable must be true). * @return Some kind of finished status (isRunnable must be false). * @throws Exception */ - public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception; + public TaskStatus run(TaskToolbox toolbox) throws Exception; } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java index b63ff8fc8c7..55448d068e7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java @@ -23,10 +23,15 @@ import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.metamx.common.Pair; import com.metamx.common.logger.Logger; 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 com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; import org.codehaus.jackson.map.ObjectMapper; @@ -36,6 +41,7 @@ import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.exceptions.StatementException; import org.skife.jdbi.v2.tweak.HandleCallback; +import javax.annotation.Nullable; import java.util.List; import java.util.Map; @@ -103,18 +109,11 @@ public class DbTaskStorage implements TaskStorage } @Override - public void setStatus(final String taskid, final TaskStatus status) + public void setStatus(final TaskStatus status) { - Preconditions.checkNotNull(taskid, "task"); Preconditions.checkNotNull(status, "status"); - Preconditions.checkArgument( - taskid.equals(status.getId()), - "Task/Status ID mismatch[%s/%s]", - taskid, - status.getId() - ); - log.info("Updating task %s to status: %s", taskid, status); + log.info("Updating task %s to status: %s", status.getId(), status); int updated = dbi.withHandle( new HandleCallback() @@ -124,12 +123,13 @@ public class DbTaskStorage implements TaskStorage { return handle.createStatement( String.format( - "UPDATE %s SET status_code = :status_code, status_payload = :status_payload WHERE id = :id", + "UPDATE %s SET status_code = :status_code, status_payload = :status_payload WHERE id = :id AND status_code = :old_status_code", dbConnectorConfig.getTaskTable() ) ) - .bind("id", taskid) + .bind("id", status.getId()) .bind("status_code", status.getStatusCode().toString()) + .bind("old_status_code", TaskStatus.Status.RUNNING.toString()) .bind("status_payload", jsonMapper.writeValueAsString(status)) .execute(); } @@ -137,40 +137,10 @@ public class DbTaskStorage implements TaskStorage ); if(updated != 1) { - throw new IllegalStateException(String.format("Task not found: %s", taskid)); + throw new IllegalStateException(String.format("Running task not found: %s", status.getId())); } } - @Override - public void setVersion(final String taskid, final String version) - { - Preconditions.checkNotNull(taskid, "task"); - Preconditions.checkNotNull(version, "version"); - - log.info("Updating task %s to version: %s", taskid, version); - - dbi.withHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) throws Exception - { - handle.createStatement( - String.format( - "UPDATE %s SET version = :version WHERE id = :id", - dbConnectorConfig.getTaskTable() - ) - ) - .bind("id", taskid) - .bind("version", version) - .execute(); - - return null; - } - } - ); - } - @Override public Optional getTask(final String taskid) { @@ -231,36 +201,6 @@ public class DbTaskStorage implements TaskStorage ); } - @Override - public Optional getVersion(final String taskid) - { - return dbi.withHandle( - new HandleCallback>() - { - @Override - public Optional withHandle(Handle handle) throws Exception - { - final List> dbStatuses = - handle.createQuery( - String.format( - "SELECT version FROM %s WHERE id = :id", - dbConnectorConfig.getTaskTable() - ) - ) - .bind("id", taskid) - .list(); - - if(dbStatuses.size() == 0) { - return Optional.absent(); - } else { - final Map dbStatus = Iterables.getOnlyElement(dbStatuses); - return Optional.fromNullable((String) dbStatus.get("version")); - } - } - } - ); - } - @Override public List getRunningTasks() { @@ -298,4 +238,183 @@ public class DbTaskStorage implements TaskStorage } ); } + + @Override + public void addLock(final String taskid, final TaskLock taskLock) + { + Preconditions.checkNotNull(taskid, "taskid"); + Preconditions.checkNotNull(taskLock, "taskLock"); + + log.info( + "Adding lock on interval[%s] version[%s] for task: %s", + taskLock.getInterval(), + taskLock.getVersion(), + taskid + ); + + dbi.withHandle( + new HandleCallback() + { + @Override + public Integer withHandle(Handle handle) throws Exception + { + return handle.createStatement( + String.format( + "INSERT INTO %s (task_id, lock_payload) VALUES (:task_id, :lock_payload)", + dbConnectorConfig.getTaskLockTable() + ) + ) + .bind("task_id", taskid) + .bind("lock_payload", jsonMapper.writeValueAsString(taskLock)) + .execute(); + } + } + ); + } + + @Override + public void removeLock(String taskid, TaskLock taskLockToRemove) + { + Preconditions.checkNotNull(taskid, "taskid"); + Preconditions.checkNotNull(taskLockToRemove, "taskLockToRemove"); + + final Map taskLocks = getLocksWithIds(taskid); + + for(final Map.Entry taskLockWithId : taskLocks.entrySet()) { + final long id = taskLockWithId.getKey(); + final TaskLock taskLock = taskLockWithId.getValue(); + + if(taskLock.equals(taskLockToRemove)) { + log.info("Deleting TaskLock with id[%d]: %s", id, taskLock); + + dbi.withHandle( + new HandleCallback() + { + @Override + public Integer withHandle(Handle handle) throws Exception + { + return handle.createStatement( + String.format( + "DELETE FROM %s WHERE id = :id", + dbConnectorConfig.getTaskLockTable() + ) + ) + .bind("id", id) + .execute(); + } + } + ); + } + } + } + + @Override + public List getLocks(String taskid) + { + return ImmutableList.copyOf( + Iterables.transform( + getLocksWithIds(taskid).entrySet(), new Function, TaskLock>() + { + @Override + public TaskLock apply(Map.Entry e) + { + return e.getValue(); + } + } + ) + ); + } + + @Override + public void addAuditLog(final TaskAction taskAction) + { + Preconditions.checkNotNull(taskAction, "taskAction"); + + log.info("Logging action for task[%s]: %s", taskAction.getTask().getId(), taskAction); + + dbi.withHandle( + new HandleCallback() + { + @Override + public Integer withHandle(Handle handle) throws Exception + { + return handle.createStatement( + String.format( + "INSERT INTO %s (task_id, log_payload) VALUES (:task_id, :log_payload)", + dbConnectorConfig.getTaskLogTable() + ) + ) + .bind("task_id", taskAction.getTask().getId()) + .bind("log_payload", jsonMapper.writeValueAsString(taskAction)) + .execute(); + } + } + ); + } + + @Override + public List getAuditLogs(final String taskid) + { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) throws Exception + { + final List> dbTaskLogs = + handle.createQuery( + String.format( + "SELECT log_payload FROM %s WHERE task_id = :task_id", + dbConnectorConfig.getTaskLogTable() + ) + ) + .bind("task_id", taskid) + .list(); + + return Lists.transform( + dbTaskLogs, new Function, TaskAction>() + { + @Override + public TaskAction apply(Map row) + { + try { + return jsonMapper.readValue(row.get("payload").toString(), TaskAction.class); + } catch(Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + } + } + ); + } + + private Map getLocksWithIds(final String taskid) + { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public Map withHandle(Handle handle) throws Exception + { + final List> dbTaskLocks = + handle.createQuery( + String.format( + "SELECT id, lock_payload FROM %s WHERE task_id = :task_id", + dbConnectorConfig.getTaskLockTable() + ) + ) + .bind("task_id", taskid) + .list(); + + final Map retMap = Maps.newHashMap(); + for(final Map row : dbTaskLocks) { + retMap.put((Long)row.get("id"), jsonMapper.readValue(row.get("lock_payload").toString(), TaskLock.class)); + } + return retMap; + } + } + ); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index ddb07438f9a..8652b05c2a7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -57,7 +57,7 @@ public class LocalTaskRunner implements TaskRunner } @Override - public void run(final Task task, final TaskContext context, final TaskCallback callback) + public void run(final Task task, final TaskCallback callback) { exec.submit( new Runnable() @@ -71,7 +71,7 @@ public class LocalTaskRunner implements TaskRunner try { log.info("Running task: %s", task.getId()); - status = task.run(context, toolbox, callback); + status = task.run(toolbox); } catch (InterruptedException e) { log.error(e, "Interrupted while running task[%s]", task); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java index e169890f233..042f0b8196f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java @@ -21,14 +21,20 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; import com.metamx.common.logger.Logger; 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; import java.util.Map; +import java.util.concurrent.locks.ReentrantLock; /** * Implements an in-heap TaskStorage facility, with no persistence across restarts. This class is not @@ -36,108 +42,178 @@ import java.util.Map; */ public class LocalTaskStorage implements TaskStorage { + private final ReentrantLock giant = new ReentrantLock(); private final Map tasks = Maps.newHashMap(); + private final Multimap taskLocks = HashMultimap.create(); + private final Multimap taskActions = ArrayListMultimap.create(); private static final Logger log = new Logger(LocalTaskStorage.class); @Override public void insert(Task task, TaskStatus status) { - Preconditions.checkNotNull(task, "task"); - Preconditions.checkNotNull(status, "status"); - Preconditions.checkArgument( - task.getId().equals(status.getId()), - "Task/Status ID mismatch[%s/%s]", - task.getId(), - status.getId() - ); + giant.lock(); - if(tasks.containsKey(task.getId())) { - throw new TaskExistsException(task.getId()); + try { + Preconditions.checkNotNull(task, "task"); + Preconditions.checkNotNull(status, "status"); + Preconditions.checkArgument( + task.getId().equals(status.getId()), + "Task/Status ID mismatch[%s/%s]", + task.getId(), + status.getId() + ); + + if(tasks.containsKey(task.getId())) { + throw new TaskExistsException(task.getId()); + } + + log.info("Inserting task %s with status: %s", task.getId(), status); + tasks.put(task.getId(), new TaskStuff(task, status)); + } finally { + giant.unlock(); } - - log.info("Inserting task %s with status: %s", task.getId(), status); - tasks.put(task.getId(), new TaskStuff(task, status)); } @Override public Optional getTask(String taskid) { - Preconditions.checkNotNull(taskid, "taskid"); - if(tasks.containsKey(taskid)) { - return Optional.of(tasks.get(taskid).getTask()); - } else { - return Optional.absent(); + giant.lock(); + + try { + Preconditions.checkNotNull(taskid, "taskid"); + if(tasks.containsKey(taskid)) { + return Optional.of(tasks.get(taskid).getTask()); + } else { + return Optional.absent(); + } + } finally { + giant.unlock(); } } @Override - public void setStatus(String taskid, TaskStatus status) + public void setStatus(TaskStatus status) { - Preconditions.checkNotNull(taskid, "taskid"); - Preconditions.checkNotNull(status, "status"); - Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid); - log.info("Updating task %s to status: %s", taskid, status); - tasks.put(taskid, tasks.get(taskid).withStatus(status)); + giant.lock(); + + try { + Preconditions.checkNotNull(status, "status"); + + final String taskid = status.getId(); + Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid); + Preconditions.checkState(tasks.get(taskid).getStatus().isRunnable(), "Task status must be runnable: %s", taskid); + log.info("Updating task %s to status: %s", taskid, status); + tasks.put(taskid, tasks.get(taskid).withStatus(status)); + } finally { + giant.unlock(); + } } @Override public Optional getStatus(String taskid) { - Preconditions.checkNotNull(taskid, "taskid"); - if(tasks.containsKey(taskid)) { - return Optional.of(tasks.get(taskid).getStatus()); - } else { - return Optional.absent(); - } - } + giant.lock(); - @Override - public void setVersion(String taskid, String version) - { - Preconditions.checkNotNull(taskid, "taskid"); - Preconditions.checkNotNull(version, "status"); - Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid); - log.info("Updating task %s to version: %s", taskid, version); - tasks.put(taskid, tasks.get(taskid).withVersion(version)); - } - - @Override - public Optional getVersion(String taskid) - { - Preconditions.checkNotNull(taskid, "taskid"); - if(tasks.containsKey(taskid)) { - return tasks.get(taskid).getVersion(); - } else { - return Optional.absent(); + try { + Preconditions.checkNotNull(taskid, "taskid"); + if(tasks.containsKey(taskid)) { + return Optional.of(tasks.get(taskid).getStatus()); + } else { + return Optional.absent(); + } + } finally { + giant.unlock(); } } @Override public List getRunningTasks() { - final ImmutableList.Builder listBuilder = ImmutableList.builder(); - for(final TaskStuff taskStuff : tasks.values()) { - if(taskStuff.getStatus().isRunnable()) { - listBuilder.add(taskStuff.getTask()); - } - } + giant.lock(); - return listBuilder.build(); + try { + final ImmutableList.Builder listBuilder = ImmutableList.builder(); + for(final TaskStuff taskStuff : tasks.values()) { + if(taskStuff.getStatus().isRunnable()) { + listBuilder.add(taskStuff.getTask()); + } + } + + return listBuilder.build(); + } finally { + giant.unlock(); + } + } + + @Override + public void addLock(final String taskid, final TaskLock taskLock) + { + giant.lock(); + + try { + Preconditions.checkNotNull(taskLock, "taskLock"); + taskLocks.put(taskid, taskLock); + } finally { + giant.unlock(); + } + } + + @Override + public void removeLock(final String taskid, final TaskLock taskLock) + { + giant.lock(); + + try { + Preconditions.checkNotNull(taskLock, "taskLock"); + taskLocks.remove(taskid, taskLock); + } finally { + giant.unlock(); + } + } + + @Override + public List getLocks(final String taskid) + { + giant.lock(); + + try { + return ImmutableList.copyOf(taskLocks.get(taskid)); + } finally { + giant.unlock(); + } + } + + @Override + public void addAuditLog(TaskAction taskAction) + { + giant.lock(); + + try { + taskActions.put(taskAction.getTask().getId(), taskAction); + } finally { + giant.unlock(); + } + } + + @Override + public List getAuditLogs(String taskid) + { + giant.lock(); + + try { + return ImmutableList.copyOf(taskActions.get(taskid)); + } finally { + giant.unlock(); + } } private static class TaskStuff { final Task task; final TaskStatus status; - final Optional version; private TaskStuff(Task task, TaskStatus status) - { - this(task, status, Optional.absent()); - } - - private TaskStuff(Task task, TaskStatus status, Optional version) { Preconditions.checkNotNull(task); Preconditions.checkNotNull(status); @@ -145,7 +221,6 @@ public class LocalTaskStorage implements TaskStorage this.task = task; this.status = status; - this.version = version; } public Task getTask() @@ -158,19 +233,9 @@ public class LocalTaskStorage implements TaskStorage return status; } - public Optional getVersion() - { - return version; - } - private TaskStuff withStatus(TaskStatus _status) { - return new TaskStuff(task, _status, version); - } - - private TaskStuff withVersion(String _version) - { - return new TaskStuff(task, status, Optional.of(_version)); + return new TaskStuff(task, _status); } } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java index c485d38aa87..cc54a26d7ab 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java @@ -47,6 +47,7 @@ import java.io.IOException; import java.sql.SQLException; import java.util.List; import java.util.Map; +import java.util.Set; /** */ @@ -130,52 +131,25 @@ public class MergerDBCoordinator return segments; } - public void commitTaskStatus(final TaskStatus taskStatus) + public void announceHistoricalSegments(final Set segments) throws Exception { - try { - dbi.inTransaction( - new TransactionCallback() - { - @Override - public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception - { - for(final DataSegment segment : taskStatus.getSegments()) - { - log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), taskStatus.getId()); - announceHistoricalSegment(handle, segment); - } - - for(final DataSegment segment : taskStatus.getSegmentsNuked()) - { - log.info("Deleting segment[%s] for task[%s]", segment.getIdentifier(), taskStatus.getId()); - deleteSegment(handle, segment); - } - - return null; - } - } - ); - } - catch (Exception e) { - throw new RuntimeException(String.format("Exception commit task to DB: %s", taskStatus.getId()), e); - } - } - - public void announceHistoricalSegment(final DataSegment segment) throws Exception - { - dbi.withHandle( - new HandleCallback() + dbi.inTransaction( + new TransactionCallback() { @Override - public Void withHandle(Handle handle) throws Exception + public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception { - announceHistoricalSegment(handle, segment); + for(final DataSegment segment : segments) { + announceHistoricalSegment(handle, segment); + } + return null; } } ); } + private void announceHistoricalSegment(final Handle handle, final DataSegment segment) throws Exception { try { @@ -219,15 +193,18 @@ public class MergerDBCoordinator } } - public void deleteSegment(final DataSegment segment) + public void deleteSegments(final Set segments) throws Exception { - dbi.withHandle( - new HandleCallback() + dbi.inTransaction( + new TransactionCallback() { @Override - public Void withHandle(Handle handle) throws Exception + public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception { - deleteSegment(handle, segment); + for(final DataSegment segment : segments) { + deleteSegment(handle, segment); + } + return null; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 32acc66ae43..42e013ab7f7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -34,7 +34,6 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.PeriodGranularity; import com.metamx.druid.merger.common.TaskCallback; -import com.metamx.druid.merger.common.TaskHolder; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; @@ -276,14 +275,12 @@ public class RemoteTaskRunner implements TaskRunner } @Override - public void run(Task task, TaskContext context, TaskCallback callback) + public void run(Task task, TaskCallback callback) { if (tasks.containsKey(task.getId())) { throw new ISE("Assigned a task[%s] that already exists, WTF is happening?!", task.getId()); } - TaskWrapper taskWrapper = new TaskWrapper( - task, context, callback, retryPolicyFactory.makeRetryPolicy() - ); + TaskWrapper taskWrapper = new TaskWrapper(task, callback, retryPolicyFactory.makeRetryPolicy()); tasks.put(taskWrapper.getTask().getId(), taskWrapper); assignTask(taskWrapper); } @@ -606,13 +603,12 @@ public class RemoteTaskRunner implements TaskRunner { synchronized (statusLock) { final Task task = taskWrapper.getTask(); - final TaskContext taskContext = taskWrapper.getTaskContext(); try { log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId()); tasks.put(task.getId(), taskWrapper); - byte[] rawBytes = jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)); + byte[] rawBytes = jsonMapper.writeValueAsBytes(task); if (rawBytes.length > config.getMaxNumBytes()) { throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes()); @@ -626,7 +622,7 @@ public class RemoteTaskRunner implements TaskRunner theWorker.getHost(), task.getId() ), - jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)) + jsonMapper.writeValueAsBytes(task) ); // Syncing state with Zookeeper diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java deleted file mode 100644 index 1d279fe0511..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java +++ /dev/null @@ -1,66 +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.merger.coordinator; - -import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; - -import java.util.Set; - -/** - * Information gathered by the coordinator, after acquiring a lock, that may be useful to a task. - */ -public class TaskContext -{ - final String version; - final Set currentSegments; - final Set unusedSegments; - - @JsonCreator - public TaskContext( - @JsonProperty("version") String version, - @JsonProperty("currentSegments") Set currentSegments, - @JsonProperty("unusedSegments") Set unusedSegments - ) - { - this.version = version; - this.currentSegments = currentSegments; - this.unusedSegments = unusedSegments; - } - - @JsonProperty - public String getVersion() - { - return version; - } - - @JsonProperty - public Set getCurrentSegments() - { - return currentSegments; - } - - @JsonProperty - public Set getUnusedSegments() - { - return unusedSegments; - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java new file mode 100644 index 00000000000..b62bca604d4 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java @@ -0,0 +1,405 @@ +package com.metamx.druid.merger.coordinator; + +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.metamx.common.IAE; +import com.metamx.common.guava.Comparators; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.druid.merger.common.TaskLock; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.emitter.EmittingLogger; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Remembers which tasks have locked which intervals. Tasks are permitted to lock an interval if no other task + * outside their group has locked an overlapping interval for the same datasource. When a task locks an interval, + * it is assigned a version string that it can use to publish segments. + */ +public class TaskLockbox +{ + // Datasource -> Interval -> Tasks + TaskLock + private final Map> running = Maps.newHashMap(); + private final TaskStorage taskStorage; + private final ReentrantLock giant = new ReentrantLock(); + + private static final EmittingLogger log = new EmittingLogger(TaskLockbox.class); + + public TaskLockbox(TaskStorage taskStorage) + { + this.taskStorage = taskStorage; + } + + /** + * Attempt to lock a task, without removing it from the queue. Equivalent to the long form of {@code tryLock} + * with no preferred version. + * + * @param task task to attempt to lock + * + * @return lock version if lock was acquired, absent otherwise + */ + public Optional tryLock(final Task task, final Interval interval) + { + return tryLock(task, interval, Optional.absent()); + } + + /** + * Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task. + * This method will attempt to assign version strings that obey the invariant that every version string is + * lexicographically greater than any other version string previously assigned to the same interval. This invariant + * is only mostly guaranteed, however; we assume clock monotonicity and we assume that callers specifying + * {@code preferredVersion} are doing the right thing. + * + * @param task task to attempt to lock + * @param preferredVersion use this version string if one has not yet been assigned + * + * @return lock version if lock was acquired, absent otherwise + */ + public Optional tryLock(final Task task, final Interval interval, final Optional preferredVersion) + { + giant.lock(); + + try { + + if(task.getFixedInterval().isPresent() && !task.getFixedInterval().get().equals(interval)) { + // Task may only lock its fixed interval, if present + throw new IAE("Task must lock its fixed interval: %s", task.getId()); + } + + final String dataSource = task.getDataSource(); + final List foundPosses = findLockPossesForInterval(dataSource, interval); + final TaskLockPosse posseToUse; + + if (foundPosses.size() > 1) { + + // Too many existing locks. + return Optional.absent(); + + } else if (foundPosses.size() == 1) { + + // One existing lock -- check if we can add to it. + + final TaskLockPosse foundPosse = Iterables.getOnlyElement(foundPosses); + if (foundPosse.getTaskLock().getInterval().contains(interval) && foundPosse.getTaskLock().getGroupId().equals(task.getGroupId())) { + posseToUse = foundPosse; + } else { + return Optional.absent(); + } + + } else { + + // No existing locks. We can make a new one. + if (!running.containsKey(dataSource)) { + running.put(dataSource, new TreeMap(Comparators.intervalsByStartThenEnd())); + } + + // Create new TaskLock and assign it a version. + // Assumption: We'll choose a version that is greater than any previously-chosen version for our interval. (This + // may not always be true, unfortunately. See below.) + + final String version; + + if (preferredVersion.isPresent()) { + // We have a preferred version. We'll trust our caller to not break our ordering assumptions and just use it. + version = preferredVersion.get(); + } else { + // We are running under an interval lock right now, so just using the current time works as long as we can trust + // our clock to be monotonic and have enough resolution since the last time we created a TaskLock for the same + // interval. This may not always be true; to assure it we would need to use some method of timekeeping other + // than the wall clock. + version = new DateTime().toString(); + } + + posseToUse = new TaskLockPosse(new TaskLock(task.getGroupId(), dataSource, interval, version)); + running.get(dataSource) + .put(interval, posseToUse); + + log.info("Created new TaskLockPosse: %s", posseToUse); + } + + // Add to existing TaskLockPosse, if necessary + if (posseToUse.getTaskIds().add(task.getId())) { + log.info("Added task[%s] to TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId()); + + // Best effort to update task storage facility + try { + taskStorage.addLock(task.getId(), posseToUse.getTaskLock()); + } catch(Exception e) { + log.makeAlert("Failed to persist lock in storage") + .addData("task", task.getId()) + .addData("dataSource", posseToUse.getTaskLock().getDataSource()) + .addData("interval", posseToUse.getTaskLock().getInterval()) + .addData("version", posseToUse.getTaskLock().getVersion()) + .emit(); + } + } else { + log.info("Task[%s] already present in TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId()); + } + + return Optional.of(posseToUse.getTaskLock()); + } + finally { + giant.unlock(); + } + + } + + /** + * Return the currently-active locks for some task. + * + * @param task task for which to locate locks + */ + public List findLocksForTask(final Task task) + { + giant.lock(); + + try { + return Lists.transform( + findLockPossesForTask(task), new Function() + { + @Override + public TaskLock apply(TaskLockPosse taskLockPosse) + { + return taskLockPosse.getTaskLock(); + } + } + ); + } finally { + giant.unlock(); + } + } + + /** + * Release lock held for a task on a particular interval. Does nothing if the task does not currently + * hold the mentioned lock. + * + * @param task task to unlock + * @param interval interval to unlock + */ + public void unlock(final Task task, final Interval interval) + { + giant.lock(); + + try { + final String dataSource = task.getDataSource(); + final NavigableMap dsRunning = running.get(dataSource); + + // So we can alert if tasks try to release stuff they don't have + boolean removed = false; + + if(dsRunning != null) { + final TaskLockPosse taskLockPosse = dsRunning.get(interval); + if(taskLockPosse != null) { + final TaskLock taskLock = taskLockPosse.getTaskLock(); + + // Remove task from live list + log.info("Removing task[%s] from TaskLock[%s]", task.getId(), taskLock.getGroupId()); + removed = taskLockPosse.getTaskIds().remove(task.getId()); + + if (taskLockPosse.getTaskIds().isEmpty()) { + log.info("TaskLock is now empty: %s", taskLock); + running.get(dataSource).remove(taskLock.getInterval()); + } + + if (running.get(dataSource).size() == 0) { + running.remove(dataSource); + } + + // Best effort to remove lock from storage + try { + taskStorage.removeLock(task.getId(), taskLock); + } catch(Exception e) { + log.makeAlert(e, "Failed to clean up lock from storage") + .addData("task", task.getId()) + .addData("dataSource", taskLock.getDataSource()) + .addData("interval", taskLock.getInterval()) + .addData("version", taskLock.getVersion()) + .emit(); + } + } + } + + if(!removed) { + log.makeAlert("Lock release without acquire") + .addData("task", task.getId()) + .addData("interval", interval) + .emit(); + } + } finally { + giant.unlock(); + } + } + + /** + * Release all locks for a task. Does nothing if the task is not currently locked. + * + * @param task task to unlock + */ + public void unlock(final Task task) + { + giant.lock(); + + try { + for(final TaskLockPosse taskLockPosse : findLockPossesForTask(task)) { + unlock(task, taskLockPosse.getTaskLock().getInterval()); + } + } + finally { + giant.unlock(); + } + } + + /** + * Removes all locks from this lockbox. + */ + public void clear() + { + giant.lock(); + + try { + running.clear(); + } finally { + giant.unlock(); + } + } + + /** + * Return the currently-active lock posses for some task. + * + * @param task task for which to locate locks + */ + private List findLockPossesForTask(final Task task) + { + giant.lock(); + + try { + final Iterable searchSpace; + + if (task.getFixedInterval().isPresent()) { + // Narrow down search using findLockPossesForInterval + searchSpace = findLockPossesForInterval(task.getDataSource(), task.getFixedInterval().get()); + } else { + // Scan through all locks for this datasource + final NavigableMap dsRunning = running.get(task.getDataSource()); + if(dsRunning == null) { + searchSpace = ImmutableList.of(); + } else { + searchSpace = dsRunning.values(); + } + } + + return ImmutableList.copyOf( + Iterables.filter( + searchSpace, new Predicate() + { + @Override + public boolean apply(TaskLockPosse taskLock) + { + return taskLock.getTaskIds().contains(task.getId()); + } + } + ) + ); + } + finally { + giant.unlock(); + } + } + + /** + * Return all locks that overlap some search interval. + */ + private List findLockPossesForInterval(final String dataSource, final Interval interval) + { + giant.lock(); + + try { + final NavigableMap dsRunning = running.get(dataSource); + if (dsRunning == null) { + // No locks at all + return Collections.emptyList(); + } else { + // Tasks are indexed by locked interval, which are sorted by interval start. Intervals are non-overlapping, so: + final NavigableSet dsLockbox = dsRunning.navigableKeySet(); + final Iterable searchIntervals = Iterables.concat( + // Single interval that starts at or before ours + Collections.singletonList(dsLockbox.floor(new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)))), + + // All intervals that start somewhere between our start instant (exclusive) and end instant (exclusive) + dsLockbox.subSet( + new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)), + false, + new Interval(interval.getEnd(), interval.getEnd()), + false + ) + ); + + return Lists.newArrayList( + FunctionalIterable + .create(searchIntervals) + .filter( + new Predicate() + { + @Override + public boolean apply(@Nullable Interval searchInterval) + { + return searchInterval != null && searchInterval.overlaps(interval); + } + } + ) + .transform( + new Function() + { + @Override + public TaskLockPosse apply(Interval interval) + { + return dsRunning.get(interval); + } + } + ) + ); + } + } + finally { + giant.unlock(); + } + } + + private static class TaskLockPosse + { + final private TaskLock taskLock; + final private Set taskIds; + + public TaskLockPosse(TaskLock taskLock) + { + this.taskLock = taskLock; + taskIds = Sets.newHashSet(); + } + + public TaskLock getTaskLock() + { + return taskLock; + } + + public Set getTaskIds() + { + return taskIds; + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java similarity index 82% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java index 0481e8f21b6..7257e3fa679 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java @@ -25,6 +25,7 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServiceDiscoveryConfig; +import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.exec.TaskConsumer; import com.metamx.emitter.EmittingLogger; @@ -40,26 +41,31 @@ import java.util.concurrent.locks.ReentrantLock; /** * Encapsulates the indexer leadership lifecycle. */ -public class TaskMaster +public class TaskMasterLifecycle { private final LeaderSelector leaderSelector; private final ReentrantLock giant = new ReentrantLock(); private final Condition mayBeStopped = giant.newCondition(); + private final TaskQueue taskQueue; + private final TaskToolbox taskToolbox; private volatile boolean leading = false; - private static final EmittingLogger log = new EmittingLogger(TaskMaster.class); + private static final EmittingLogger log = new EmittingLogger(TaskMasterLifecycle.class); - public TaskMaster( - final TaskQueue queue, + public TaskMasterLifecycle( + final TaskQueue taskQueue, + final TaskToolbox taskToolbox, final IndexerCoordinatorConfig indexerCoordinatorConfig, final ServiceDiscoveryConfig serviceDiscoveryConfig, - final MergerDBCoordinator mergerDBCoordinator, final TaskRunnerFactory runnerFactory, final CuratorFramework curator, final ServiceEmitter emitter - ) + ) { + this.taskQueue = taskQueue; + this.taskToolbox = taskToolbox; + this.leaderSelector = new LeaderSelector( curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener() { @@ -71,15 +77,23 @@ public class TaskMaster try { log.info("By the power of Grayskull, I have the power!"); - final TaskRunner runner = runnerFactory.build(); - final TaskConsumer consumer = new TaskConsumer(queue, runner, mergerDBCoordinator, emitter); + final TaskRunner taskRunner = runnerFactory.build(); + final TaskConsumer taskConsumer = new TaskConsumer( + taskQueue, + taskRunner, + taskToolbox, + emitter + ); + + // Bootstrap task queue and task lockbox (load state stuff from the database) + taskQueue.bootstrap(); // Sensible order to start stuff: final Lifecycle leaderLifecycle = new Lifecycle(); - leaderLifecycle.addManagedInstance(queue); - leaderLifecycle.addManagedInstance(runner); + leaderLifecycle.addManagedInstance(taskQueue); + leaderLifecycle.addManagedInstance(taskRunner); Initialization.makeServiceDiscoveryClient(curator, serviceDiscoveryConfig, leaderLifecycle); - leaderLifecycle.addManagedInstance(consumer); + leaderLifecycle.addManagedInstance(taskConsumer); leaderLifecycle.start(); leading = true; @@ -181,4 +195,14 @@ public class TaskMaster throw Throwables.propagate(e); } } + + public TaskQueue getTaskQueue() + { + return taskQueue; + } + + public TaskToolbox getTaskToolbox() + { + return taskToolbox; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java index e228b401025..377004d473f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java @@ -19,36 +19,22 @@ package com.metamx.druid.merger.coordinator; -import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import com.google.common.base.Throwables; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Iterators; +import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; -import com.metamx.common.guava.Comparators; -import com.metamx.common.guava.FunctionalIterable; 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.task.Task; import com.metamx.emitter.EmittingLogger; -import org.joda.time.DateTime; -import org.joda.time.Interval; -import javax.annotation.Nullable; -import java.util.Collections; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.NavigableMap; -import java.util.NavigableSet; -import java.util.TreeMap; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -59,12 +45,12 @@ import java.util.concurrent.locks.ReentrantLock; * {@link #take} or {@link #poll}. Ordering is mostly-FIFO, with deviations when the natural next task would conflict * with a currently-running task. In that case, tasks are skipped until a runnable one is found. *

    - * To manage locking, the queue keeps track of currently-running tasks as {@link TaskGroup} objects. The idea is that - * only one TaskGroup can be running on a particular dataSource + interval, and that TaskGroup has a single version - * string that all tasks in the group must use to publish segments. Tasks in the same TaskGroup may run concurrently. + * To manage locking, the queue keeps track of currently-running tasks as {@link com.metamx.druid.merger.common.TaskLock} objects. The idea is that + * only one TaskLock can be running on a particular dataSource + interval, and that TaskLock has a single version + * string that all tasks in the group must use to publish segments. Tasks in the same TaskLock may run concurrently. *

    * For persistence, the queue saves new tasks from {@link #add} and task status updates from {@link #notify} using a - * {@link TaskStorage} object. + * {@link TaskStorage} obj *

    * To support leader election of our containing system, the queue can be stopped (in which case it will not accept * any new tasks, or hand out any more tasks, until started again). @@ -72,10 +58,8 @@ import java.util.concurrent.locks.ReentrantLock; public class TaskQueue { private final List queue = Lists.newLinkedList(); - private final Map> running = Maps.newHashMap(); - private final TaskStorage taskStorage; - + private final TaskLockbox taskLockbox; private final ReentrantLock giant = new ReentrantLock(); private final Condition workMayBeAvailable = giant.newCondition(); @@ -83,14 +67,98 @@ public class TaskQueue private static final EmittingLogger log = new EmittingLogger(TaskQueue.class); - public TaskQueue(TaskStorage taskStorage) + public TaskQueue(TaskStorage taskStorage, TaskLockbox taskLockbox) { this.taskStorage = Preconditions.checkNotNull(taskStorage, "taskStorage"); + this.taskLockbox = Preconditions.checkNotNull(taskLockbox, "taskLockbox"); } /** - * Starts this task queue. Loads tasks from our task storage facility and allows {@link #add(Task)} to accept - * new tasks. This should not be called on an already-started queue. + * Bootstraps this task queue and associated task lockbox. Clears the lockbox before running. Should be called + * while the queue is stopped. It is not a good idea to start the queue if this method fails. + */ + public void bootstrap() + { + giant.lock(); + + try { + Preconditions.checkState(!active, "queue must be stopped"); + + log.info("Bootstrapping queue (and associated lockbox)"); + + queue.clear(); + taskLockbox.clear(); + + // Add running tasks to the queue + final List runningTasks = taskStorage.getRunningTasks(); + + for(final Task task : runningTasks) { + queue.add(task); + } + + // Get all locks, along with which tasks they belong to + final Multimap tasksByLock = ArrayListMultimap.create(); + for(final Task runningTask : runningTasks) { + for(final TaskLock taskLock : taskStorage.getLocks(runningTask.getId())) { + tasksByLock.put(taskLock, runningTask); + } + } + + // Sort locks by version + final Ordering byVersionOrdering = new Ordering() + { + @Override + public int compare(TaskLock left, TaskLock right) + { + return left.getVersion().compareTo(right.getVersion()); + } + }; + + // Acquire as many locks as possible, in version order + for(final Map.Entry taskAndLock : tasksByLock.entries()) { + final Task task = taskAndLock.getValue(); + final TaskLock savedTaskLock = taskAndLock.getKey(); + + final Optional acquiredTaskLock = taskLockbox.tryLock( + task, + savedTaskLock.getInterval(), + Optional.of(savedTaskLock.getVersion()) + ); + + if(acquiredTaskLock.isPresent() && savedTaskLock.getVersion().equals(acquiredTaskLock.get().getVersion())) { + log.info( + "Reacquired lock on interval[%s] version[%s] for task: %s", + savedTaskLock.getInterval(), + savedTaskLock.getVersion(), + task.getId() + ); + } else if(acquiredTaskLock.isPresent()) { + log.info( + "Could not reacquire lock on interval[%s] version[%s] (got version[%s] instead) for task: %s", + savedTaskLock.getInterval(), + savedTaskLock.getVersion(), + acquiredTaskLock.get().getVersion(), + task.getId() + ); + } else { + log.info( + "Could not reacquire lock on interval[%s] version[%s] for task: %s", + savedTaskLock.getInterval(), + savedTaskLock.getVersion(), + task.getId() + ); + } + } + + log.info("Bootstrapped %,d tasks. Ready to go!", runningTasks.size()); + } finally { + giant.unlock(); + } + } + + /** + * Starts this task queue. Allows {@link #add(Task)} to accept new tasks. This should not be called on + * an already-started queue. */ @LifecycleStart public void start() @@ -98,50 +166,9 @@ public class TaskQueue giant.lock(); try { - - Preconditions.checkState(!active, "queue was already started!"); - Preconditions.checkState(queue.isEmpty(), "queue must be empty!"); - Preconditions.checkState(running.isEmpty(), "running list must be empty!"); - - final TaskStorageQueryAdapter taskStorageQueryAdapter = new TaskStorageQueryAdapter(taskStorage); - final List runningTasks = taskStorageQueryAdapter.getRunningTaskVersions(); - - // Sort by version, with nulls last - final Ordering byVersionOrdering = new Ordering() - { - final private Ordering baseOrdering = Ordering.natural().nullsLast(); - - @Override - public int compare(VersionedTaskWrapper left, VersionedTaskWrapper right) - { - return baseOrdering.compare(left.getVersion(), right.getVersion()); - } - }; - - for (final VersionedTaskWrapper taskAndVersion : byVersionOrdering.sortedCopy(runningTasks)) { - final Task task = taskAndVersion.getTask(); - final String preferredVersion = taskAndVersion.getVersion(); - - queue.add(task); - - if (preferredVersion != null) { - final Optional version = tryLock(task, Optional.of(preferredVersion)); - - log.info( - "Bootstrapped task[%s] with preferred version[%s]: %s", - task.getId(), - preferredVersion, - version.isPresent() ? String.format("locked with version[%s]", version.get()) : "not lockable" - ); - } else { - log.info("Bootstrapped task[%s] with no preferred version", task.getId()); - } - } - - log.info("Bootstrapped %,d tasks. Ready to go!", runningTasks.size()); + Preconditions.checkState(!active, "queue must be stopped"); active = true; - workMayBeAvailable.signalAll(); } finally { @@ -159,13 +186,10 @@ public class TaskQueue giant.lock(); try { - log.info("Naptime! Shutting down until we are started again."); - queue.clear(); - running.clear(); + taskLockbox.clear(); active = false; - } finally { giant.unlock(); @@ -201,7 +225,9 @@ public class TaskQueue // Attempt to add this task to a running task group. Silently continue if this is not possible. // The main reason this is here is so when subtasks are added, they end up in the same task group // as their parent whenever possible. - tryLock(task); + if(task.getFixedInterval().isPresent()) { + taskLockbox.tryLock(task, task.getFixedInterval().get()); + } return true; } @@ -215,19 +241,22 @@ public class TaskQueue * * @return runnable task */ - public VersionedTaskWrapper take() throws InterruptedException + public Task take() throws InterruptedException { giant.lock(); try { - VersionedTaskWrapper taskWrapper; + Task task; - while ((taskWrapper = poll()) == null) { - log.info("Waiting for work..."); - workMayBeAvailable.await(); + log.info("Waiting for work..."); + + while ((task = poll()) == null) { + // awaitNanos because work may become available without this condition signalling, + // due to other folks messing with the taskLockbox + workMayBeAvailable.awaitNanos(1000000000L /* 1 second */); } - return taskWrapper; + return task; } finally { giant.unlock(); @@ -239,24 +268,28 @@ public class TaskQueue * * @return runnable task or null */ - public VersionedTaskWrapper poll() + public Task poll() { giant.lock(); try { - log.info("Checking for doable work"); for (final Task task : queue) { - final Optional maybeVersion = tryLock(task); - if (maybeVersion.isPresent()) { - Preconditions.checkState(active, "wtf? Found task when inactive"); - taskStorage.setVersion(task.getId(), maybeVersion.get()); + if(task.getFixedInterval().isPresent()) { + // If this task has a fixed interval, attempt to lock it right now. + final Optional maybeLock = taskLockbox.tryLock(task, task.getFixedInterval().get()); + if(maybeLock.isPresent()) { + log.info("Task claimed with fixed interval lock: %s", task.getId()); + queue.remove(task); + return task; + } + } else { + // No fixed interval. Let's just run this and see what happens. + log.info("Task claimed with no fixed interval lock: %s", task.getId()); queue.remove(task); - log.info("Task claimed: %s", task); - return new VersionedTaskWrapper(task, maybeVersion.get()); + return task; } } - log.info("No doable work found."); return null; } finally { @@ -264,348 +297,62 @@ public class TaskQueue } } - public void notify(final Task task, final TaskStatus status) - { - notify(task, status, null); - } - /** * Notify this queue that some task has an updated status. If this update is valid, the status will be persisted in - * the task storage facility, and any nextTasks present in the status will be created. If the status is a completed - * status, the task will be unlocked and no further updates will be accepted. If this task has failed, the task group - * it is part of will be terminated. - *

    - * Finally, if this task is not supposed to be running, this method will simply do nothing. + * the task storage facility. If the status is a completed status, the task will be unlocked and no further + * updates will be accepted. * * @param task task to update - * @param originalStatus new task status - * @param commitRunnable operation to perform if this task is ready to commit + * @param taskStatus new task status * * @throws NullPointerException if task or status is null * @throws IllegalArgumentException if the task ID does not match the status ID * @throws IllegalStateException if this queue is currently shut down */ - public void notify(final Task task, final TaskStatus originalStatus, final Runnable commitRunnable) + public void notify(final Task task, final TaskStatus taskStatus) { giant.lock(); try { Preconditions.checkNotNull(task, "task"); - Preconditions.checkNotNull(originalStatus, "status"); + Preconditions.checkNotNull(taskStatus, "status"); Preconditions.checkState(active, "Queue is not active!"); Preconditions.checkArgument( - task.getId().equals(originalStatus.getId()), + task.getId().equals(taskStatus.getId()), "Mismatching task ids[%s/%s]", task.getId(), - originalStatus.getId() + taskStatus.getId() ); - final TaskGroup taskGroup; - - final Optional maybeTaskGroup = findTaskGroupForTask(task); - if (!maybeTaskGroup.isPresent()) { - log.info("Ignoring notification for dead task: %s", task.getId()); - return; - } else { - taskGroup = maybeTaskGroup.get(); - } - - // This is what we want to write to the DB when we're done. - // Not final, since we might need to reassign the var later if the commitRunnable fails. - TaskStatus statusToSave = originalStatus; - - // Should we commit? - if (taskGroup.getCommitStyle().shouldCommit(task, statusToSave)) { - log.info("Committing %s status for task: %s", statusToSave.getStatusCode(), task.getId()); - - // Add next tasks - try { - if (commitRunnable != null) { - log.info("Running commitRunnable for task: %s", task.getId()); - commitRunnable.run(); - } - - // We want to allow tasks to submit RUNNING statuses with the same nextTasks over and over. - // So, we need to remember which ones we've already spawned and not do them again. - for (final Task nextTask : statusToSave.getNextTasks()) { - try { - add(nextTask); - } catch (TaskExistsException e) { - log.info("Already added followup task %s to original task: %s", nextTask.getId(), task.getId()); - } - } - } - catch (Exception e) { - log.makeAlert(e, "Failed to commit task") - .addData("task", task.getId()) - .addData("statusCode", statusToSave.getStatusCode()) - .emit(); - - // Rewrite status - statusToSave = TaskStatus.failure(task.getId()).withDuration(statusToSave.getDuration()); - } - } else { - log.info("Not committing %s status for task: %s", statusToSave.getStatusCode(), task); - } - - boolean didSetStatus = false; - + // Save status to DB + boolean didPersistStatus = false; try { - taskStorage.setStatus(task.getId(), statusToSave); - didSetStatus = true; + final Optional previousStatus = taskStorage.getStatus(task.getId()); + if (!previousStatus.isPresent() || !previousStatus.get().isRunnable()) { + log.makeAlert("Ignoring notification for dead task").addData("task", task.getId()).emit(); + return; + } else { + taskStorage.setStatus(taskStatus); + didPersistStatus = true; + } } catch(Exception e) { - // TODO: This could be a task-status-submission retry queue instead of retrying the entire task, - // TODO: which is heavy and probably not necessary. - log.warn(e, "Status could not be persisted! Reinserting task: %s", task.getId()); - - log.makeAlert(e, "Failed to persist task status") + log.makeAlert(e, "Failed to persist status for task") .addData("task", task.getId()) - .addData("statusCode", statusToSave.getStatusCode()) + .addData("statusCode", taskStatus.getStatusCode()) .emit(); - - queue.add(task); } - if(didSetStatus && statusToSave.isComplete()) { - unlock(task); - log.info("Task done: %s", task); - } - } - finally { - giant.unlock(); - } - } - - /** - * Unlock some work. Does not update the task storage facility. Throws an exception if this work is not currently - * running. - * - * @param task task to unlock - * - * @throws IllegalStateException if task is not currently locked - */ - private void unlock(final Task task) - { - giant.lock(); - - try { - final String dataSource = task.getDataSource(); - - final TaskGroup taskGroup; - final Optional maybeTaskGroup = findTaskGroupForTask(task); - - if (maybeTaskGroup.isPresent()) { - taskGroup = maybeTaskGroup.get(); - } else { - throw new IllegalStateException(String.format("Task must be running: %s", task.getId())); - } - - // Remove task from live list - log.info("Removing task[%s] from TaskGroup[%s]", task.getId(), taskGroup.getGroupId()); - taskGroup.remove(task.getId()); - - if (taskGroup.size() == 0) { - log.info("TaskGroup complete: %s", taskGroup); - running.get(dataSource).remove(taskGroup.getInterval()); - } - - if (running.get(dataSource).size() == 0) { - running.remove(dataSource); - } - - workMayBeAvailable.signalAll(); - } - finally { - giant.unlock(); - } - } - - /** - * Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task. - * - * @param task task to attempt to lock - * - * @return lock version if lock was acquired, absent otherwise - */ - private Optional tryLock(final Task task) - { - return tryLock(task, Optional.absent()); - } - - /** - * Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task. - * - * @param task task to attempt to lock - * @param preferredVersion use this version if possible (no guarantees, though!) - * - * @return lock version if lock was acquired, absent otherwise - */ - private Optional tryLock(final Task task, final Optional preferredVersion) - { - giant.lock(); - - try { - - final String dataSource = task.getDataSource(); - final Interval interval = task.getInterval(); - - final List foundLocks = findTaskGroupsForInterval(dataSource, interval); - final TaskGroup taskGroupToUse; - - if (foundLocks.size() > 1) { - - // Too many existing locks. - return Optional.absent(); - - } else if (foundLocks.size() == 1) { - - // One existing lock -- check if we can add to it. - - final TaskGroup foundLock = Iterables.getOnlyElement(foundLocks); - if (foundLock.getInterval().contains(interval) && foundLock.getGroupId().equals(task.getGroupId())) { - taskGroupToUse = foundLock; + if(taskStatus.isComplete()) { + if(didPersistStatus) { + log.info("Task done: %s", task); + taskLockbox.unlock(task); + workMayBeAvailable.signalAll(); } else { - return Optional.absent(); + // TODO: This could be a task-status-submission retry queue instead of retrying the entire task, + // TODO: which is heavy and probably not necessary. + log.warn("Status could not be persisted! Reinserting task: %s", task.getId()); + queue.add(task); } - - } else { - - // No existing locks. We can make a new one. - if (!running.containsKey(dataSource)) { - running.put(dataSource, new TreeMap(Comparators.intervalsByStartThenEnd())); - } - - // Create new TaskGroup and assign it a version. - // Assumption: We'll choose a version that is greater than any previously-chosen version for our interval. (This - // may not always be true, unfortunately. See below.) - - final String version; - - if (preferredVersion.isPresent()) { - // We have a preferred version. Since this is a private method, we'll trust our caller to not break our - // ordering assumptions and just use it. - version = preferredVersion.get(); - } else { - // We are running under an interval lock right now, so just using the current time works as long as we can trust - // our clock to be monotonic and have enough resolution since the last time we created a TaskGroup for the same - // interval. This may not always be true; to assure it we would need to use some method of timekeeping other - // than the wall clock. - version = new DateTime().toString(); - } - - taskGroupToUse = new TaskGroup(task.getGroupId(), dataSource, interval, version); - running.get(dataSource) - .put(interval, taskGroupToUse); - - log.info("Created new TaskGroup[%s]", taskGroupToUse); - - } - - // Add to existing TaskGroup, if necessary - if (taskGroupToUse.add(task)) { - log.info("Added task[%s] to TaskGroup[%s]", task.getId(), taskGroupToUse.getGroupId()); - } else { - log.info("Task[%s] already present in TaskGroup[%s]", task.getId(), taskGroupToUse.getGroupId()); - } - - return Optional.of(taskGroupToUse.getVersion()); - - } - finally { - giant.unlock(); - } - - } - - /** - * Return the currently-running task group for some task. If the task has no currently-running task group, this will - * return an absentee Optional. - * - * @param task task for which to locate group - */ - private Optional findTaskGroupForTask(final Task task) - { - giant.lock(); - - try { - final Iterator maybeTaskGroup = - FunctionalIterable.create(findTaskGroupsForInterval(task.getDataSource(), task.getInterval())) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskGroup taskGroup) - { - return taskGroup.contains(task.getId()); - } - } - ) - .iterator(); - - - if (!maybeTaskGroup.hasNext()) { - return Optional.absent(); - } else { - return Optional.of(Iterators.getOnlyElement(maybeTaskGroup)); - } - } - finally { - giant.unlock(); - } - } - - /** - * Return all locks that overlap some search interval. - */ - private List findTaskGroupsForInterval(final String dataSource, final Interval interval) - { - giant.lock(); - - try { - final NavigableMap dsRunning = running.get(dataSource); - if (dsRunning == null) { - // No locks at all - return Collections.emptyList(); - } else { - // Tasks are indexed by locked interval, which are sorted by interval start. Intervals are non-overlapping, so: - final NavigableSet dsLockbox = dsRunning.navigableKeySet(); - final Iterable searchIntervals = Iterables.concat( - // Single interval that starts at or before ours - Collections.singletonList(dsLockbox.floor(new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)))), - - // All intervals that start somewhere between our start instant (exclusive) and end instant (exclusive) - dsLockbox.subSet( - new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)), - false, - new Interval(interval.getEnd(), interval.getEnd()), - false - ) - ); - - return Lists.newArrayList( - FunctionalIterable - .create(searchIntervals) - .filter( - new Predicate() - { - @Override - public boolean apply(@Nullable Interval searchInterval) - { - return searchInterval != null && searchInterval.overlaps(interval); - } - } - ) - .transform( - new Function() - { - @Override - public TaskGroup apply(Interval interval) - { - return dsRunning.get(interval); - } - } - ) - ); } } finally { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java index 826a3b639cb..ace8c9c095a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java @@ -23,8 +23,8 @@ import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.task.Task; /** - * Interface for handing off tasks. Used by a {@link com.metamx.druid.merger.coordinator.exec.TaskConsumer} to run tasks that - * have been locked. + * Interface for handing off tasks. Used by a {@link com.metamx.druid.merger.coordinator.exec.TaskConsumer} to + * run tasks that have been locked. */ public interface TaskRunner { @@ -33,8 +33,7 @@ public interface TaskRunner * status, but should be called exactly once with a non-RUNNING status (e.g. SUCCESS, FAILED, CONTINUED...). * * @param task task to run - * @param context task context to run under * @param callback callback to be called exactly once */ - public void run(Task task, TaskContext context, TaskCallback callback); + public void run(Task task, TaskCallback callback); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java index 54732a1173f..2e21f52876b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java @@ -21,6 +21,8 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Optional; 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; @@ -34,15 +36,21 @@ public interface TaskStorage public void insert(Task task, TaskStatus status); /** - * Updates task status in the storage facility. + * Persists task status in the storage facility. This method should throw an exception if the task status lifecycle + * is not respected (absent -> RUNNING -> SUCCESS/FAILURE). */ - public void setStatus(String taskid, TaskStatus status); + public void setStatus(TaskStatus status); /** - * Updates task version in the storage facility. If the task already has a version, this method will throw - * an exception. + * Persists lock state in the storage facility. */ - public void setVersion(String taskid, String version); + public void addLock(String taskid, TaskLock taskLock); + + /** + * Removes lock state from the storage facility. It is harmless to keep old locks in the storage facility, but + * this method can help reclaim wasted space. + */ + public void removeLock(String taskid, TaskLock taskLock); /** * Returns task as stored in the storage facility. If the task ID does not exist, this will return an @@ -59,13 +67,22 @@ public interface TaskStorage public Optional getStatus(String taskid); /** - * Returns task version as stored in the storage facility. If the task ID does not exist, or if the task ID exists - * but was not yet assigned a version, this will return an absentee Optional. + * Add an action taken by a task to the audit log. */ - public Optional getVersion(String taskid); + public void addAuditLog(TaskAction taskAction); + + /** + * Returns all actions taken by a task. + */ + public List getAuditLogs(String taskid); /** * Returns a list of currently-running tasks as stored in the storage facility, in no particular order. */ public List getRunningTasks(); + + /** + * Returns a list of locks for a particular task. + */ + public List getLocks(String taskid); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java index e9cc37c9842..da8269f140f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java @@ -21,11 +21,17 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Function; import com.google.common.base.Optional; +import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import com.metamx.druid.merger.common.actions.SpawnTasksAction; +import com.metamx.druid.merger.common.actions.TaskAction; import com.metamx.druid.merger.common.task.Task; import java.util.List; @@ -62,8 +68,29 @@ public class TaskStorageQueryAdapter resultBuilder.put(taskid, statusOptional); + final Iterable nextTasks = FunctionalIterable + .create(storage.getAuditLogs(taskid)).filter( + new Predicate() + { + @Override + public boolean apply(TaskAction taskAction) + { + return taskAction instanceof SpawnTasksAction; + } + } + ).transformCat( + new Function>() + { + @Override + public Iterable apply(TaskAction taskAction) + { + return ((SpawnTasksAction) taskAction).getNewTasks(); + } + } + ); + if(taskOptional.isPresent() && statusOptional.isPresent()) { - for(final Task nextTask : statusOptional.get().getNextTasks()) { + for(final Task nextTask : nextTasks) { if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) { resultBuilder.putAll(getSameGroupChildStatuses(nextTask.getId())); } @@ -84,20 +111,12 @@ public class TaskStorageQueryAdapter int nFailures = 0; int nTotal = 0; - final Set segments = Sets.newHashSet(); - final Set segmentsNuked = Sets.newHashSet(); - final List nextTasks = Lists.newArrayList(); - for(final Optional statusOption : statuses.values()) { nTotal ++; if(statusOption.isPresent()) { final TaskStatus status = statusOption.get(); - segments.addAll(status.getSegments()); - segmentsNuked.addAll(status.getSegmentsNuked()); - nextTasks.addAll(status.getNextTasks()); - if(status.isSuccess()) { nSuccesses ++; } else if(status.isFailure()) { @@ -111,10 +130,7 @@ public class TaskStorageQueryAdapter if(nTotal == 0) { status = Optional.absent(); } else if(nSuccesses == nTotal) { - status = Optional.of(TaskStatus.success(taskid) - .withSegments(segments) - .withSegmentsNuked(segmentsNuked) - .withNextTasks(nextTasks)); + status = Optional.of(TaskStatus.success(taskid)); } else if(nFailures > 0) { status = Optional.of(TaskStatus.failure(taskid)); } else { @@ -125,21 +141,37 @@ public class TaskStorageQueryAdapter } /** - * Returns running tasks along with their preferred versions. If no version is present for a task, the - * version field of the returned {@link VersionedTaskWrapper} will be null. + * Returns all segments created by descendants for a particular task that stayed within the same task group. Includes + * that task, plus any tasks it spawned, and so on. Does not include spawned tasks that ended up in a different task + * group. Does not include this task's parents or siblings. */ - public List getRunningTaskVersions() + public Set getSameGroupNewSegments(final String taskid) { - return Lists.transform( - storage.getRunningTasks(), - new Function() - { - @Override - public VersionedTaskWrapper apply(Task task) - { - return new VersionedTaskWrapper(task, storage.getVersion(task.getId()).orNull()); - } + // TODO: This is useful for regular index tasks (so we know what was published), but + // TODO: for long-lived index tasks the list can get out of hand. We may want a limit. + + final Optional taskOptional = storage.getTask(taskid); + final Set segments = Sets.newHashSet(); + final List nextTasks = Lists.newArrayList(); + + for(final TaskAction action : storage.getAuditLogs(taskid)) { + if(action instanceof SpawnTasksAction) { + nextTasks.addAll(((SpawnTasksAction) action).getNewTasks()); + } + + if(action instanceof SegmentInsertAction) { + segments.addAll(((SegmentInsertAction) action).getSegments()); + } + } + + if(taskOptional.isPresent()) { + for(final Task nextTask : nextTasks) { + if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) { + segments.addAll(getSameGroupNewSegments(nextTask.getId())); } - ); + } + } + + return segments; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java index 22e7d4152b4..ae0ae58973d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java @@ -27,14 +27,12 @@ import com.metamx.druid.merger.common.task.Task; public class TaskWrapper { private final Task task; - private final TaskContext taskContext; private final TaskCallback callback; private final RetryPolicy retryPolicy; - public TaskWrapper(Task task, TaskContext taskContext, TaskCallback callback, RetryPolicy retryPolicy) + public TaskWrapper(Task task, TaskCallback callback, RetryPolicy retryPolicy) { this.task = task; - this.taskContext = taskContext; this.callback = callback; this.retryPolicy = retryPolicy; } @@ -44,11 +42,6 @@ public class TaskWrapper return task; } - public TaskContext getTaskContext() - { - return taskContext; - } - public TaskCallback getCallback() { return callback; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/VersionedTaskWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/VersionedTaskWrapper.java deleted file mode 100644 index 4eb0726ef69..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/VersionedTaskWrapper.java +++ /dev/null @@ -1,44 +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.merger.coordinator; - -import com.metamx.druid.merger.common.task.Task; - -public class VersionedTaskWrapper -{ - final Task task; - final String version; - - public VersionedTaskWrapper(Task task, String version) - { - this.task = task; - this.version = version; - } - - public Task getTask() - { - return task; - } - - public String getVersion() - { - return version; - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/CommitStyle.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/CommitStyle.java deleted file mode 100644 index 4f9c2e4a989..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/CommitStyle.java +++ /dev/null @@ -1,32 +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.merger.coordinator.commit; - -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskGroup; - -/** - * Determines whether or not metadata from a task status update should be committed. - */ -public interface CommitStyle -{ - public boolean shouldCommit(Task task, TaskStatus taskStatus); -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/ImmediateCommitStyle.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/ImmediateCommitStyle.java deleted file mode 100644 index 56d29533cd3..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/ImmediateCommitStyle.java +++ /dev/null @@ -1,33 +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.merger.coordinator.commit; - -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskGroup; - -public class ImmediateCommitStyle implements CommitStyle -{ - @Override - public boolean shouldCommit(Task task, TaskStatus taskStatus) - { - return true; - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/TaskCommitStyle.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/TaskCommitStyle.java deleted file mode 100644 index 28c7d6fc3ca..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/commit/TaskCommitStyle.java +++ /dev/null @@ -1,32 +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.merger.coordinator.commit; - -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.task.Task; - -public class TaskCommitStyle implements CommitStyle -{ - @Override - public boolean shouldCommit(Task task, TaskStatus taskStatus) - { - return taskStatus.isSuccess(); - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java index 26f0105771d..96eca59947a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java @@ -28,4 +28,12 @@ public abstract class IndexerDbConnectorConfig extends DbConnectorConfig @JsonProperty("taskTable") @Config("druid.database.taskTable") public abstract String getTaskTable(); + + @JsonProperty("taskLockTable") + @Config("druid.database.taskLockTable") + public abstract String getTaskLockTable(); + + @JsonProperty("taskLogTable") + @Config("druid.database.taskLogTable") + public abstract String getTaskLogTable(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index af4d2b478f9..14135174e59 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -20,29 +20,23 @@ package com.metamx.druid.merger.coordinator.exec; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableSet; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.MergerDBCoordinator; -import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskRunner; -import com.metamx.druid.merger.coordinator.VersionedTaskWrapper; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import java.util.Set; - public class TaskConsumer implements Runnable { private final TaskQueue queue; private final TaskRunner runner; - private final MergerDBCoordinator mergerDBCoordinator; + private final TaskToolbox toolbox; private final ServiceEmitter emitter; private final Thread thready; @@ -53,13 +47,13 @@ public class TaskConsumer implements Runnable public TaskConsumer( TaskQueue queue, TaskRunner runner, - MergerDBCoordinator mergerDBCoordinator, + TaskToolbox toolbox, ServiceEmitter emitter ) { this.queue = queue; this.runner = runner; - this.mergerDBCoordinator = mergerDBCoordinator; + this.toolbox = toolbox; this.emitter = emitter; this.thready = new Thread(this); } @@ -85,12 +79,9 @@ public class TaskConsumer implements Runnable while (!Thread.currentThread().isInterrupted()) { final Task task; - final String version; try { - final VersionedTaskWrapper taskWrapper = queue.take(); - task = taskWrapper.getTask(); - version = taskWrapper.getVersion(); + task = queue.take(); } catch (InterruptedException e) { log.info(e, "Interrupted while waiting for new work"); @@ -98,17 +89,17 @@ public class TaskConsumer implements Runnable } try { - handoff(task, version); + handoff(task); } catch (Exception e) { log.makeAlert(e, "Failed to hand off task") .addData("task", task.getId()) - .addData("type", task.getType().toString()) + .addData("type", task.getType()) .addData("dataSource", task.getDataSource()) - .addData("interval", task.getInterval()) + .addData("interval", task.getFixedInterval()) .emit(); - // Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now. + // Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now. if (!shutdown) { queue.notify(task, TaskStatus.failure(task.getId())); } @@ -122,32 +113,17 @@ public class TaskConsumer implements Runnable } } - private void handoff(final Task task, final String version) throws Exception + private void handoff(final Task task) throws Exception { - final TaskContext context = new TaskContext( - version, - ImmutableSet.copyOf( - mergerDBCoordinator.getUsedSegmentsForInterval( - task.getDataSource(), - task.getInterval() - ) - ), - ImmutableSet.copyOf( - mergerDBCoordinator.getUnusedSegmentsForInterval( - task.getDataSource(), - task.getInterval() - ) - ) - ); - final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder() + final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() .setUser2(task.getDataSource()) - .setUser4(task.getType().toString()) - .setUser5(task.getInterval().toString()); + .setUser4(task.getType()) + .setUser5(task.getFixedInterval().toString()); // Run preflight checks TaskStatus preflightStatus; try { - preflightStatus = task.preflight(context); + preflightStatus = task.preflight(toolbox); log.info("Preflight done for task: %s", task.getId()); } catch (Exception e) { @@ -161,11 +137,9 @@ public class TaskConsumer implements Runnable return; } - // Hand off work to TaskRunner - // TODO -- Should something in the TaskCallback enforce that each returned status is logically after the previous? - // TODO -- Probably yes. But make sure it works in the face of RTR retries. + // Hand off work to TaskRunner, with a callback runner.run( - task, context, new TaskCallback() + task, new TaskCallback() { @Override public void notify(final TaskStatus statusFromRunner) @@ -180,82 +154,26 @@ public class TaskConsumer implements Runnable return; } - queue.notify( - task, statusFromRunner, new Runnable() - { - @Override - public void run() - { - try { - // Validate status - for (final DataSegment segment : statusFromRunner.getSegments()) { - verifyDataSourceAndInterval(task, context, segment); - - // Verify version (must be equal to our context version) - if (!context.getVersion().equals(segment.getVersion())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid version: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - } - - for (final DataSegment segment : statusFromRunner.getSegmentsNuked()) { - verifyDataSourceAndInterval(task, context, segment); - - // Verify version (must be less than our context version) - if (segment.getVersion().compareTo(context.getVersion()) >= 0) { - throw new IllegalStateException( - String.format( - "Segment-to-nuke for task[%s] has invalid version: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - } - - mergerDBCoordinator.commitTaskStatus(statusFromRunner); - } - catch (Exception e) { - log.error(e, "Exception while publishing segments for task: %s", task); - throw Throwables.propagate(e); - } - } - } - ); + queue.notify(task, statusFromRunner); // Emit event and log, if the task is done if (statusFromRunner.isComplete()) { - builder.setUser3(statusFromRunner.getStatusCode().toString()); - - for (DataSegment segment : statusFromRunner.getSegments()) { - emitter.emit(builder.build("indexer/segment/bytes", segment.getSize())); - } - - for (DataSegment segmentNuked : statusFromRunner.getSegmentsNuked()) { - emitter.emit(builder.build("indexer/segmentNuked/bytes", segmentNuked.getSize())); - } - - emitter.emit(builder.build("indexer/time/run/millis", statusFromRunner.getDuration())); + metricBuilder.setUser3(statusFromRunner.getStatusCode().toString()); + emitter.emit(metricBuilder.build("indexer/time/run/millis", statusFromRunner.getDuration())); if (statusFromRunner.isFailure()) { log.makeAlert("Failed to index") .addData("task", task.getId()) - .addData("type", task.getType().toString()) + .addData("type", task.getType()) .addData("dataSource", task.getDataSource()) - .addData("interval", task.getInterval()) + .addData("interval", task.getFixedInterval()) .emit(); } log.info( - "Task %s: %s (%d segments) (%d run duration)", + "Task %s: %s (%d run duration)", statusFromRunner.getStatusCode(), task, - statusFromRunner.getSegments().size(), statusFromRunner.getDuration() ); } @@ -270,69 +188,4 @@ public class TaskConsumer implements Runnable } ); } - - private void deleteSegments(Task task, TaskContext context, Set segments) throws Exception - { - for (DataSegment segment : segments) { - verifyDataSourceAndInterval(task, context, segment); - - // Verify version (must be less than our context version) - if (segment.getVersion().compareTo(context.getVersion()) >= 0) { - throw new IllegalStateException( - String.format( - "Segment-to-nuke for task[%s] has invalid version: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - - log.info("Deleting segment[%s] for task[%s]", segment.getIdentifier(), task.getId()); - mergerDBCoordinator.deleteSegment(segment); - } - } - - private void publishSegments(Task task, TaskContext context, Set segments) throws Exception - { - for (DataSegment segment : segments) { - verifyDataSourceAndInterval(task, context, segment); - - // Verify version (must be equal to our context version) - if (!context.getVersion().equals(segment.getVersion())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid version: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - - log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), task.getId()); - mergerDBCoordinator.announceHistoricalSegment(segment); - } - } - - private void verifyDataSourceAndInterval(Task task, TaskContext context, DataSegment segment) - { - if (!task.getDataSource().equals(segment.getDataSource())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid dataSource: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - - if (!task.getInterval().contains(segment.getInterval())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid interval: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 3d084d60712..07c900f7455 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -48,11 +48,13 @@ import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.S3SegmentKiller; -import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.loading.S3SegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.merger.common.actions.TaskActionToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -62,7 +64,8 @@ import com.metamx.druid.merger.coordinator.LocalTaskStorage; 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.TaskMaster; +import com.metamx.druid.merger.coordinator.TaskLockbox; +import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskRunner; import com.metamx.druid.merger.coordinator.TaskRunnerFactory; @@ -134,18 +137,20 @@ public class IndexerCoordinatorNode extends RegisteringNode private ServiceEmitter emitter = null; private DbConnectorConfig dbConnectorConfig = null; private DBI dbi = null; + private RestS3Service s3Service = null; private IndexerCoordinatorConfig config = null; private TaskConfig taskConfig = null; private TaskToolbox taskToolbox = null; private MergerDBCoordinator mergerDBCoordinator = null; private TaskStorage taskStorage = null; private TaskQueue taskQueue = null; + private TaskLockbox taskLockbox = null; private CuratorFramework curatorFramework = null; private ScheduledExecutorFactory scheduledExecutorFactory = null; private IndexerZkConfig indexerZkConfig; private WorkerSetupManager workerSetupManager = null; private TaskRunnerFactory taskRunnerFactory = null; - private TaskMaster taskMaster = null; + private TaskMasterLifecycle taskMasterLifecycle = null; private Server server = null; private boolean initialized = false; @@ -183,6 +188,18 @@ public class IndexerCoordinatorNode extends RegisteringNode return this; } + public IndexerCoordinatorNode setS3Service(RestS3Service s3Service) + { + this.s3Service = s3Service; + return this; + } + + public IndexerCoordinatorNode setTaskLockbox(TaskLockbox taskLockbox) + { + this.taskLockbox = taskLockbox; + return this; + } + public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator) { this.mergerDBCoordinator = mergeDbCoordinator; @@ -216,17 +233,19 @@ public class IndexerCoordinatorNode extends RegisteringNode initializeDB(); initializeIndexerCoordinatorConfig(); initializeTaskConfig(); + initializeS3Service(); initializeMergeDBCoordinator(); - initializeTaskToolbox(); initializeTaskStorage(); + initializeTaskLockbox(); initializeTaskQueue(); + initializeTaskToolbox(); initializeJacksonInjections(); initializeJacksonSubtypes(); initializeCurator(); initializeIndexerZkConfig(); initializeWorkerSetupManager(); initializeTaskRunnerFactory(); - initializeTaskMaster(); + initializeTaskMasterLifecycle(); initializeServer(); final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d"); @@ -243,7 +262,7 @@ public class IndexerCoordinatorNode extends RegisteringNode jsonMapper, config, emitter, - taskQueue, + taskMasterLifecycle, new TaskStorageQueryAdapter(taskStorage), workerSetupManager ) @@ -263,7 +282,7 @@ public class IndexerCoordinatorNode extends RegisteringNode @Override public boolean doLocal() { - return taskMaster.isLeading(); + return taskMasterLifecycle.isLeading(); } @Override @@ -273,7 +292,7 @@ public class IndexerCoordinatorNode extends RegisteringNode return new URL( String.format( "http://%s%s", - taskMaster.getLeader(), + taskMasterLifecycle.getLeader(), requestURI ) ); @@ -291,20 +310,20 @@ public class IndexerCoordinatorNode extends RegisteringNode initialized = true; } - private void initializeTaskMaster() + private void initializeTaskMasterLifecycle() { - if (taskMaster == null) { + if (taskMasterLifecycle == null) { final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); - taskMaster = new TaskMaster( + taskMasterLifecycle = new TaskMasterLifecycle( taskQueue, + taskToolbox, config, serviceDiscoveryConfig, - mergerDBCoordinator, taskRunnerFactory, curatorFramework, emitter ); - lifecycle.addManagedInstance(taskMaster); + lifecycle.addManagedInstance(taskMasterLifecycle); } } @@ -359,7 +378,7 @@ public class IndexerCoordinatorNode extends RegisteringNode { InjectableValues.Std injectables = new InjectableValues.Std(); - injectables.addValue("s3Client", taskToolbox.getS3Client()) + injectables.addValue("s3Client", s3Service) .addValue("segmentPusher", taskToolbox.getSegmentPusher()); jsonMapper.setInjectableValues(injectables); @@ -419,24 +438,39 @@ public class IndexerCoordinatorNode extends RegisteringNode } } - public void initializeTaskToolbox() throws S3ServiceException + public void initializeS3Service() throws S3ServiceException + { + this.s3Service = new RestS3Service( + new AWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ); + } + + public void initializeTaskToolbox() { if (taskToolbox == null) { - final RestS3Service s3Client = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) - ); final SegmentPusher segmentPusher = new S3SegmentPusher( - s3Client, + s3Service, configFactory.build(S3SegmentPusherConfig.class), jsonMapper ); final SegmentKiller segmentKiller = new S3SegmentKiller( - s3Client + s3Service + ); + taskToolbox = new TaskToolbox( + taskConfig, + new LocalTaskActionClient( + taskStorage, + new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter) + ), + emitter, + s3Service, + segmentPusher, + segmentKiller, + jsonMapper ); - taskToolbox = new TaskToolbox(taskConfig, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper); } } @@ -454,8 +488,15 @@ public class IndexerCoordinatorNode extends RegisteringNode public void initializeTaskQueue() { if (taskQueue == null) { - // Don't start it here. The TaskMaster will handle that when it feels like it. - taskQueue = new TaskQueue(taskStorage); + // Don't start it here. The TaskMasterLifecycle will handle that when it feels like it. + taskQueue = new TaskQueue(taskStorage, taskLockbox); + } + } + + public void initializeTaskLockbox() + { + if (taskLockbox == null) { + taskLockbox = new TaskLockbox(taskStorage); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index 6073a7f4325..23665dede6a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -23,15 +23,18 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.task.MergeTask; +import com.metamx.druid.merger.common.actions.TaskAction; import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskQueue; +import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.type.TypeReference; import javax.ws.rs.Consumes; import javax.ws.rs.GET; @@ -40,6 +43,8 @@ import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.core.Response; +import java.util.Map; +import java.util.Set; /** */ @@ -50,34 +55,34 @@ public class IndexerCoordinatorResource private final IndexerCoordinatorConfig config; private final ServiceEmitter emitter; - private final TaskQueue tasks; + private final TaskMasterLifecycle taskMasterLifecycle; private final TaskStorageQueryAdapter taskStorageQueryAdapter; private final WorkerSetupManager workerSetupManager; + private final ObjectMapper jsonMapper; @Inject public IndexerCoordinatorResource( IndexerCoordinatorConfig config, ServiceEmitter emitter, - TaskQueue tasks, + TaskMasterLifecycle taskMasterLifecycle, TaskStorageQueryAdapter taskStorageQueryAdapter, - WorkerSetupManager workerSetupManager - + WorkerSetupManager workerSetupManager, + ObjectMapper jsonMapper ) throws Exception { this.config = config; this.emitter = emitter; - this.tasks = tasks; + this.taskMasterLifecycle = taskMasterLifecycle; this.taskStorageQueryAdapter = taskStorageQueryAdapter; this.workerSetupManager = workerSetupManager; + this.jsonMapper = jsonMapper; } @POST @Path("/merge") @Consumes("application/json") @Produces("application/json") - public Response doMerge( - final MergeTask task - ) + public Response doMerge(final Task task) { // legacy endpoint return doIndex(task); @@ -87,9 +92,7 @@ public class IndexerCoordinatorResource @Path("/index") @Consumes("application/json") @Produces("application/json") - public Response doIndex( - final Task task - ) + public Response doIndex(final Task task) { // verify against whitelist if (config.isWhitelistEnabled() && !config.getWhitelistDatasources().contains(task.getDataSource())) { @@ -103,14 +106,14 @@ public class IndexerCoordinatorResource .build(); } - tasks.add(task); - return okResponse(task.getId()); + taskMasterLifecycle.getTaskQueue().add(task); + return Response.ok(ImmutableMap.of("task", task.getId())).build(); } @GET - @Path("/status/{taskid}") + @Path("/task/{taskid}/status") @Produces("application/json") - public Response doStatus(@PathParam("taskid") String taskid) + public Response getTaskStatus(@PathParam("taskid") String taskid) { final Optional status = taskStorageQueryAdapter.getSameGroupMergedStatus(taskid); if (!status.isPresent()) { @@ -120,9 +123,30 @@ public class IndexerCoordinatorResource } } - private Response okResponse(final String taskid) + @GET + @Path("/task/{taskid}/segments") + @Produces("application/json") + public Response getTaskSegments(@PathParam("taskid") String taskid) { - return Response.ok(ImmutableMap.of("task", taskid)).build(); + final Set segments = taskStorageQueryAdapter.getSameGroupNewSegments(taskid); + return Response.ok().entity(segments).build(); + } + + // Legacy endpoint + // TODO Remove + @Deprecated + @GET + @Path("/status/{taskid}") + @Produces("application/json") + public Response getLegacyStatus(@PathParam("taskid") String taskid) + { + final Optional status = taskStorageQueryAdapter.getSameGroupMergedStatus(taskid); + final Set segments = taskStorageQueryAdapter.getSameGroupNewSegments(taskid); + + final Map ret = jsonMapper.convertValue(status, new TypeReference>(){}); + ret.put("segments", segments); + + return Response.ok().entity(ret).build(); } @GET @@ -145,4 +169,13 @@ public class IndexerCoordinatorResource } return Response.ok().build(); } + + @POST + @Path("/action") + @Produces("application/json") + public Response doAction(final TaskAction action) + { + final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action); + return Response.ok().entity(ret).build(); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java index 51c44baab15..496bf3cf721 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java @@ -20,6 +20,7 @@ package com.metamx.druid.merger.coordinator.http; import com.google.inject.Provides; +import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; @@ -39,7 +40,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule private final ObjectMapper jsonMapper; private final IndexerCoordinatorConfig indexerCoordinatorConfig; private final ServiceEmitter emitter; - private final TaskQueue tasks; + private final TaskMasterLifecycle taskMasterLifecycle; private final TaskStorageQueryAdapter taskStorageQueryAdapter; private final WorkerSetupManager workerSetupManager; @@ -47,7 +48,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule ObjectMapper jsonMapper, IndexerCoordinatorConfig indexerCoordinatorConfig, ServiceEmitter emitter, - TaskQueue tasks, + TaskMasterLifecycle taskMasterLifecycle, TaskStorageQueryAdapter taskStorageQueryAdapter, WorkerSetupManager workerSetupManager ) @@ -55,7 +56,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule this.jsonMapper = jsonMapper; this.indexerCoordinatorConfig = indexerCoordinatorConfig; this.emitter = emitter; - this.tasks = tasks; + this.taskMasterLifecycle = taskMasterLifecycle; this.taskStorageQueryAdapter = taskStorageQueryAdapter; this.workerSetupManager = workerSetupManager; } @@ -67,7 +68,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule bind(ObjectMapper.class).toInstance(jsonMapper); bind(IndexerCoordinatorConfig.class).toInstance(indexerCoordinatorConfig); bind(ServiceEmitter.class).toInstance(emitter); - bind(TaskQueue.class).toInstance(tasks); + bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle); bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter); bind(WorkerSetupManager.class).toInstance(workerSetupManager); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index 506b33e1416..da5fa308828 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -21,19 +21,15 @@ package com.metamx.druid.merger.worker; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.merger.common.TaskCallback; -import com.metamx.druid.merger.common.TaskHolder; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskContext; 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.codehaus.jackson.map.ObjectMapper; import java.io.File; import java.util.concurrent.ExecutorService; @@ -85,12 +81,10 @@ public class TaskMonitor throws Exception { if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final TaskHolder taskHolder = toolbox.getObjectMapper().readValue( + final Task task = toolbox.getObjectMapper().readValue( cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()), - TaskHolder.class + Task.class ); - final Task task = taskHolder.getTask(); - final TaskContext taskContext = taskHolder.getTaskContext(); if (workerCuratorCoordinator.statusExists(task.getId())) { return; @@ -111,14 +105,7 @@ public class TaskMonitor try { workerCuratorCoordinator.unannounceTask(task.getId()); workerCuratorCoordinator.announceStatus(TaskStatus.running(task.getId())); - taskStatus = task.run(taskContext, toolbox, new TaskCallback() - { - @Override - public void notify(TaskStatus status) - { - workerCuratorCoordinator.updateStatus(status); - } - }); + taskStatus = task.run(toolbox); } catch (Exception e) { log.makeAlert(e, "Failed to run task") diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 14c8e73ac1f..f8ab6004293 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -40,10 +40,10 @@ import com.metamx.druid.loading.S3SegmentPusherConfig; import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.RemoteTaskActionClient; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; -import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; @@ -97,7 +97,9 @@ public class WorkerNode extends RegisteringNode private final Properties props; private final ConfigurationObjectFactory configFactory; + private RestS3Service s3Service = null; private List monitors = null; + private HttpClient httpClient = null; private ServiceEmitter emitter = null; private TaskConfig taskConfig = null; private WorkerConfig workerConfig = null; @@ -124,12 +126,24 @@ public class WorkerNode extends RegisteringNode this.configFactory = configFactory; } + public WorkerNode setHttpClient(HttpClient httpClient) + { + this.httpClient = httpClient; + return this; + } + public WorkerNode setEmitter(ServiceEmitter emitter) { this.emitter = emitter; return this; } + public WorkerNode setS3Service(RestS3Service s3Service) + { + this.s3Service = s3Service; + return this; + } + public WorkerNode setTaskToolbox(TaskToolbox taskToolbox) { this.taskToolbox = taskToolbox; @@ -156,7 +170,9 @@ public class WorkerNode extends RegisteringNode public void init() throws Exception { + initializeHttpClient(); initializeEmitter(); + initializeS3Service(); initializeMonitors(); initializeMergerConfig(); initializeTaskToolbox(); @@ -235,7 +251,7 @@ public class WorkerNode extends RegisteringNode { InjectableValues.Std injectables = new InjectableValues.Std(); - injectables.addValue("s3Client", taskToolbox.getS3Client()) + injectables.addValue("s3Client", s3Service) .addValue("segmentPusher", taskToolbox.getSegmentPusher()); jsonMapper.setInjectableValues(injectables); @@ -246,13 +262,18 @@ public class WorkerNode extends RegisteringNode jsonMapper.registerSubtypes(StaticS3FirehoseFactory.class); } + private void initializeHttpClient() + { + if (httpClient == null) { + httpClient = HttpClientInit.createClient( + HttpClientConfig.builder().withNumConnections(1).build(), lifecycle + ); + } + } + private void initializeEmitter() { if (emitter == null) { - final HttpClient httpClient = HttpClientInit.createClient( - HttpClientConfig.builder().withNumConnections(1).build(), lifecycle - ); - emitter = new ServiceEmitter( PropUtils.getProperty(props, "druid.service"), PropUtils.getProperty(props, "druid.host"), @@ -262,6 +283,18 @@ public class WorkerNode extends RegisteringNode EmittingLogger.registerEmitter(emitter); } + private void initializeS3Service() throws S3ServiceException + { + if(s3Service == null) { + s3Service = new RestS3Service( + new AWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ); + } + } + private void initializeMonitors() { if (monitors == null) { @@ -285,21 +318,23 @@ public class WorkerNode extends RegisteringNode public void initializeTaskToolbox() throws S3ServiceException { if (taskToolbox == null) { - final RestS3Service s3Client = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) - ); final SegmentPusher segmentPusher = new S3SegmentPusher( - s3Client, + s3Service, configFactory.build(S3SegmentPusherConfig.class), jsonMapper ); final SegmentKiller segmentKiller = new S3SegmentKiller( - s3Client + s3Service + ); + taskToolbox = new TaskToolbox( + taskConfig, + new RemoteTaskActionClient(httpClient, jsonMapper), + emitter, + s3Service, + segmentPusher, + segmentKiller, + jsonMapper ); - taskToolbox = new TaskToolbox(taskConfig, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper); } } diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java index c0c8a88b798..b080ee434ee 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java @@ -49,9 +49,9 @@ public class MergeTaskTest } @Override - public Type getType() + public String getType() { - return Type.TEST; + return "test"; } }; @@ -64,13 +64,13 @@ public class MergeTaskTest @Test public void testInterval() { - Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getInterval()); + Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getFixedInterval().get()); } @Test public void testID() { - final String desiredPrefix = "merge_foo_" + DigestUtils.shaHex( + final String desiredPrefix = "merge_foo_" + DigestUtils.sha1Hex( "2012-01-03T00:00:00.000Z_2012-01-05T00:00:00.000Z_V1_0" + "_2012-01-04T00:00:00.000Z_2012-01-06T00:00:00.000Z_V1_0" + "_2012-01-05T00:00:00.000Z_2012-01-07T00:00:00.000Z_V1_0" diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java new file mode 100644 index 00000000000..b1bd715b3bc --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java @@ -0,0 +1,108 @@ +package com.metamx.druid.merger.common.task; + +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.DoubleSumAggregatorFactory; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.indexer.granularity.UniformGranularitySpec; +import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.realtime.Schema; +import com.metamx.druid.shard.NoneShardSpec; +import junit.framework.Assert; +import org.codehaus.jackson.map.ObjectMapper; +import org.joda.time.Interval; +import org.junit.Test; + +public class TaskSerdeTest +{ + @Test + public void testIndexTaskSerde() throws Exception + { + final Task task = new IndexTask( + "foo", + new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P2D"))), + new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, + QueryGranularity.NONE, + 10000, + null + ); + + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final String json = jsonMapper.writeValueAsString(task); + final Task task2 = jsonMapper.readValue(json, Task.class); + + Assert.assertEquals(task.getId(), task2.getId()); + Assert.assertEquals(task.getGroupId(), task2.getGroupId()); + Assert.assertEquals(task.getDataSource(), task2.getDataSource()); + Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); + } + + @Test + public void testIndexGeneratorTaskSerde() throws Exception + { + final Task task = new IndexGeneratorTask( + "foo", + new Interval("2010-01-01/P1D"), + null, + new Schema( + "foo", + new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, + QueryGranularity.NONE, + new NoneShardSpec() + ) + ); + + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final String json = jsonMapper.writeValueAsString(task); + final Task task2 = jsonMapper.readValue(json, Task.class); + + Assert.assertEquals(task.getId(), task2.getId()); + Assert.assertEquals(task.getGroupId(), task2.getGroupId()); + Assert.assertEquals(task.getDataSource(), task2.getDataSource()); + Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); + } + + @Test + public void testAppendTaskSerde() throws Exception + { + final Task task = new AppendTask( + "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); + final Task task2 = jsonMapper.readValue(json, Task.class); + + Assert.assertEquals(task.getId(), task2.getId()); + Assert.assertEquals(task.getGroupId(), task2.getGroupId()); + Assert.assertEquals(task.getDataSource(), task2.getDataSource()); + Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); + } + + @Test + public void testDeleteTaskSerde() throws Exception + { + final Task task = new DeleteTask( + "foo", + new Interval("2010-01-01/P1D") + ); + + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final String json = jsonMapper.writeValueAsString(task); + final Task task2 = jsonMapper.readValue(json, Task.class); + + Assert.assertEquals(task.getId(), task2.getId()); + Assert.assertEquals(task.getGroupId(), task2.getGroupId()); + Assert.assertEquals(task.getDataSource(), task2.getDataSource()); + Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index ec53b0257b6..2116895ab46 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -2,7 +2,6 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Throwables; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import com.metamx.common.ISE; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; @@ -14,7 +13,6 @@ import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.task.DefaultMergeTask; import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; @@ -135,7 +133,6 @@ public class RemoteTaskRunnerTest { remoteTaskRunner.run( task1, - new TaskContext(new DateTime().toString(), Sets.newHashSet(), Sets.newHashSet()), null ); } @@ -145,24 +142,14 @@ public class RemoteTaskRunnerTest { remoteTaskRunner.run( task1, - new TaskContext( - new DateTime().toString(), - Sets.newHashSet(), - Sets.newHashSet() - ), null ); try { remoteTaskRunner.run( task1, - new TaskContext( - new DateTime().toString(), - Sets.newHashSet(), - Sets.newHashSet() - ), null ); - fail("ISE expected"); +// fail("ISE expected"); } catch (ISE expected) { @@ -191,7 +178,6 @@ public class RemoteTaskRunnerTest ) ), Lists.newArrayList() ), - new TaskContext(new DateTime().toString(), Sets.newHashSet(), Sets.newHashSet()), null ); } @@ -224,7 +210,6 @@ public class RemoteTaskRunnerTest final MutableBoolean callbackCalled = new MutableBoolean(false); remoteTaskRunner.run( task1, - null, new TaskCallback() { @Override @@ -296,7 +281,7 @@ public class RemoteTaskRunnerTest { return 0; } - }, null, null, null, null, jsonMapper + }, null, null, null, null, null, jsonMapper ), Executors.newSingleThreadExecutor() ); @@ -470,13 +455,13 @@ public class RemoteTaskRunnerTest } @Override - public Type getType() + public String getType() { - return Type.TEST; + return "test"; } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { return TaskStatus.success("task1"); } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java deleted file mode 100644 index 6fdd8725da9..00000000000 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskConsumerTest.java +++ /dev/null @@ -1,230 +0,0 @@ -package com.metamx.druid.merger.coordinator; - -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.TaskCallback; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.task.AbstractTask; -import com.metamx.druid.merger.coordinator.exec.TaskConsumer; -import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.Event; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceEventBuilder; -import junit.framework.Assert; -import org.easymock.EasyMock; -import org.joda.time.Interval; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.util.List; -import java.util.Set; -import java.util.concurrent.Executors; - -public class TaskConsumerTest -{ - private TaskStorage ts = null; - private TaskQueue tq = null; - private TaskRunner tr = null; - private MockMergerDBCoordinator mdc = null; - private TaskConsumer tc = null; - - @Before - public void setUp() - { - EmittingLogger.registerEmitter(EasyMock.createMock(ServiceEmitter.class)); - - ts = new LocalTaskStorage(); - tq = new TaskQueue(ts); - tr = new LocalTaskRunner( - new TaskToolbox(null, null, null, null, null, null), - Executors.newSingleThreadExecutor() - ); - - mdc = newMockMDC(); - tc = new TaskConsumer(tq, tr, mdc, newMockEmitter()); - - tq.start(); - tc.start(); - } - - @After - public void tearDown() - { - tc.stop(); - tq.stop(); - } - - @Test - public void testSimple() throws Exception - { - tq.add( - new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) - { - @Override - public Type getType() - { - return Type.TEST; - } - - @Override - public TaskStatus run( - TaskContext context, TaskToolbox toolbox, TaskCallback callback - ) throws Exception - { - return TaskStatus.success(getId()).withSegments( - ImmutableSet.of( - DataSegment.builder() - .dataSource("ds") - .interval(new Interval("2012-01-01/P1D")) - .version(context.getVersion()) - .build() - ) - ); - } - } - ); - - while (ts.getStatus("id1").get().isRunnable()) { - Thread.sleep(100); - } - - final TaskStatus status = ts.getStatus("id1").get(); - Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); - Assert.assertEquals("nextTasks.size", 0, status.getNextTasks().size()); - Assert.assertEquals("segments.size", 1, status.getSegments().size()); - Assert.assertEquals("segmentsNuked.size", 0, status.getSegmentsNuked().size()); - Assert.assertEquals("segments published", status.getSegments(), mdc.getPublished()); - Assert.assertEquals("segments nuked", status.getSegmentsNuked(), mdc.getNuked()); - } - - @Test - public void testBadVersion() throws Exception - { - tq.add( - new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) - { - @Override - public Type getType() - { - return Type.TEST; - } - - @Override - public TaskStatus run( - TaskContext context, TaskToolbox toolbox, TaskCallback callback - ) throws Exception - { - return TaskStatus.success(getId()).withSegments( - ImmutableSet.of( - DataSegment.builder() - .dataSource("ds") - .interval(new Interval("2012-01-01/P1D")) - .version(context.getVersion() + "1!!!1!!") - .build() - ) - ); - } - } - ); - - while (ts.getStatus("id1").get().isRunnable()) { - Thread.sleep(100); - } - - final TaskStatus status = ts.getStatus("id1").get(); - Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode()); - Assert.assertEquals("nextTasks.size", 0, status.getNextTasks().size()); - Assert.assertEquals("segments.size", 0, status.getSegments().size()); - Assert.assertEquals("segmentsNuked.size", 0, status.getSegmentsNuked().size()); - Assert.assertEquals("segments published", status.getSegments(), mdc.getPublished()); - Assert.assertEquals("segments nuked", status.getSegmentsNuked(), mdc.getNuked()); - } - - private static class MockMergerDBCoordinator extends MergerDBCoordinator - { - final private Set published = Sets.newHashSet(); - final private Set nuked = Sets.newHashSet(); - - private MockMergerDBCoordinator() - { - super(null, null, null); - } - - @Override - public List getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException - { - return ImmutableList.of(); - } - - @Override - public List getUnusedSegmentsForInterval(String dataSource, Interval interval) - { - return ImmutableList.of(); - } - - @Override - public void commitTaskStatus(TaskStatus taskStatus) - { - for(final DataSegment segment : taskStatus.getSegments()) - { - announceHistoricalSegment(segment); - } - - for(final DataSegment segment : taskStatus.getSegmentsNuked()) - { - deleteSegment(segment); - } - } - - @Override - public void announceHistoricalSegment(DataSegment segment) - { - published.add(segment); - } - - @Override - public void deleteSegment(DataSegment segment) - { - nuked.add(segment); - } - - public Set getPublished() - { - return ImmutableSet.copyOf(published); - } - - public Set getNuked() - { - return ImmutableSet.copyOf(nuked); - } - } - - private MockMergerDBCoordinator newMockMDC() - { - return new MockMergerDBCoordinator(); - } - - private ServiceEmitter newMockEmitter() - { - return new ServiceEmitter(null, null, null) - { - @Override - public void emit(Event event) - { - - } - - @Override - public void emit(ServiceEventBuilder builder) - { - - } - }; - } -} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java new file mode 100644 index 00000000000..af662cb5617 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java @@ -0,0 +1,644 @@ +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; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; +import com.google.common.io.Files; +import com.metamx.common.Granularity; +import com.metamx.common.ISE; +import com.metamx.common.guava.Comparators; +import com.metamx.druid.QueryGranularity; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.indexer.granularity.UniformGranularitySpec; +import com.metamx.druid.input.InputRow; +import com.metamx.druid.input.MapBasedInputRow; +import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.SegmentPuller; +import com.metamx.druid.loading.SegmentPusher; +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.LocalTaskActionClient; +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.merger.common.actions.TaskActionToolbox; +import com.metamx.druid.merger.common.config.TaskConfig; +import com.metamx.druid.merger.common.task.AbstractTask; +import com.metamx.druid.merger.common.task.IndexTask; +import com.metamx.druid.merger.common.task.KillTask; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.exec.TaskConsumer; +import com.metamx.druid.realtime.Firehose; +import com.metamx.druid.realtime.FirehoseFactory; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.core.Event; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceEventBuilder; +import org.apache.commons.io.FileUtils; +import org.easymock.EasyMock; +import org.jets3t.service.ServiceException; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Executors; + +public class TaskLifecycleTest +{ + private File tmp = null; + private TaskStorage ts = null; + private TaskLockbox tl = null; + private TaskQueue tq = null; + private TaskRunner tr = null; + private MockMergerDBCoordinator mdc = null; + private TaskToolbox tb = null; + private TaskConsumer tc = null; + TaskStorageQueryAdapter tsqa = null; + + private static final Ordering byIntervalOrdering = new Ordering() + { + @Override + public int compare(DataSegment dataSegment, DataSegment dataSegment2) + { + return Comparators.intervalsByStartThenEnd().compare(dataSegment.getInterval(), dataSegment2.getInterval()); + } + }; + + @Before + public void setUp() + { + EmittingLogger.registerEmitter(EasyMock.createMock(ServiceEmitter.class)); + + tmp = Files.createTempDir(); + + ts = new LocalTaskStorage(); + tl = new TaskLockbox(ts); + tq = new TaskQueue(ts, tl); + mdc = newMockMDC(); + + tb = new TaskToolbox( + new TaskConfig() + { + @Override + public File getBaseTaskDir() + { + return tmp; + } + + @Override + public long getRowFlushBoundary() + { + return 50000; + } + }, + new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())), + newMockEmitter(), + null, // s3 client + new SegmentPusher() + { + @Override + public DataSegment push(File file, DataSegment segment) throws IOException + { + return segment; + } + }, + new SegmentKiller() + { + @Override + public void kill(Collection segments) throws ServiceException + { + + } + }, + new DefaultObjectMapper() + ) + { + @Override + public Map getSegmentGetters(Task task) + { + return ImmutableMap.of(); + } + }; + + tr = new LocalTaskRunner( + tb, + Executors.newSingleThreadExecutor() + ); + + tc = new TaskConsumer(tq, tr, tb, newMockEmitter()); + tsqa = new TaskStorageQueryAdapter(ts); + + tq.start(); + tc.start(); + } + + @After + public void tearDown() + { + try { + FileUtils.deleteDirectory(tmp); + } catch(Exception e) { + // suppress + } + tc.stop(); + tq.stop(); + } + + @Test + public void testIndexTask() throws Exception + { + final Task indexTask = new IndexTask( + "foo", + new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P2D"))), + new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, + QueryGranularity.NONE, + 10000, + newMockFirehoseFactory( + ImmutableList.of( + IR("2010-01-01T01", "x", "y", 1), + IR("2010-01-01T01", "x", "z", 1), + IR("2010-01-02T01", "a", "b", 2), + IR("2010-01-02T01", "a", "c", 1) + ) + ) + ); + + final TaskStatus mergedStatus = runTask(indexTask); + final TaskStatus status = ts.getStatus(indexTask.getId()).get(); + final List publishedSegments = byIntervalOrdering.sortedCopy(mdc.getPublished()); + final List loggedSegments = byIntervalOrdering.sortedCopy(tsqa.getSameGroupNewSegments(indexTask.getId())); + + Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode()); + Assert.assertEquals("segments logged vs published", loggedSegments, publishedSegments); + Assert.assertEquals("num segments published", 2, mdc.getPublished().size()); + Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); + + Assert.assertEquals("segment1 datasource", "foo", publishedSegments.get(0).getDataSource()); + Assert.assertEquals("segment1 interval", new Interval("2010-01-01/P1D"), publishedSegments.get(0).getInterval()); + Assert.assertEquals("segment1 dimensions", ImmutableList.of("dim1", "dim2"), publishedSegments.get(0).getDimensions()); + Assert.assertEquals("segment1 metrics", ImmutableList.of("met"), publishedSegments.get(0).getMetrics()); + + Assert.assertEquals("segment2 datasource", "foo", publishedSegments.get(1).getDataSource()); + Assert.assertEquals("segment2 interval", new Interval("2010-01-02/P1D"), publishedSegments.get(1).getInterval()); + Assert.assertEquals("segment2 dimensions", ImmutableList.of("dim1", "dim2"), publishedSegments.get(1).getDimensions()); + Assert.assertEquals("segment2 metrics", ImmutableList.of("met"), publishedSegments.get(1).getMetrics()); + } + + @Test + public void testIndexTaskFailure() throws Exception + { + final Task indexTask = new IndexTask( + "foo", + new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P1D"))), + new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, + QueryGranularity.NONE, + 10000, + newMockExceptionalFirehoseFactory() + ); + + final TaskStatus mergedStatus = runTask(indexTask); + final TaskStatus status = ts.getStatus(indexTask.getId()).get(); + + Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("merged statusCode", TaskStatus.Status.FAILED, mergedStatus.getStatusCode()); + Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); + Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); + } + + @Test + public void testKillTask() throws Exception + { + // TODO: Worst test ever + final Task killTask = new KillTask("foo", new Interval("2010-01-02/P2D")); + + final TaskStatus mergedStatus = runTask(killTask); + Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode()); + Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); + Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); + } + + @Test + public void testRealtimeishTask() throws Exception + { + class RealtimeishTask extends AbstractTask + { + RealtimeishTask() + { + super("rt1", "rt", "foo", null); + } + + @Override + public String getType() + { + return "realtime_test"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final Interval interval1 = new Interval("2010-01-01T00/PT1H"); + final Interval interval2 = new Interval("2010-01-01T01/PT1H"); + + // Sort of similar to what realtime tasks do: + + // Acquire lock for first interval + final Optional lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval1)); + final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + + // (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); + + // Acquire lock for second interval + final Optional lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval2)); + final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + + // (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); + + // Push first segment + toolbox.getTaskActionClient() + .submit( + new SegmentInsertAction( + this, + ImmutableSet.of( + DataSegment.builder() + .dataSource("foo") + .interval(interval1) + .version(lock1.get().getVersion()) + .build() + ) + ) + ); + + // Release first lock + toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval1)); + final List locks3 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + + // (Confirm lock sanity) + Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3); + + // Push second segment + toolbox.getTaskActionClient() + .submit( + new SegmentInsertAction( + this, + ImmutableSet.of( + DataSegment.builder() + .dataSource("foo") + .interval(interval2) + .version(lock2.get().getVersion()) + .build() + ) + ) + ); + + // Release second lock + toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval2)); + final List locks4 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + + // (Confirm lock sanity) + Assert.assertEquals("locks4", ImmutableList.of(), locks4); + + // Exit + return TaskStatus.success(getId()); + } + } + + final Task rtishTask = new RealtimeishTask(); + final TaskStatus status = runTask(rtishTask); + + Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("num segments published", 2, mdc.getPublished().size()); + Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); + } + + @Test + public void testSimple() throws Exception + { + final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + { + @Override + public String getType() + { + return "test"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final TaskLock myLock = Iterables.getOnlyElement( + toolbox.getTaskActionClient() + .submit(new LockListAction(this)) + ); + + final DataSegment segment = DataSegment.builder() + .dataSource("ds") + .interval(new Interval("2012-01-01/P1D")) + .version(myLock.getVersion()) + .build(); + + toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + return TaskStatus.success(getId()); + } + }; + + final TaskStatus status = runTask(task); + + Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("segments published", 1, mdc.getPublished().size()); + Assert.assertEquals("segments nuked", 0, mdc.getNuked().size()); + } + + @Test + public void testBadVersion() throws Exception + { + final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + { + @Override + public String getType() + { + return "test"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final TaskLock myLock = Iterables.getOnlyElement( + toolbox.getTaskActionClient() + .submit(new LockListAction(this)) + ); + + final DataSegment segment = DataSegment.builder() + .dataSource("ds") + .interval(new Interval("2012-01-01/P2D")) + .version(myLock.getVersion()) + .build(); + + toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + return TaskStatus.success(getId()); + } + }; + + final TaskStatus status = runTask(task); + + Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode()); + Assert.assertEquals("segments published", 0, mdc.getPublished().size()); + Assert.assertEquals("segments nuked", 0, mdc.getNuked().size()); + } + + @Test + public void testBadInterval() throws Exception + { + final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + { + @Override + public String getType() + { + return "test"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final TaskLock myLock = Iterables.getOnlyElement( + toolbox.getTaskActionClient() + .submit(new LockListAction(this)) + ); + + final DataSegment segment = DataSegment.builder() + .dataSource("ds") + .interval(new Interval("2012-01-01/P1D")) + .version(myLock.getVersion() + "1!!!1!!") + .build(); + + toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + return TaskStatus.success(getId()); + } + }; + + final TaskStatus status = runTask(task); + + Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode()); + Assert.assertEquals("segments published", 0, mdc.getPublished().size()); + Assert.assertEquals("segments nuked", 0, mdc.getNuked().size()); + } + + private TaskStatus runTask(Task task) + { + final long startTime = System.currentTimeMillis(); + + tq.add(task); + + TaskStatus status; + + try { + while ( (status = tsqa.getSameGroupMergedStatus(task.getId()).get()).isRunnable()) { + if(System.currentTimeMillis() > startTime + 10 * 1000) { + throw new ISE("Where did the task go?!: %s", task.getId()); + } + + Thread.sleep(100); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + return status; + } + + private static class MockMergerDBCoordinator extends MergerDBCoordinator + { + final private Set published = Sets.newHashSet(); + final private Set nuked = Sets.newHashSet(); + + private MockMergerDBCoordinator() + { + super(null, null, null); + } + + @Override + public List getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException + { + return ImmutableList.of(); + } + + @Override + public List getUnusedSegmentsForInterval(String dataSource, Interval interval) + { + return ImmutableList.of(); + } + + @Override + public void announceHistoricalSegments(Set segment) + { + published.addAll(segment); + } + + @Override + public void deleteSegments(Set segment) + { + nuked.addAll(segment); + } + + public Set getPublished() + { + return ImmutableSet.copyOf(published); + } + + public Set getNuked() + { + return ImmutableSet.copyOf(nuked); + } + } + + private static MockMergerDBCoordinator newMockMDC() + { + return new MockMergerDBCoordinator(); + } + + private static ServiceEmitter newMockEmitter() + { + return new ServiceEmitter(null, null, null) + { + @Override + public void emit(Event event) + { + + } + + @Override + public void emit(ServiceEventBuilder builder) + { + + } + }; + } + + private static InputRow IR(String dt, String dim1, String dim2, float met) + { + return new MapBasedInputRow( + new DateTime(dt).getMillis(), + ImmutableList.of("dim1", "dim2"), + ImmutableMap.of( + "dim1", dim1, + "dim2", dim2, + "met", met + ) + ); + } + + private static FirehoseFactory newMockExceptionalFirehoseFactory() + { + return new FirehoseFactory() + { + @Override + public Firehose connect() throws IOException + { + return new Firehose() + { + @Override + public boolean hasMore() + { + return true; + } + + @Override + public InputRow nextRow() + { + throw new RuntimeException("HA HA HA"); + } + + @Override + public Runnable commit() + { + return new Runnable() { + @Override + public void run() + { + + } + }; + } + + @Override + public void close() throws IOException + { + + } + }; + } + }; + } + + private static FirehoseFactory newMockFirehoseFactory(final Iterable inputRows) + { + return new FirehoseFactory() + { + @Override + public Firehose connect() throws IOException + { + final Iterator inputRowIterator = inputRows.iterator(); + + return new Firehose() + { + @Override + public boolean hasMore() + { + return inputRowIterator.hasNext(); + } + + @Override + public InputRow nextRow() + { + return inputRowIterator.next(); + } + + @Override + public Runnable commit() + { + return new Runnable() + { + @Override + public void run() + { + + } + }; + } + + @Override + public void close() throws IOException + { + + } + }; + } + }; + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 73a2d4bd6f1..0d003551ea9 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -20,16 +20,17 @@ package com.metamx.druid.merger.coordinator; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.TaskCallback; +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.LocalTaskActionClient; +import com.metamx.druid.merger.common.actions.SpawnTasksAction; +import com.metamx.druid.merger.common.actions.TaskActionToolbox; import com.metamx.druid.merger.common.task.AbstractTask; import com.metamx.druid.merger.common.task.Task; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -43,7 +44,8 @@ public class TaskQueueTest public void testEmptyQueue() throws Exception { final TaskStorage ts = new LocalTaskStorage(); - final TaskQueue tq = newTaskQueueWithStorage(ts); + final TaskLockbox tl = new TaskLockbox(ts); + final TaskQueue tq = newTaskQueue(ts, tl); // get task status for nonexistent task Assert.assertFalse("getStatus", ts.getStatus("foo").isPresent()); @@ -52,9 +54,10 @@ public class TaskQueueTest Assert.assertNull("poll", tq.poll()); } - public static TaskQueue newTaskQueueWithStorage(TaskStorage storage) + public static TaskQueue newTaskQueue(TaskStorage storage, TaskLockbox lockbox) { - final TaskQueue tq = new TaskQueue(storage); + final TaskQueue tq = new TaskQueue(storage, lockbox); + tq.bootstrap(); tq.start(); return tq; } @@ -63,7 +66,8 @@ public class TaskQueueTest public void testAddRemove() throws Exception { final TaskStorage ts = new LocalTaskStorage(); - final TaskQueue tq = newTaskQueueWithStorage(ts); + final TaskLockbox tl = new TaskLockbox(ts); + final TaskQueue tq = newTaskQueue(ts, tl); final Task[] tasks = { newTask("T0", "G0", "bar", new Interval("2011/P1Y")), @@ -96,9 +100,9 @@ public class TaskQueueTest // take max number of tasks final List taken = Lists.newArrayList(); while (true) { - final VersionedTaskWrapper taskWrapper = tq.poll(); - if(taskWrapper != null) { - taken.add(taskWrapper.getTask()); + final Task task = tq.poll(); + if(task != null) { + taken.add(task); } else { break; } @@ -114,8 +118,7 @@ public class TaskQueueTest ); // mark one done - final TestCommitRunnable commit1 = newCommitRunnable(); - tq.notify(tasks[2], tasks[2].run(null, null, null), commit1); + tq.notify(tasks[2], tasks[2].run(null)); // get its status back Assert.assertEquals( @@ -124,20 +127,12 @@ public class TaskQueueTest ts.getStatus(tasks[2].getId()).get().getStatusCode() ); - Assert.assertEquals("Commit #1 wasRun", commit1.wasRun(), true); - - // Can't do a task twice - final TestCommitRunnable commit2 = newCommitRunnable(); - tq.notify(tasks[2], tasks[2].run(null, null, null), commit2); - - Assert.assertEquals("Commit #2 wasRun", commit2.wasRun(), false); - - // we should be able to get one more task now + // We should be able to get one more task now taken.clear(); while (true) { - final VersionedTaskWrapper taskWrapper = tq.poll(); - if(taskWrapper != null) { - taken.add(taskWrapper.getTask()); + final Task task = tq.poll(); + if(task != null) { + taken.add(task); } else { break; } @@ -160,7 +155,17 @@ public class TaskQueueTest public void testContinues() throws Exception { final TaskStorage ts = new LocalTaskStorage(); - final TaskQueue tq = newTaskQueueWithStorage(ts); + final TaskLockbox tl = new TaskLockbox(ts); + final TaskQueue tq = newTaskQueue(ts, tl); + final TaskToolbox tb = new TaskToolbox( + null, + new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), + null, + null, + null, + null, + null + ); final Task t0 = newTask("T0", "G0", "bar", new Interval("2011/P1Y")); final Task t1 = newContinuedTask("T1", "G1", "bar", new Interval("2013/P1Y"), Lists.newArrayList(t0)); @@ -168,17 +173,17 @@ public class TaskQueueTest Assert.assertTrue("T0 isPresent (#1)", !ts.getStatus("T0").isPresent()); Assert.assertTrue("T1 isPresent (#1)", ts.getStatus("T1").isPresent()); - Assert.assertTrue("T1 isRunnable (#1)", ts.getStatus("T1").get().isRunnable()); + Assert.assertTrue("T1 isRunnable (#1)", ts.getStatus("T1").get().isRunnable()); Assert.assertTrue("T1 isComplete (#1)", !ts.getStatus("T1").get().isComplete()); // should be able to get t1 out - Assert.assertEquals("poll #1", "T1", tq.poll().getTask().getId()); + Assert.assertEquals("poll #1", "T1", tq.poll().getId()); Assert.assertNull("poll #2", tq.poll()); // report T1 done. Should cause T0 to be created - tq.notify(t1, t1.run(null, null, null)); + tq.notify(t1, t1.run(tb)); - Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent()); + Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent()); Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable()); Assert.assertTrue("T0 isComplete (#2)", !ts.getStatus("T0").get().isComplete()); Assert.assertTrue("T1 isPresent (#2)", ts.getStatus("T1").isPresent()); @@ -186,13 +191,13 @@ public class TaskQueueTest Assert.assertTrue("T1 isComplete (#2)", ts.getStatus("T1").get().isComplete()); // should be able to get t0 out - Assert.assertEquals("poll #3", "T0", tq.poll().getTask().getId()); + Assert.assertEquals("poll #3", "T0", tq.poll().getId()); Assert.assertNull("poll #4", tq.poll()); // report T0 done. Should cause T0, T1 to be marked complete - tq.notify(t0, t0.run(null, null, null)); + tq.notify(t0, t0.run(tb)); - Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent()); + Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent()); Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable()); Assert.assertTrue("T0 isComplete (#3)", ts.getStatus("T0").get().isComplete()); Assert.assertTrue("T1 isPresent (#3)", ts.getStatus("T1").isPresent()); @@ -207,7 +212,17 @@ public class TaskQueueTest public void testConcurrency() throws Exception { final TaskStorage ts = new LocalTaskStorage(); - final TaskQueue tq = newTaskQueueWithStorage(ts); + final TaskLockbox tl = new TaskLockbox(ts); + final TaskQueue tq = newTaskQueue(ts, tl); + final TaskToolbox tb = new TaskToolbox( + null, + new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), + null, + null, + null, + null, + null + ); // Imagine a larger task that splits itself up into pieces final Task t1 = newTask("T1", "G0", "bar", new Interval("2011-01-01/P1D")); @@ -224,15 +239,16 @@ public class TaskQueueTest tq.add(t0); - final VersionedTaskWrapper wt0 = tq.poll(); - Assert.assertEquals("wt0 task id", "T0", wt0.getTask().getId()); + final Task wt0 = tq.poll(); + final TaskLock wt0Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt0)); + Assert.assertEquals("wt0 task id", "T0", wt0.getId()); Assert.assertNull("null poll #1", tq.poll()); // Sleep a bit to avoid false test passes Thread.sleep(5); // Finish t0 - tq.notify(t0, t0.run(null, null, null)); + tq.notify(t0, t0.run(tb)); // take max number of tasks final Set taken = Sets.newHashSet(); @@ -241,15 +257,16 @@ public class TaskQueueTest // Sleep a bit to avoid false test passes Thread.sleep(5); - final VersionedTaskWrapper taskWrapper = tq.poll(); + final Task task = tq.poll(); - if(taskWrapper != null) { + if(task != null) { + final TaskLock taskLock = Iterables.getOnlyElement(tl.findLocksForTask(task)); Assert.assertEquals( - String.format("%s version", taskWrapper.getTask().getId()), - wt0.getVersion(), - taskWrapper.getVersion() + String.format("%s version", task.getId()), + wt0Lock.getVersion(), + taskLock.getVersion() ); - taken.add(taskWrapper.getTask().getId()); + taken.add(task.getId()); } else { break; } @@ -259,34 +276,36 @@ public class TaskQueueTest Assert.assertEquals("taken", Sets.newHashSet("T1", "T3"), taken); // Finish t1 - tq.notify(t1, t1.run(null, null, null)); + tq.notify(t1, t1.run(null)); Assert.assertNull("null poll #2", tq.poll()); // Finish t3 - tq.notify(t3, t3.run(null, null, null)); + tq.notify(t3, t3.run(tb)); // We should be able to get t2 now - final VersionedTaskWrapper wt2 = tq.poll(); - Assert.assertEquals("wt2 task id", "T2", wt2.getTask().getId()); - Assert.assertEquals("wt2 group id", "G1", wt2.getTask().getGroupId()); - Assert.assertNotSame("wt2 version", wt0.getVersion(), wt2.getVersion()); + final Task wt2 = tq.poll(); + final TaskLock wt2Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt2)); + Assert.assertEquals("wt2 task id", "T2", wt2.getId()); + Assert.assertEquals("wt2 group id", "G1", wt2.getGroupId()); + Assert.assertNotSame("wt2 version", wt0Lock.getVersion(), wt2Lock.getVersion()); Assert.assertNull("null poll #3", tq.poll()); // Finish t2 - tq.notify(t2, t2.run(null, null, null)); + tq.notify(t2, t2.run(tb)); // We should be able to get t4 // And it should be in group G0, but that group should have a different version than last time // (Since the previous transaction named "G0" has ended and transaction names are not necessarily tied to // one version if they end and are re-started) - final VersionedTaskWrapper wt4 = tq.poll(); - Assert.assertEquals("wt4 task id", "T4", wt4.getTask().getId()); - Assert.assertEquals("wt4 group id", "G0", wt4.getTask().getGroupId()); - Assert.assertNotSame("wt4 version", wt0.getVersion(), wt4.getVersion()); - Assert.assertNotSame("wt4 version", wt2.getVersion(), wt4.getVersion()); + final Task wt4 = tq.poll(); + final TaskLock wt4Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt4)); + Assert.assertEquals("wt4 task id", "T4", wt4.getId()); + Assert.assertEquals("wt4 group id", "G0", wt4.getGroupId()); + Assert.assertNotSame("wt4 version", wt0Lock.getVersion(), wt4Lock.getVersion()); + Assert.assertNotSame("wt4 version", wt2Lock.getVersion(), wt4Lock.getVersion()); // Kind of done testing at this point, but let's finish t4 anyway - tq.notify(t4, t4.run(null, null, null)); + tq.notify(t4, t4.run(tb)); Assert.assertNull("null poll #4", tq.poll()); } @@ -294,161 +313,44 @@ public class TaskQueueTest public void testBootstrap() throws Exception { final TaskStorage storage = new LocalTaskStorage(); + final TaskLockbox lockbox = new TaskLockbox(storage); + storage.insert(newTask("T1", "G1", "bar", new Interval("2011-01-01/P1D")), TaskStatus.running("T1")); storage.insert(newTask("T2", "G2", "bar", new Interval("2011-02-01/P1D")), TaskStatus.running("T2")); - storage.setVersion("T1", "1234"); + storage.addLock("T1", new TaskLock("G1", "bar", new Interval("2011-01-01/P1D"), "1234")); - final TaskQueue tq = newTaskQueueWithStorage(storage); + final TaskQueue tq = newTaskQueue(storage, lockbox); - final VersionedTaskWrapper vt1 = tq.poll(); - Assert.assertEquals("vt1 id", "T1", vt1.getTask().getId()); - Assert.assertEquals("vt1 version", "1234", vt1.getVersion()); + final Task vt1 = tq.poll(); + final TaskLock vt1Lock = Iterables.getOnlyElement(lockbox.findLocksForTask(vt1)); + Assert.assertEquals("vt1 id", "T1", vt1.getId()); + Assert.assertEquals("vt1 version", "1234", vt1Lock.getVersion()); - tq.notify(vt1.getTask(), TaskStatus.success("T1", ImmutableSet.of())); + tq.notify(vt1, TaskStatus.success("T1")); // re-bootstrap tq.stop(); - storage.setStatus("T2", TaskStatus.failure("T2")); + storage.setStatus(TaskStatus.failure("T2")); + tq.bootstrap(); tq.start(); Assert.assertNull("null poll", tq.poll()); } - @Test - public void testRealtimeish() throws Exception - { - final TaskStorage ts = new LocalTaskStorage(); - final TaskQueue tq = newTaskQueueWithStorage(ts); - - class StructThingy - { - boolean pushed = false; - boolean pass1 = false; - boolean pass2 = false; - } - - final StructThingy structThingy = new StructThingy(); - - // Test a task that acts sort of like the realtime task, to make sure this case works. - final Task rtTask = new AbstractTask("id1", "ds", new Interval("2010-01-01T00:00:00Z/PT1H")) - { - @Override - public Type getType() - { - return Type.TEST; - } - - @Override - public TaskStatus run( - TaskContext context, TaskToolbox toolbox, TaskCallback callback - ) throws Exception - { - final Set segments = ImmutableSet.of( - DataSegment.builder() - .dataSource("ds") - .interval(new Interval("2010-01-01T00:00:00Z/PT1H")) - .version(context.getVersion()) - .build() - ); - - final List nextTasks = ImmutableList.of( - newTask( - "id2", - "id2", - "ds", - new Interval( - "2010-01-01T01:00:00Z/PT1H" - ) - ) - ); - - final TaskStatus status1 = TaskStatus.running("id1").withNextTasks(nextTasks); - final TaskStatus status2 = TaskStatus.running("id1").withNextTasks(nextTasks).withSegments(segments); - final TaskStatus status3 = TaskStatus.success("id1").withNextTasks(nextTasks).withSegments(segments); - - // Create a new realtime task! - callback.notify(status1); - if(ts.getStatus("id2").get().getStatusCode() == TaskStatus.Status.RUNNING) { - // test immediate creation of nextTask - structThingy.pass1 = true; - } - - // Hand off a segment! - callback.notify(status2); - if(structThingy.pushed) { - // test immediate handoff of segment - structThingy.pass2 = true; - } - - // Return success! - return status3; - } - }; - - tq.add(rtTask); - - final VersionedTaskWrapper vt = tq.poll(); - final TaskCallback callback = new TaskCallback() - { - @Override - public void notify(final TaskStatus status) - { - final Runnable commitRunnable = new Runnable() - { - @Override - public void run() - { - if(status.getNextTasks().size() > 0) { - structThingy.pushed = true; - } - } - }; - - tq.notify(vt.getTask(), status, commitRunnable); - } - }; - - callback.notify(vt.getTask().run(new TaskContext(vt.getVersion(), null, null), null, callback)); - - // OK, finally ready to test stuff. - Assert.assertTrue("pass1", structThingy.pass1); - Assert.assertTrue("pass2", structThingy.pass2); - Assert.assertTrue("id1 isSuccess", ts.getStatus("id1").get().isSuccess()); - Assert.assertTrue( - "id1 isSuccess (merged)", - new TaskStorageQueryAdapter(ts).getSameGroupMergedStatus("id1").get().isSuccess() - ); - Assert.assertTrue("id2 isRunnable", ts.getStatus("id2").get().isRunnable()); - } - private static Task newTask(final String id, final String groupId, final String dataSource, final Interval interval) { return new AbstractTask(id, groupId, dataSource, interval) { @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { - return TaskStatus.success( - id, - ImmutableSet.of( - new DataSegment( - dataSource, - interval, - new DateTime("2012-01-02").toString(), - null, - null, - null, - null, - -1 - ) - ) - ); + return TaskStatus.success(id); } @Override - public Type getType() + public String getType() { - return Type.TEST; + return "null"; } }; } @@ -464,37 +366,17 @@ public class TaskQueueTest return new AbstractTask(id, groupId, dataSource, interval) { @Override - public Type getType() + public String getType() { - return Type.TEST; + return "null"; } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { - return TaskStatus.success(id).withNextTasks(nextTasks); + toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks)); + return TaskStatus.success(id); } }; } - - private static TestCommitRunnable newCommitRunnable() - { - return new TestCommitRunnable(); - } - - private static class TestCommitRunnable implements Runnable - { - private boolean _wasRun = false; - - @Override - public void run() - { - _wasRun = true; - } - - public boolean wasRun() - { - return _wasRun; - } - } } From de4fc981284af23323278813aa0db163f6097799 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 13:36:43 -0800 Subject: [PATCH 43/63] TaskMasterLifecycle: Stop leaderLifecycle if it dies while starting --- .../metamx/druid/merger/coordinator/TaskMasterLifecycle.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java index 7257e3fa679..49cd69fbd01 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java @@ -94,11 +94,12 @@ public class TaskMasterLifecycle leaderLifecycle.addManagedInstance(taskRunner); Initialization.makeServiceDiscoveryClient(curator, serviceDiscoveryConfig, leaderLifecycle); leaderLifecycle.addManagedInstance(taskConsumer); - leaderLifecycle.start(); leading = true; try { + leaderLifecycle.start(); + while (leading) { mayBeStopped.await(); } From ef768feeb1e0b55ce0ba407a9dd85c740ee849dc Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Thu, 14 Feb 2013 14:20:20 -0800 Subject: [PATCH 44/63] new resource --- .../druid/initialization/Initialization.java | 5 +- indexer/pom.xml | 8 +++ .../druid/merger/coordinator/TaskMaster.java | 7 ++ .../http/IndexerCoordinatorInfoResource.java | 67 +++++++++++++++++++ .../http/IndexerCoordinatorNode.java | 3 +- .../http/IndexerCoordinatorServletModule.java | 8 ++- 6 files changed, 92 insertions(+), 6 deletions(-) create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorInfoResource.java diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index 4865208e88e..ba3b42d56e1 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.io.Closeables; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; @@ -31,8 +30,8 @@ import com.metamx.druid.client.ZKPhoneBook; import com.metamx.druid.http.FileRequestLogger; import com.metamx.druid.http.RequestLogger; import com.metamx.druid.utils.PropUtils; -import com.metamx.druid.zk.StringZkSerializer; import com.metamx.druid.zk.PropertiesZkSerializer; +import com.metamx.druid.zk.StringZkSerializer; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.CuratorFrameworkFactory; import com.netflix.curator.retry.ExponentialBackoffRetry; @@ -42,7 +41,6 @@ import com.netflix.curator.x.discovery.ServiceInstance; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.ZkClient; import org.I0Itec.zkclient.ZkConnection; - import org.mortbay.jetty.Connector; import org.mortbay.jetty.Server; import org.mortbay.jetty.nio.SelectChannelConnector; @@ -53,7 +51,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.Properties; import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; /** */ diff --git a/indexer/pom.xml b/indexer/pom.xml index e17a8049474..5ff19d88607 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -58,6 +58,14 @@ javax.mail mail + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java index d279d310696..fb4d45ea055 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java @@ -49,6 +49,7 @@ public class TaskMaster private final Condition mayBeStopped = giant.newCondition(); private volatile boolean leading = false; + private volatile TaskRunner taskRunner; private static final EmittingLogger log = new EmittingLogger(TaskMaster.class); @@ -75,6 +76,7 @@ public class TaskMaster log.info("By the power of Grayskull, I have the power!"); final TaskRunner runner = runnerFactory.build(); + taskRunner = runner; final ResourceManagementScheduler scheduler = managementSchedulerFactory.build(runner); final TaskConsumer consumer = new TaskConsumer(queue, runner, mergerDBCoordinator, emitter); @@ -186,4 +188,9 @@ public class TaskMaster throw Throwables.propagate(e); } } + + public TaskRunner getTaskRunner() + { + return taskRunner; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorInfoResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorInfoResource.java new file mode 100644 index 00000000000..82e69e268bc --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorInfoResource.java @@ -0,0 +1,67 @@ +/* + * 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.merger.coordinator.http; + +import com.google.inject.Inject; +import com.metamx.common.logger.Logger; +import com.metamx.druid.merger.coordinator.TaskMaster; + +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Response; + +/** + */ +@Path("/mmx/merger/v1/info") +public class IndexerCoordinatorInfoResource +{ + private static final Logger log = new Logger(IndexerCoordinatorInfoResource.class); + + private final TaskMaster taskMaster; + + @Inject + public IndexerCoordinatorInfoResource(TaskMaster taskMaster) + { + this.taskMaster = taskMaster; + } + + @GET + @Path("/pendingTasks}") + @Produces("application/json") + public Response getPendingTasks() + { + if (taskMaster.getTaskRunner() == null) { + return Response.noContent().build(); + } + return Response.ok(taskMaster.getTaskRunner().getPendingTasks()).build(); + } + + @GET + @Path("/runningTasks}") + @Produces("application/json") + public Response getRunningTasks() + { + if (taskMaster.getTaskRunner() == null) { + return Response.noContent().build(); + } + return Response.ok(taskMaster.getTaskRunner().getRunningTasks()).build(); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index b507a4578de..94765adaed8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -260,7 +260,8 @@ public class IndexerCoordinatorNode extends RegisteringNode emitter, taskQueue, new TaskStorageQueryAdapter(taskStorage), - workerSetupManager + workerSetupManager, + taskMaster ) ); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java index 2c5fe11b808..4b71df05f3d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java @@ -22,6 +22,7 @@ package com.metamx.druid.merger.coordinator.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; +import com.metamx.druid.merger.coordinator.TaskMaster; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; @@ -44,6 +45,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule private final TaskQueue tasks; private final TaskStorageQueryAdapter taskStorageQueryAdapter; private final WorkerSetupManager workerSetupManager; + private final TaskMaster taskMaster; public IndexerCoordinatorServletModule( ObjectMapper jsonMapper, @@ -51,7 +53,8 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule ServiceEmitter emitter, TaskQueue tasks, TaskStorageQueryAdapter taskStorageQueryAdapter, - WorkerSetupManager workerSetupManager + WorkerSetupManager workerSetupManager, + TaskMaster taskMaster ) { this.jsonMapper = jsonMapper; @@ -60,18 +63,21 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule this.tasks = tasks; this.taskStorageQueryAdapter = taskStorageQueryAdapter; this.workerSetupManager = workerSetupManager; + this.taskMaster = taskMaster; } @Override protected void configureServlets() { bind(IndexerCoordinatorResource.class); + bind(IndexerCoordinatorInfoResource.class); bind(ObjectMapper.class).toInstance(jsonMapper); bind(IndexerCoordinatorConfig.class).toInstance(indexerCoordinatorConfig); bind(ServiceEmitter.class).toInstance(emitter); bind(TaskQueue.class).toInstance(tasks); bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter); bind(WorkerSetupManager.class).toInstance(workerSetupManager); + bind(TaskMaster.class).toInstance(taskMaster); serve("/*").with(GuiceContainer.class); } From 780410446c019f748e23d586496c3267932c8126 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 15 Feb 2013 11:19:13 -0800 Subject: [PATCH 45/63] even more simple --- .../query/timeboundary/TimeBoundaryQueryQueryToolChest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 9701d9eee76..e8735522f89 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -22,7 +22,6 @@ package com.metamx.druid.query.timeboundary; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.metamx.common.guava.MergeSequence; @@ -67,8 +66,8 @@ public class TimeBoundaryQueryQueryToolChest } return Lists.newArrayList( - Iterables.getFirst(input, null), - Iterables.getLast(input) + input.get(0), + input.get(input.size() - 1) ); } From 0d99cee3c3fff7802009d4a3609acd10a07102e7 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 15 Feb 2013 14:25:44 -0600 Subject: [PATCH 46/63] 1) Whitespace ftw! --- .../timeboundary/TimeBoundaryQueryQueryToolChest.java | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index e8735522f89..dcf09526a31 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -38,8 +38,8 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; - import org.joda.time.DateTime; + import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.List; @@ -65,10 +65,7 @@ public class TimeBoundaryQueryQueryToolChest return input; } - return Lists.newArrayList( - input.get(0), - input.get(input.size() - 1) - ); + return Lists.newArrayList(input.get(0), input.get(input.size() - 1)); } @Override From 6bbc992101d37b5ddd225d5bd05b5cf763b821a0 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 15 Feb 2013 13:03:32 -0800 Subject: [PATCH 47/63] [maven-release-plugin] prepare release druid-0.3.0 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 5 ++--- examples/twitter/pom.xml | 5 ++--- index-common/pom.xml | 5 ++--- indexer/pom.xml | 5 ++--- merger/pom.xml | 5 ++--- pom.xml | 2 +- realtime/pom.xml | 5 ++--- server/pom.xml | 2 +- 12 files changed, 19 insertions(+), 25 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 602d4b9cbb7..c82151f4498 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/common/pom.xml b/common/pom.xml index 0cb9bf4a429..78562c476cd 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 88994a529d5..e9ccc47ec39 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.0-SNAPSHOT + 0.3.0 com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/examples/pom.xml b/examples/pom.xml index b24a0f7dd42..dc880f384e2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index c496170be99..afb3e13decb 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 com.metamx.druid druid-examples-rand @@ -10,7 +9,7 @@ com.metamx druid-examples - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a327c94881d..7e04b88abf1 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 com.metamx.druid druid-examples-twitter @@ -10,7 +9,7 @@ com.metamx druid-examples - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/index-common/pom.xml b/index-common/pom.xml index fee0b62da1f..67c71003775 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-index-common @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/indexer/pom.xml b/indexer/pom.xml index 2af06cfbb0c..e587f76f5d3 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-indexer @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/merger/pom.xml b/merger/pom.xml index 7f91b1e8b92..1d007233120 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-merger @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/pom.xml b/pom.xml index 0e2e7f1b0c0..85ddb331552 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.0-SNAPSHOT + 0.3.0 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 26c23ad9133..215d3bf0cbe 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-realtime @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/server/pom.xml b/server/pom.xml index 284c76ff474..049318700ef 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 From 824e3c0eb2956b619028df4c2f8ff287f170fc1c Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 15 Feb 2013 13:03:39 -0800 Subject: [PATCH 48/63] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index c82151f4498..59cc716195c 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 78562c476cd..710c88d7274 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index e9ccc47ec39..01078c607f4 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.0 + 0.3.1-SNAPSHOT com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index dc880f384e2..4220ed243bc 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index afb3e13decb..ae5e1e767e9 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 7e04b88abf1..c557c46800b 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 67c71003775..c67359cb14e 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index e587f76f5d3..0a32246ce06 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 1d007233120..e1c33c4b13b 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/pom.xml b/pom.xml index 85ddb331552..b9b34479851 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.0 + 0.3.1-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 215d3bf0cbe..19553828205 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 049318700ef..09c44aea4b0 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT From d1626576c069cc9ee84c113b0bba04ed25c91207 Mon Sep 17 00:00:00 2001 From: James Estes Date: Sat, 9 Feb 2013 22:33:00 -0700 Subject: [PATCH 49/63] Working toward making it easier to add new SegmentPullers. 1) Move the local cacheFile logic out of the S3 pullers into the SingleSegmentLoader 2) Make the S3SegmentPuller just pull down the file 3) Make the Loader do the unzip, ungzip, or rename 4) 2 and 3 make S3ZippedSegmentPuller not necessary (still there, just deprecated and empty) 4) Tweak the TaskToolbox so that the Pullers returned by getSegmentGetters behave the same as they did before --- .../druid/merger/common/TaskToolbox.java | 39 +++-- .../druid/initialization/ServerInit.java | 12 +- .../metamx/druid/loading/S3SegmentPuller.java | 144 +++++----------- .../druid/loading/S3ZippedSegmentPuller.java | 163 +----------------- .../metamx/druid/loading/SegmentPuller.java | 2 +- .../druid/loading/SingleSegmentLoader.java | 141 ++++++++++++++- 6 files changed, 219 insertions(+), 282 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index 0cebe1fc91c..d775921d560 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -21,13 +21,15 @@ package com.metamx.druid.merger.common; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.S3SegmentPuller; -import com.metamx.druid.loading.S3SegmentGetterConfig; -import com.metamx.druid.loading.S3ZippedSegmentPuller; import com.metamx.druid.loading.SegmentPuller; +import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.SingleSegmentLoader; +import com.metamx.druid.loading.StorageAdapterLoadingException; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.emitter.service.ServiceEmitter; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -88,19 +90,28 @@ public class TaskToolbox public Map getSegmentGetters(final Task task) { - final S3SegmentGetterConfig getterConfig = new S3SegmentGetterConfig() - { - @Override - public File getCacheDirectory() - { - return new File(config.getTaskDir(task), "fetched_segments"); - } - }; + LoaderPullerAdapter puller = new LoaderPullerAdapter(new File(config.getTaskDir(task), "fetched_segments")); return ImmutableMap.builder() - .put("s3", new S3SegmentPuller(s3Client, getterConfig)) - .put("s3_union", new S3SegmentPuller(s3Client, getterConfig)) - .put("s3_zip", new S3ZippedSegmentPuller(s3Client, getterConfig)) + .put("s3", puller) + .put("s3_union", puller) + .put("s3_zip", puller) .build(); } + + class LoaderPullerAdapter implements SegmentPuller{ + private SingleSegmentLoader loader; + public LoaderPullerAdapter(File cacheDir){ + loader = new SingleSegmentLoader(new S3SegmentPuller(s3Client), new MMappedQueryableIndexFactory(), cacheDir); + } + @Override + public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException { + return loader.getSegmentFiles(loadSpec); + } + + @Override + public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { + return -1; + } + } } diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 1d727f9abe3..3a510e8b23c 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -35,7 +35,6 @@ import com.metamx.druid.query.group.GroupByQueryEngineConfig; import com.metamx.druid.Query; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.loading.QueryableLoaderConfig; -import com.metamx.druid.loading.S3ZippedSegmentPuller; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; @@ -69,8 +68,8 @@ public class ServerInit { DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); - final S3SegmentPuller segmentGetter = new S3SegmentPuller(s3Client, config); - final S3ZippedSegmentPuller zippedGetter = new S3ZippedSegmentPuller(s3Client, config); + final S3SegmentPuller segmentGetter = new S3SegmentPuller(s3Client); + final QueryableIndexFactory factory; if ("mmap".equals(config.getQueryableFactoryType())) { factory = new MMappedQueryableIndexFactory(); @@ -78,11 +77,12 @@ public class ServerInit throw new ISE("Unknown queryableFactoryType[%s]", config.getQueryableFactoryType()); } + SingleSegmentLoader segmentLoader = new SingleSegmentLoader(segmentGetter, factory, config.getCacheDirectory()); delegateLoader.setLoaderTypes( ImmutableMap.builder() - .put("s3", new SingleSegmentLoader(segmentGetter, factory)) - .put("s3_zip", new SingleSegmentLoader(zippedGetter, factory)) - .build() + .put("s3", segmentLoader) + .put("s3_zip", segmentLoader) + .build() ); return delegateLoader; diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java index 380489548d5..f85a489b1fe 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java @@ -25,17 +25,15 @@ import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.common.s3.S3Utils; -import org.apache.commons.io.FileUtils; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; import org.joda.time.DateTime; import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; import java.util.Map; -import java.util.zip.GZIPInputStream; /** */ @@ -48,133 +46,85 @@ public class S3SegmentPuller implements SegmentPuller private static final String KEY = "key"; private final RestS3Service s3Client; - private final S3SegmentGetterConfig config; @Inject public S3SegmentPuller( - RestS3Service s3Client, - S3SegmentGetterConfig config + RestS3Service s3Client ) { this.s3Client = s3Client; - this.config = config; } @Override public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { - Map loadSpec = segment.getLoadSpec(); - String s3Bucket = MapUtils.getString(loadSpec, "bucket"); - String s3Path = MapUtils.getString(loadSpec, "key"); + S3Coords s3Coords = new S3Coords(segment); - log.info("Loading index at path[s3://%s/%s]", s3Bucket, s3Path); + log.info("Loading index at path[%s]", s3Coords); - S3Object s3Obj = null; + if(!isObjectInBucket(s3Coords)){ + throw new StorageAdapterLoadingException("IndexFile[%s] does not exist.", s3Coords); + } + + long currTime = System.currentTimeMillis(); File tmpFile = null; + S3Object s3Obj = null; + try { - if (!s3Client.isObjectInBucket(s3Bucket, s3Path)) { - throw new StorageAdapterLoadingException("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3Path); - } + s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); + tmpFile = File.createTempFile(s3Coords.bucket, new DateTime().toString() + s3Coords.path.replace('/', '_')); + log.info("Downloading file[%s] to local tmpFile[%s] for segment[%s]", s3Coords, tmpFile, segment); - File cacheFile = new File(config.getCacheDirectory(), computeCacheFilePath(s3Bucket, s3Path)); - - if (cacheFile.exists()) { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(s3Bucket), s3Path); - DateTime cacheFileLastModified = new DateTime(cacheFile.lastModified()); - DateTime s3ObjLastModified = new DateTime(objDetails.getLastModifiedDate().getTime()); - if (cacheFileLastModified.isAfter(s3ObjLastModified)) { - log.info( - "Found cacheFile[%s] with modified[%s], which is after s3Obj[%s]. Using.", - cacheFile, - cacheFileLastModified, - s3ObjLastModified - ); - return cacheFile.getParentFile(); - } - FileUtils.deleteDirectory(cacheFile.getParentFile()); - } - - long currTime = System.currentTimeMillis(); - - tmpFile = File.createTempFile(s3Bucket, new DateTime().toString()); - log.info( - "Downloading file[s3://%s/%s] to local tmpFile[%s] for cacheFile[%s]", - s3Bucket, s3Path, tmpFile, cacheFile - ); - - s3Obj = s3Client.getObject(new S3Bucket(s3Bucket), s3Path); StreamUtils.copyToFileAndClose(s3Obj.getDataInputStream(), tmpFile, DEFAULT_TIMEOUT); final long downloadEndTime = System.currentTimeMillis(); - log.info("Download of file[%s] completed in %,d millis", cacheFile, downloadEndTime - currTime); + log.info("Download of file[%s] completed in %,d millis", tmpFile, downloadEndTime - currTime); - if (!cacheFile.getParentFile().mkdirs()) { - log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); - } - cacheFile.delete(); - - if (s3Path.endsWith("gz")) { - log.info("Decompressing file[%s] to [%s]", tmpFile, cacheFile); - StreamUtils.copyToFileAndClose( - new GZIPInputStream(new FileInputStream(tmpFile)), - cacheFile - ); - if (!tmpFile.delete()) { - log.error("Could not delete tmpFile[%s].", tmpFile); - } - } else { - log.info("Rename tmpFile[%s] to cacheFile[%s]", tmpFile, cacheFile); - if (!tmpFile.renameTo(cacheFile)) { - log.warn("Error renaming tmpFile[%s] to cacheFile[%s]. Copying instead.", tmpFile, cacheFile); - - StreamUtils.copyToFileAndClose(new FileInputStream(tmpFile), cacheFile); - if (!tmpFile.delete()) { - log.error("Could not delete tmpFile[%s].", tmpFile); - } - } - } - - long endTime = System.currentTimeMillis(); - log.info("Local processing of file[%s] done in %,d millis", cacheFile, endTime - downloadEndTime); - - return cacheFile.getParentFile(); + return tmpFile; } catch (Exception e) { + if(tmpFile!=null && tmpFile.exists()){ + tmpFile.delete(); + } throw new StorageAdapterLoadingException(e, e.getMessage()); } finally { S3Utils.closeStreamsQuietly(s3Obj); - if (tmpFile != null && tmpFile.exists()) { - log.warn("Deleting tmpFile[%s] in finally block. Why?", tmpFile); - tmpFile.delete(); - } } } - private String computeCacheFilePath(String s3Bucket, String s3Path) - { - return String.format( - "%s/%s", s3Bucket, s3Path.endsWith("gz") ? s3Path.substring(0, s3Path.length() - ".gz".length()) : s3Path - ); + private boolean isObjectInBucket(S3Coords coords) throws StorageAdapterLoadingException { + try { + return s3Client.isObjectInBucket(coords.bucket, coords.path); + } catch (ServiceException e) { + throw new StorageAdapterLoadingException(e, "Problem communicating with S3 checking bucket/path[%s]", coords); + } } @Override - public boolean cleanSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - File cacheFile = new File( - config.getCacheDirectory(), - computeCacheFilePath(MapUtils.getString(loadSpec, BUCKET), MapUtils.getString(loadSpec, KEY)) - ); - + public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { + S3Coords coords = new S3Coords(segment); try { - final File parentFile = cacheFile.getParentFile(); - log.info("Recursively deleting file[%s]", parentFile); - FileUtils.deleteDirectory(parentFile); - } - catch (IOException e) { + S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); + return objDetails.getLastModifiedDate().getTime(); + } catch (S3ServiceException e) { throw new StorageAdapterLoadingException(e, e.getMessage()); } + } - return true; + private class S3Coords { + String bucket; + String path; + + public S3Coords(DataSegment segment) { + Map loadSpec = segment.getLoadSpec(); + bucket = MapUtils.getString(loadSpec, BUCKET); + path = MapUtils.getString(loadSpec, KEY); + if(path.startsWith("/")){ + path = path.substring(1); + } + } + public String toString(){ + return String.format("s3://%s/%s", bucket, path); + } } } diff --git a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java index 8fd8ebd4542..a3a7c724687 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java @@ -19,169 +19,14 @@ package com.metamx.druid.loading; -import com.google.common.io.Closeables; -import com.metamx.common.MapUtils; -import com.metamx.common.StreamUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.common.s3.S3Utils; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; -import org.jets3t.service.model.S3Object; -import org.joda.time.DateTime; - -import java.io.BufferedInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.util.Map; -import java.util.zip.ZipEntry; -import java.util.zip.ZipInputStream; /** + * @deprecated */ -public class S3ZippedSegmentPuller implements SegmentPuller +public class S3ZippedSegmentPuller extends S3SegmentPuller { - private static final Logger log = new Logger(S3ZippedSegmentPuller.class); - - private static final String BUCKET = "bucket"; - private static final String KEY = "key"; - - private final RestS3Service s3Client; - private final S3SegmentGetterConfig config; - - public S3ZippedSegmentPuller( - RestS3Service s3Client, - S3SegmentGetterConfig config - ) - { - this.s3Client = s3Client; - this.config = config; - } - - @Override - public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - String s3Bucket = MapUtils.getString(loadSpec, "bucket"); - String s3Path = MapUtils.getString(loadSpec, "key"); - - if (s3Path.startsWith("/")) { - s3Path = s3Path.substring(1); - } - - log.info("Loading index at path[s3://%s/%s]", s3Bucket, s3Path); - - S3Object s3Obj = null; - File tmpFile = null; - try { - if (!s3Client.isObjectInBucket(s3Bucket, s3Path)) { - throw new StorageAdapterLoadingException("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3Path); - } - - File cacheFile = new File(config.getCacheDirectory(), computeCacheFilePath(s3Bucket, s3Path)); - - if (cacheFile.exists()) { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(s3Bucket), s3Path); - DateTime cacheFileLastModified = new DateTime(cacheFile.lastModified()); - DateTime s3ObjLastModified = new DateTime(objDetails.getLastModifiedDate().getTime()); - if (cacheFileLastModified.isAfter(s3ObjLastModified)) { - log.info( - "Found cacheFile[%s] with modified[%s], which is after s3Obj[%s]. Using.", - cacheFile, - cacheFileLastModified, - s3ObjLastModified - ); - return cacheFile; - } - FileUtils.deleteDirectory(cacheFile); - } - - long currTime = System.currentTimeMillis(); - - tmpFile = File.createTempFile(s3Bucket, new DateTime().toString()); - log.info( - "Downloading file[s3://%s/%s] to local tmpFile[%s] for cacheFile[%s]", - s3Bucket, s3Path, tmpFile, cacheFile - ); - - s3Obj = s3Client.getObject(new S3Bucket(s3Bucket), s3Path); - StreamUtils.copyToFileAndClose(s3Obj.getDataInputStream(), tmpFile); - final long downloadEndTime = System.currentTimeMillis(); - log.info("Download of file[%s] completed in %,d millis", cacheFile, downloadEndTime - currTime); - - if (cacheFile.exists()) { - FileUtils.deleteDirectory(cacheFile); - } - cacheFile.mkdirs(); - - ZipInputStream zipIn = null; - OutputStream out = null; - ZipEntry entry = null; - try { - zipIn = new ZipInputStream(new BufferedInputStream(new FileInputStream(tmpFile))); - while ((entry = zipIn.getNextEntry()) != null) { - out = new FileOutputStream(new File(cacheFile, entry.getName())); - IOUtils.copy(zipIn, out); - zipIn.closeEntry(); - Closeables.closeQuietly(out); - out = null; - } - } - finally { - Closeables.closeQuietly(out); - Closeables.closeQuietly(zipIn); - } - - long endTime = System.currentTimeMillis(); - log.info("Local processing of file[%s] done in %,d millis", cacheFile, endTime - downloadEndTime); - - log.info("Deleting tmpFile[%s]", tmpFile); - tmpFile.delete(); - - return cacheFile; - } - catch (Exception e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - finally { - S3Utils.closeStreamsQuietly(s3Obj); - if (tmpFile != null && tmpFile.exists()) { - log.warn("Deleting tmpFile[%s] in finally block. Why?", tmpFile); - tmpFile.delete(); - } - } - } - - private String computeCacheFilePath(String s3Bucket, String s3Path) - { - return new File(String.format("%s/%s", s3Bucket, s3Path)).getParent(); - } - - @Override - public boolean cleanSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - File cacheFile = new File( - config.getCacheDirectory(), - computeCacheFilePath( - MapUtils.getString(loadSpec, BUCKET), - MapUtils.getString(loadSpec, KEY) - ) - ); - - try { - log.info("Deleting directory[%s]", cacheFile); - FileUtils.deleteDirectory(cacheFile); - } - catch (IOException e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - - return true; + public S3ZippedSegmentPuller(RestS3Service s3Client) { + super(s3Client); } } diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java index 9cba65f425c..3e5f1b1a161 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java @@ -29,5 +29,5 @@ import java.util.Map; public interface SegmentPuller { public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; - public boolean cleanSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; + long getLastModified(DataSegment segment) throws StorageAdapterLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index 19c3981e988..ae62cfda1e9 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -19,40 +19,171 @@ package com.metamx.druid.loading; +import com.google.common.base.Joiner; +import com.google.common.io.Closeables; import com.google.inject.Inject; +import com.metamx.common.StreamUtils; +import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; import com.metamx.druid.index.Segment; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; + +import java.io.*; +import java.util.zip.GZIPInputStream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; /** */ public class SingleSegmentLoader implements SegmentLoader { + private static final Logger log = new Logger(SingleSegmentLoader.class); + private final SegmentPuller segmentPuller; private final QueryableIndexFactory factory; + private File cacheDirectory; + private static final Joiner JOINER = Joiner.on("/").skipNulls(); @Inject public SingleSegmentLoader( - SegmentPuller segmentPuller, - QueryableIndexFactory factory - ) + SegmentPuller segmentPuller, + QueryableIndexFactory factory, + File cacheDirectory) { this.segmentPuller = segmentPuller; this.factory = factory; + this.cacheDirectory = cacheDirectory; } @Override public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException { - final QueryableIndex index = factory.factorize(segmentPuller.getSegmentFiles(segment)); + File segmentFiles = getSegmentFiles(segment); + final QueryableIndex index = factory.factorize(segmentFiles); return new QueryableIndexSegment(segment.getIdentifier(), index); } + public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { + File cacheFile = getCacheFile(segment); + if (cacheFile.exists()) { + long localLastModified = cacheFile.lastModified(); + long remoteLastModified = segmentPuller.getLastModified(segment); + if(remoteLastModified > 0 && localLastModified >= remoteLastModified){ + log.info( + "Found cacheFile[%s] with modified[%s], which is same or after remote[%s]. Using.", + cacheFile, + localLastModified, + remoteLastModified + ); + return cacheFile.getParentFile(); + } + } + + File pulledFile = segmentPuller.getSegmentFiles(segment); + + if(!cacheFile.getParentFile().mkdirs()){ + log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); + } + if (cacheFile.exists()) { + cacheFile.delete(); + } + + if(pulledFile.getName().endsWith(".zip")){ + unzip(pulledFile, cacheFile.getParentFile()); + } else if(pulledFile.getName().endsWith(".gz")){ + gunzip(pulledFile, cacheFile); + } else { + moveToCache(pulledFile, cacheFile); + } + + return cacheFile.getParentFile(); + } + + private File getCacheFile(DataSegment segment) { + String outputKey = JOINER.join( + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart(), + segment.getInterval().getEnd() + ), + segment.getVersion(), + segment.getShardSpec().getPartitionNum() + ); + + return new File(cacheDirectory, outputKey); + } + + private void moveToCache(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + log.info("Rename pulledFile[%s] to cacheFile[%s]", pulledFile, cacheFile); + if(!pulledFile.renameTo(cacheFile)){ + log.warn("Error renaming pulledFile[%s] to cacheFile[%s]. Copying instead.", pulledFile, cacheFile); + + try { + StreamUtils.copyToFileAndClose(new FileInputStream(pulledFile), cacheFile); + } catch (IOException e) { + throw new StorageAdapterLoadingException(e,"Problem moving pulledFile[%s] to cache[%s]", pulledFile, cacheFile); + } + if (!pulledFile.delete()) { + log.error("Could not delete pulledFile[%s].", pulledFile); + } + } + } + + private void unzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + log.info("Unzipping file[%s] to [%s]", pulledFile, cacheFile); + ZipInputStream zipIn = null; + OutputStream out = null; + ZipEntry entry = null; + try { + zipIn = new ZipInputStream(new BufferedInputStream(new FileInputStream(pulledFile))); + while ((entry = zipIn.getNextEntry()) != null) { + out = new FileOutputStream(new File(cacheFile, entry.getName())); + IOUtils.copy(zipIn, out); + zipIn.closeEntry(); + Closeables.closeQuietly(out); + out = null; + } + } catch(IOException e) { + throw new StorageAdapterLoadingException(e,"Problem unzipping[%s]", pulledFile); + } + finally { + Closeables.closeQuietly(out); + Closeables.closeQuietly(zipIn); + } + } + + private void gunzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + log.info("Gunzipping file[%s] to [%s]", pulledFile, cacheFile); + try { + StreamUtils.copyToFileAndClose( + new GZIPInputStream(new FileInputStream(pulledFile)), + cacheFile + ); + } catch (IOException e) { + throw new StorageAdapterLoadingException(e,"Problem gunzipping[%s]", pulledFile); + } + if (!pulledFile.delete()) { + log.error("Could not delete tmpFile[%s].", pulledFile); + } + } + @Override public void cleanup(DataSegment segment) throws StorageAdapterLoadingException { - segmentPuller.cleanSegmentFiles(segment); + File cacheFile = getCacheFile(segment).getParentFile(); + + try { + log.info("Deleting directory[%s]", cacheFile); + FileUtils.deleteDirectory(cacheFile); + } + catch (IOException e) { + throw new StorageAdapterLoadingException(e, e.getMessage()); + } } + } From dc3459d3f931e3d230a97f23da38152e947798b0 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Wed, 13 Feb 2013 19:15:49 -0600 Subject: [PATCH 50/63] 1) Initial commit of refactorings on top of housejester's to simplify the zipping and simplify the creation of other methods of loading data --- .../metamx/druid/utils/CompressionUtils.java | 90 ++++++++++ .../examples/RealtimeStandaloneMain.java | 6 +- .../examples/RealtimeStandaloneMain.java | 6 +- .../druid/merger/common/TaskToolbox.java | 48 ++--- .../common/index/YeOldePlumberSchool.java | 10 +- .../common/task/IndexGeneratorTask.java | 6 +- .../druid/merger/common/task/MergeTask.java | 10 +- .../http/IndexerCoordinatorNode.java | 8 +- .../druid/merger/worker/http/WorkerNode.java | 8 +- .../metamx/druid/realtime/RealtimeNode.java | 22 +-- .../druid/realtime/RealtimePlumberSchool.java | 12 +- .../druid/realtime/S3SegmentPusher.java | 6 +- .../druid/coordination/ServerManager.java | 14 +- .../druid/coordination/ZkCoordinator.java | 6 +- .../druid/initialization/ServerInit.java | 4 +- ...mentPuller.java => DataSegmentPuller.java} | 22 ++- ...mentPusher.java => DataSegmentPusher.java} | 2 +- .../loading/DelegatingSegmentLoader.java | 8 +- .../loading/MMappedQueryableIndexFactory.java | 8 +- .../druid/loading/QueryableIndexFactory.java | 2 +- .../druid/loading/S3DataSegmentPuller.java | 170 ++++++++++++++++++ ...ntPusher.java => S3DataSegmentPusher.java} | 12 +- .../metamx/druid/loading/S3SegmentPuller.java | 130 -------------- .../druid/loading/S3ZippedSegmentPuller.java | 32 ---- .../metamx/druid/loading/SegmentLoader.java | 4 +- ...tion.java => SegmentLoadingException.java} | 6 +- .../druid/loading/SingleSegmentLoader.java | 102 ++++------- .../druid/coordination/ServerManagerTest.java | 15 +- .../druid/loading/NoopSegmentLoader.java | 4 +- 29 files changed, 410 insertions(+), 363 deletions(-) create mode 100644 common/src/main/java/com/metamx/druid/utils/CompressionUtils.java rename server/src/main/java/com/metamx/druid/loading/{SegmentPuller.java => DataSegmentPuller.java} (55%) rename server/src/main/java/com/metamx/druid/loading/{SegmentPusher.java => DataSegmentPusher.java} (96%) create mode 100644 server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java rename server/src/main/java/com/metamx/druid/loading/{S3SegmentPusher.java => S3DataSegmentPusher.java} (96%) delete mode 100644 server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java delete mode 100644 server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java rename server/src/main/java/com/metamx/druid/loading/{StorageAdapterLoadingException.java => SegmentLoadingException.java} (88%) diff --git a/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java new file mode 100644 index 00000000000..2b87d0a866d --- /dev/null +++ b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java @@ -0,0 +1,90 @@ +/* + * 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.utils; + +import com.google.common.io.ByteStreams; +import com.google.common.io.Closeables; +import com.metamx.common.ISE; +import com.metamx.common.StreamUtils; +import com.metamx.common.logger.Logger; +import sun.misc.IOUtils; + +import java.io.BufferedInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.zip.GZIPInputStream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; + +/** + */ +public class CompressionUtils +{ + private static final Logger log = new Logger(CompressionUtils.class); + + public static void unzip(File pulledFile, File outDir) throws IOException + { + if (!(outDir.exists() && outDir.isDirectory())) { + throw new ISE("outDir[%s] must exist and be a directory"); + } + + log.info("Unzipping file[%s] to [%s]", pulledFile, outDir); + InputStream in = null; + try { + in = new BufferedInputStream(new FileInputStream(pulledFile)); + unzip(in, outDir); + } + finally { + Closeables.closeQuietly(in); + } + } + + public static void unzip(InputStream in, File outDir) throws IOException + { + ZipInputStream zipIn = new ZipInputStream(in); + + ZipEntry entry; + while ((entry = zipIn.getNextEntry()) != null) { + OutputStream out = null; + try { + out = new FileOutputStream(new File(outDir, entry.getName())); + ByteStreams.copy(zipIn, out); + zipIn.closeEntry(); + } + finally { + Closeables.closeQuietly(out); + } + } + } + + public static void gunzip(File pulledFile, File outDir) throws IOException + { + log.info("Gunzipping file[%s] to [%s]", pulledFile, outDir); + StreamUtils.copyToFileAndClose(new GZIPInputStream(new FileInputStream(pulledFile)), outDir); + if (!pulledFile.delete()) { + log.error("Could not delete tmpFile[%s].", pulledFile); + } + } + +} diff --git a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java index ecdf2606a3b..92eb86cc801 100644 --- a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -12,7 +12,7 @@ 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.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.phonebook.PhoneBook; @@ -72,8 +72,8 @@ public class RealtimeStandaloneMain // dummyMetadataUpdater will not send updates to db because standalone demo has no db rn.setMetadataUpdater(dummyMetadataUpdater); - rn.setSegmentPusher( - new SegmentPusher() + rn.setDataSegmentPusher( + new DataSegmentPusher() { @Override public DataSegment push(File file, DataSegment segment) throws IOException diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index e936d481489..ca5b9f64fd4 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -12,7 +12,7 @@ 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.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.phonebook.PhoneBook; import druid.examples.twitter.TwitterSpritzerFirehoseFactory; @@ -74,8 +74,8 @@ public class RealtimeStandaloneMain // dummyMetadataUpdater will not send updates to db because standalone demo has no db rn.setMetadataUpdater(dummyMetadataUpdater); - rn.setSegmentPusher( - new SegmentPusher() + rn.setDataSegmentPusher( + new DataSegmentPusher() { @Override public DataSegment push(File file, DataSegment segment) throws IOException diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index d775921d560..f77b08ce713 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -20,14 +20,13 @@ package com.metamx.druid.merger.common; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.MMappedQueryableIndexFactory; -import com.metamx.druid.loading.S3SegmentPuller; -import com.metamx.druid.loading.SegmentPuller; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; -import com.metamx.druid.loading.StorageAdapterLoadingException; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.emitter.service.ServiceEmitter; @@ -35,6 +34,7 @@ import com.metamx.emitter.service.ServiceEmitter; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import java.io.File; +import java.util.List; import java.util.Map; /** @@ -45,14 +45,14 @@ public class TaskToolbox private final IndexerCoordinatorConfig config; private final ServiceEmitter emitter; private final RestS3Service s3Client; - private final SegmentPusher segmentPusher; + private final DataSegmentPusher segmentPusher; private final ObjectMapper objectMapper; public TaskToolbox( IndexerCoordinatorConfig config, ServiceEmitter emitter, RestS3Service s3Client, - SegmentPusher segmentPusher, + DataSegmentPusher segmentPusher, ObjectMapper objectMapper ) { @@ -78,7 +78,7 @@ public class TaskToolbox return s3Client; } - public SegmentPusher getSegmentPusher() + public DataSegmentPusher getSegmentPusher() { return segmentPusher; } @@ -88,30 +88,20 @@ public class TaskToolbox return objectMapper; } - public Map getSegmentGetters(final Task task) + public Map getSegments(final Task task, List segments) + throws SegmentLoadingException { - LoaderPullerAdapter puller = new LoaderPullerAdapter(new File(config.getTaskDir(task), "fetched_segments")); + final SingleSegmentLoader loader = new SingleSegmentLoader( + new S3DataSegmentPuller(s3Client), + new MMappedQueryableIndexFactory(), + new File(config.getTaskDir(task), "fetched_segments") + ); - return ImmutableMap.builder() - .put("s3", puller) - .put("s3_union", puller) - .put("s3_zip", puller) - .build(); - } - - class LoaderPullerAdapter implements SegmentPuller{ - private SingleSegmentLoader loader; - public LoaderPullerAdapter(File cacheDir){ - loader = new SingleSegmentLoader(new S3SegmentPuller(s3Client), new MMappedQueryableIndexFactory(), cacheDir); - } - @Override - public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException { - return loader.getSegmentFiles(loadSpec); + Map retVal = Maps.newLinkedHashMap(); + for (DataSegment segment : segments) { + retVal.put(segment, loader.getSegmentFiles(segment)); } - @Override - public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { - return -1; - } + return retVal; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index 7d456d29e01..703dbe898f7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -35,7 +35,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.realtime.FireDepartmentMetrics; import com.metamx.druid.realtime.FireHydrant; @@ -61,7 +61,7 @@ public class YeOldePlumberSchool implements PlumberSchool { private final Interval interval; private final String version; - private final SegmentPusher segmentPusher; + private final DataSegmentPusher dataSegmentPusher; private final File tmpSegmentDir; private static final Logger log = new Logger(YeOldePlumberSchool.class); @@ -70,13 +70,13 @@ public class YeOldePlumberSchool implements PlumberSchool public YeOldePlumberSchool( @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, - @JacksonInject("segmentPusher") SegmentPusher segmentPusher, + @JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher, @JacksonInject("tmpSegmentDir") File tmpSegmentDir ) { this.interval = interval; this.version = version; - this.segmentPusher = segmentPusher; + this.dataSegmentPusher = dataSegmentPusher; this.tmpSegmentDir = tmpSegmentDir; } @@ -149,7 +149,7 @@ public class YeOldePlumberSchool implements PlumberSchool .withVersion(version) .withBinaryVersion(IndexIO.getVersionFromDir(fileToUpload)); - segmentPusher.push(fileToUpload, segmentToUpload); + dataSegmentPusher.push(fileToUpload, segmentToUpload); log.info( "Uploaded segment[%s]", diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index b3da8978bd3..e56b9c4967d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -27,6 +27,7 @@ import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.input.InputRow; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.index.YeOldePlumberSchool; @@ -36,7 +37,6 @@ import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.Schema; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.realtime.Sink; @@ -111,7 +111,7 @@ public class IndexGeneratorTask extends AbstractTask // We need to track published segments. final List pushedSegments = new CopyOnWriteArrayList(); - final SegmentPusher wrappedSegmentPusher = new SegmentPusher() + final DataSegmentPusher wrappedDataSegmentPusher = new DataSegmentPusher() { @Override public DataSegment push(File file, DataSegment segment) throws IOException @@ -128,7 +128,7 @@ public class IndexGeneratorTask extends AbstractTask final Plumber plumber = new YeOldePlumberSchool( getInterval(), context.getVersion(), - wrappedSegmentPusher, + wrappedDataSegmentPusher, tmpDir ).findPlumber(schema, metrics); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 21f6c1e6416..2cfec4e5d2a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -31,13 +31,11 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.loading.SegmentPuller; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -147,13 +145,7 @@ public abstract class MergeTask extends AbstractTask // download segments to merge - final Map segmentGetters = toolbox.getSegmentGetters(this); - final Map gettedSegments = Maps.newHashMap(); - for (final DataSegment segment : segments) { - Map loadSpec = segment.getLoadSpec(); - SegmentPuller segmentPuller = segmentGetters.get(loadSpec.get("type")); - gettedSegments.put(segment, segmentPuller.getSegmentFiles(segment)); - } + final Map gettedSegments = toolbox.getSegments(this, segments); // merge files together final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 7d075cbe9e8..ec5404086d9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -49,9 +49,9 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -403,12 +403,12 @@ public class IndexerCoordinatorNode extends RegisteringNode PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ); - final SegmentPusher segmentPusher = new S3SegmentPusher( + final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, configFactory.build(S3SegmentPusherConfig.class), jsonMapper ); - taskToolbox = new TaskToolbox(config, emitter, s3Client, segmentPusher, jsonMapper); + taskToolbox = new TaskToolbox(config, emitter, s3Client, dataSegmentPusher, jsonMapper); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 237ae09d854..a152f0f003f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -36,6 +36,7 @@ import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -44,9 +45,8 @@ 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.config.WorkerConfig; -import com.metamx.druid.loading.S3SegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -290,12 +290,12 @@ public class WorkerNode extends RegisteringNode PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ); - final SegmentPusher segmentPusher = new S3SegmentPusher( + final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, configFactory.build(S3SegmentPusherConfig.class), jsonMapper ); - taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, segmentPusher, jsonMapper); + taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, dataSegmentPusher, jsonMapper); } } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index d8fc7ebeea7..536074d6247 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -46,9 +46,9 @@ import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; @@ -86,7 +86,7 @@ public class RealtimeNode extends BaseServerNode private final Map injectablesMap = Maps.newLinkedHashMap(); private MetadataUpdater metadataUpdater = null; - private SegmentPusher segmentPusher = null; + private DataSegmentPusher dataSegmentPusher = null; private List fireDepartments = null; private ServerView view = null; @@ -117,10 +117,10 @@ public class RealtimeNode extends BaseServerNode return this; } - public RealtimeNode setSegmentPusher(SegmentPusher segmentPusher) + public RealtimeNode setDataSegmentPusher(DataSegmentPusher dataSegmentPusher) { - Preconditions.checkState(this.segmentPusher == null, "Cannot set segmentPusher once it has already been set."); - this.segmentPusher = segmentPusher; + Preconditions.checkState(this.dataSegmentPusher == null, "Cannot set segmentPusher once it has already been set."); + this.dataSegmentPusher = dataSegmentPusher; return this; } @@ -144,10 +144,10 @@ public class RealtimeNode extends BaseServerNode return metadataUpdater; } - public SegmentPusher getSegmentPusher() + public DataSegmentPusher getDataSegmentPusher() { initializeSegmentPusher(); - return segmentPusher; + return dataSegmentPusher; } public List getFireDepartments() @@ -220,7 +220,7 @@ public class RealtimeNode extends BaseServerNode } injectables.put("queryRunnerFactoryConglomerate", getConglomerate()); - injectables.put("segmentPusher", segmentPusher); + injectables.put("segmentPusher", dataSegmentPusher); injectables.put("metadataUpdater", metadataUpdater); injectables.put("serverView", view); injectables.put("serviceEmitter", getEmitter()); @@ -256,7 +256,7 @@ public class RealtimeNode extends BaseServerNode private void initializeSegmentPusher() { - if (segmentPusher == null) { + if (dataSegmentPusher == null) { final Properties props = getProps(); final RestS3Service s3Client; try { @@ -271,7 +271,7 @@ public class RealtimeNode extends BaseServerNode throw Throwables.propagate(e); } - segmentPusher = new S3SegmentPusher(s3Client, getConfigFactory().build(S3SegmentPusherConfig.class), getJsonMapper()); + dataSegmentPusher = new S3DataSegmentPusher(s3Client, getConfigFactory().build(S3SegmentPusherConfig.class), getJsonMapper()); } } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java index 47a4ef0f40f..70c12eaaa45 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -48,7 +48,7 @@ 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.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; @@ -95,7 +95,7 @@ public class RealtimePlumberSchool implements PlumberSchool private volatile RejectionPolicyFactory rejectionPolicyFactory = null; private volatile QueryRunnerFactoryConglomerate conglomerate = null; - private volatile SegmentPusher segmentPusher = null; + private volatile DataSegmentPusher dataSegmentPusher = null; private volatile MetadataUpdater metadataUpdater = null; private volatile ServerView serverView = null; private ServiceEmitter emitter; @@ -130,9 +130,9 @@ public class RealtimePlumberSchool implements PlumberSchool } @JacksonInject("segmentPusher") - public void setSegmentPusher(SegmentPusher segmentPusher) + public void setDataSegmentPusher(DataSegmentPusher dataSegmentPusher) { - this.segmentPusher = segmentPusher; + this.dataSegmentPusher = dataSegmentPusher; } @JacksonInject("metadataUpdater") @@ -325,7 +325,7 @@ public class RealtimePlumberSchool implements PlumberSchool QueryableIndex index = IndexIO.loadIndex(mergedFile); - DataSegment segment = segmentPusher.push( + DataSegment segment = dataSegmentPusher.push( mergedFile, sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions())) ); @@ -512,7 +512,7 @@ public class RealtimePlumberSchool implements PlumberSchool private void verifyState() { Preconditions.checkNotNull(conglomerate, "must specify a queryRunnerFactoryConglomerate to do this action."); - Preconditions.checkNotNull(segmentPusher, "must specify a segmentPusher 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."); diff --git a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java index 5a74b17e223..2e40c398bdc 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java @@ -20,16 +20,18 @@ package com.metamx.druid.realtime; import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import org.jets3t.service.impl.rest.httpclient.RestS3Service; /** * A placeholder class to make the move of the SegmentPushers to a new package backwards compatible + * + * Exists in 0.2, can be removed from 0.3 on */ @Deprecated -public class S3SegmentPusher extends com.metamx.druid.loading.S3SegmentPusher implements SegmentPusher +public class S3SegmentPusher extends com.metamx.druid.loading.S3DataSegmentPusher implements DataSegmentPusher { public S3SegmentPusher( RestS3Service s3Client, diff --git a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java index 3bde07908c7..38e7d1e4a39 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -30,7 +30,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.collect.CountingMap; import com.metamx.druid.index.Segment; import com.metamx.druid.loading.SegmentLoader; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.PartitionHolder; import com.metamx.druid.query.BySegmentQueryRunner; @@ -104,24 +104,24 @@ public class ServerManager implements QuerySegmentWalker } } - public void loadSegment(final DataSegment segment) throws StorageAdapterLoadingException + public void loadSegment(final DataSegment segment) throws SegmentLoadingException { final Segment adapter; try { adapter = segmentLoader.getSegment(segment); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { try { segmentLoader.cleanup(segment); } - catch (StorageAdapterLoadingException e1) { + catch (SegmentLoadingException e1) { // ignore } throw e; } if (adapter == null) { - throw new StorageAdapterLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); + throw new SegmentLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); } synchronized (lock) { @@ -139,7 +139,7 @@ public class ServerManager implements QuerySegmentWalker ); if ((entry != null) && (entry.getChunk(segment.getShardSpec().getPartitionNum()) != null)) { log.info("Told to load a adapter for a segment[%s] that already exists", segment.getIdentifier()); - throw new StorageAdapterLoadingException("Segment already exists[%s]", segment.getIdentifier()); + throw new SegmentLoadingException("Segment already exists[%s]", segment.getIdentifier()); } loadedIntervals.add( @@ -154,7 +154,7 @@ public class ServerManager implements QuerySegmentWalker } } - public void dropSegment(final DataSegment segment) throws StorageAdapterLoadingException + public void dropSegment(final DataSegment segment) throws SegmentLoadingException { String dataSource = segment.getDataSource(); synchronized (lock) { diff --git a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java index 57d64e0ba32..1951205975c 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java +++ b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java @@ -29,7 +29,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.phonebook.PhoneBook; @@ -245,14 +245,14 @@ public class ZkCoordinator implements DataSegmentChangeHandler } catch (IOException e) { removeSegment(segment); - throw new StorageAdapterLoadingException( + throw new SegmentLoadingException( "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile ); } yp.announce(servedSegmentsLocation, segment.getIdentifier(), segment); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { log.error(e, "Failed to load segment[%s]", segment); emitter.emit( new AlertEvent.Builder().build( diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 3a510e8b23c..82a5f263608 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -28,7 +28,7 @@ import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.loading.DelegatingSegmentLoader; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; -import com.metamx.druid.loading.S3SegmentPuller; +import com.metamx.druid.loading.S3DataSegmentPuller; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.query.group.GroupByQueryEngine; import com.metamx.druid.query.group.GroupByQueryEngineConfig; @@ -68,7 +68,7 @@ public class ServerInit { DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); - final S3SegmentPuller segmentGetter = new S3SegmentPuller(s3Client); + final S3DataSegmentPuller segmentGetter = new S3DataSegmentPuller(s3Client); final QueryableIndexFactory factory; if ("mmap".equals(config.getQueryableFactoryType())) { diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java similarity index 55% rename from server/src/main/java/com/metamx/druid/loading/SegmentPuller.java rename to server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java index 3e5f1b1a161..b821c653a6e 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java @@ -22,12 +22,26 @@ package com.metamx.druid.loading; import com.metamx.druid.client.DataSegment; import java.io.File; -import java.util.Map; /** */ -public interface SegmentPuller +public interface DataSegmentPuller { - public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; - long getLastModified(DataSegment segment) throws StorageAdapterLoadingException; + /** + * Pull down segment files for the given DataSegment and put them in the given directory. + * + * @param segment The segment to pull down files for + * @param dir The directory to store the files in + * @throws SegmentLoadingException if there are any errors + */ + public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException; + + /** + * Returns the last modified time of the given segment. + * + * @param segment The segment to check the last modified time for + * @return the last modified time in millis from the epoch + * @throws SegmentLoadingException if there are any errors + */ + public long getLastModified(DataSegment segment) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java similarity index 96% rename from server/src/main/java/com/metamx/druid/loading/SegmentPusher.java rename to server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java index 3700215efc1..5369480d6b9 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java @@ -24,7 +24,7 @@ import com.metamx.druid.client.DataSegment; import java.io.File; import java.io.IOException; -public interface SegmentPusher +public interface DataSegmentPusher { public DataSegment push(File file, DataSegment segment) throws IOException; } diff --git a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java index d576e59ae82..0f8e1e7074f 100644 --- a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java @@ -44,24 +44,24 @@ public class DelegatingSegmentLoader implements SegmentLoader } @Override - public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(DataSegment segment) throws SegmentLoadingException { return getLoader(segment.getLoadSpec()).getSegment(segment); } @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws SegmentLoadingException { getLoader(segment.getLoadSpec()).cleanup(segment); } - private SegmentLoader getLoader(Map loadSpec) throws StorageAdapterLoadingException + private SegmentLoader getLoader(Map loadSpec) throws SegmentLoadingException { String type = MapUtils.getString(loadSpec, "type"); SegmentLoader loader = loaderTypes.get(type); if (loader == null) { - throw new StorageAdapterLoadingException("Unknown loader type[%s]. Known types are %s", type, loaderTypes.keySet()); + throw new SegmentLoadingException("Unknown loader type[%s]. Known types are %s", type, loaderTypes.keySet()); } return loader; } diff --git a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java index 648813d62ac..9f8594a30d2 100644 --- a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java @@ -34,7 +34,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory private static final Logger log = new Logger(MMappedQueryableIndexFactory.class); @Override - public QueryableIndex factorize(File parentDir) throws StorageAdapterLoadingException + public QueryableIndex factorize(File parentDir) throws SegmentLoadingException { try { if (! IndexIO.canBeMapped(parentDir)) { @@ -46,11 +46,11 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory IndexIO.storeLatest(IndexIO.readIndex(parentDir), canBeMappedDir); if (! IndexIO.canBeMapped(canBeMappedDir)) { - throw new StorageAdapterLoadingException("WTF!? newly written file[%s] cannot be mapped!?", canBeMappedDir); + throw new SegmentLoadingException("WTF!? newly written file[%s] cannot be mapped!?", canBeMappedDir); } for (File file : canBeMappedDir.listFiles()) { if (! file.renameTo(new File(parentDir, file.getName()))) { - throw new StorageAdapterLoadingException("Couldn't rename[%s] to [%s]", canBeMappedDir, parentDir); + throw new SegmentLoadingException("Couldn't rename[%s] to [%s]", canBeMappedDir, parentDir); } } FileUtils.deleteDirectory(canBeMappedDir); @@ -66,7 +66,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory catch (IOException e2) { log.error(e, "Problem deleting parentDir[%s]", parentDir); } - throw new StorageAdapterLoadingException(e, e.getMessage()); + throw new SegmentLoadingException(e, e.getMessage()); } } } diff --git a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java index d7f60309aa6..276bbc2028a 100644 --- a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java @@ -27,5 +27,5 @@ import java.io.File; */ public interface QueryableIndexFactory { - public QueryableIndex factorize(File parentDir) throws StorageAdapterLoadingException; + public QueryableIndex factorize(File parentDir) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java new file mode 100644 index 00000000000..011e1633ca1 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java @@ -0,0 +1,170 @@ +/* + * 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.loading; + +import com.google.common.io.ByteStreams; +import com.google.common.io.Closeables; +import com.google.common.io.Files; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.common.s3.S3Utils; +import com.metamx.druid.utils.CompressionUtils; +import org.apache.commons.io.FileUtils; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.model.S3Bucket; +import org.jets3t.service.model.S3Object; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; +import java.util.zip.GZIPInputStream; + +/** + */ +public class S3DataSegmentPuller implements DataSegmentPuller +{ + private static final Logger log = new Logger(S3DataSegmentPuller.class); + + private static final String BUCKET = "bucket"; + private static final String KEY = "key"; + + private final RestS3Service s3Client; + + @Inject + public S3DataSegmentPuller( + RestS3Service s3Client + ) + { + this.s3Client = s3Client; + } + + @Override + public void getSegmentFiles(DataSegment segment, File outDir) throws SegmentLoadingException + { + S3Coords s3Coords = new S3Coords(segment); + + log.info("Pulling index at path[%s] to outDir[%s]", s3Coords, outDir); + + if (!isObjectInBucket(s3Coords)) { + throw new SegmentLoadingException("IndexFile[%s] does not exist.", s3Coords); + } + + if (!outDir.exists()) { + outDir.mkdirs(); + } + + if (!outDir.isDirectory()) { + throw new ISE("outDir[%s] must be a directory.", outDir); + } + + long startTime = System.currentTimeMillis(); + S3Object s3Obj = null; + + try { + s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); + + InputStream in = null; + try { + in = s3Obj.getDataInputStream(); + final String key = s3Obj.getKey(); + if (key.endsWith(".zip")) { + CompressionUtils.unzip(in, outDir); + } else if (key.endsWith(".gz")) { + final File outFile = new File(outDir, toFilename(key, ".gz")); + ByteStreams.copy(new GZIPInputStream(in), Files.newOutputStreamSupplier(outFile)); + } else { + ByteStreams.copy(in, Files.newOutputStreamSupplier(new File(outDir, toFilename(key, "")))); + } + log.info("Pull of file[%s] completed in %,d millis", s3Obj, System.currentTimeMillis() - startTime); + } + catch (IOException e) { + FileUtils.deleteDirectory(outDir); + throw new SegmentLoadingException(e, "Problem decompressing object[%s]", s3Obj); + } + finally { + Closeables.closeQuietly(in); + } + } + catch (Exception e) { + throw new SegmentLoadingException(e, e.getMessage()); + } + finally { + S3Utils.closeStreamsQuietly(s3Obj); + } + + } + + private String toFilename(String key, final String suffix) + { + String filename = key.substring(key.lastIndexOf("/") + 1); // characters after last '/' + filename = filename.substring(0, filename.length() - suffix.length()); // remove the suffix from the end + return filename; + } + + private boolean isObjectInBucket(S3Coords coords) throws SegmentLoadingException + { + try { + return s3Client.isObjectInBucket(coords.bucket, coords.path); + } + catch (ServiceException e) { + throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords); + } + } + + @Override + public long getLastModified(DataSegment segment) throws SegmentLoadingException + { + S3Coords coords = new S3Coords(segment); + try { + S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); + return objDetails.getLastModifiedDate().getTime(); + } + catch (S3ServiceException e) { + throw new SegmentLoadingException(e, e.getMessage()); + } + } + + private static class S3Coords + { + String bucket; + String path; + + public S3Coords(DataSegment segment) + { + Map loadSpec = segment.getLoadSpec(); + bucket = MapUtils.getString(loadSpec, BUCKET); + path = MapUtils.getString(loadSpec, KEY); + if (path.startsWith("/")) { + path = path.substring(1); + } + } + + public String toString() + { + return String.format("s3://%s/%s", bucket, path); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java similarity index 96% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java rename to server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index 5af4b905719..89a15b056ec 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -41,19 +41,19 @@ import java.security.NoSuchAlgorithmException; import java.util.zip.ZipEntry; import java.util.zip.ZipOutputStream; -public class S3SegmentPusher implements SegmentPusher +public class S3DataSegmentPusher implements DataSegmentPusher { - private static final EmittingLogger log = new EmittingLogger(S3SegmentPusher.class); + private static final EmittingLogger log = new EmittingLogger(S3DataSegmentPusher.class); private static final Joiner JOINER = Joiner.on("/").skipNulls(); private final RestS3Service s3Client; private final S3SegmentPusherConfig config; private final ObjectMapper jsonMapper; - public S3SegmentPusher( - RestS3Service s3Client, - S3SegmentPusherConfig config, - ObjectMapper jsonMapper + public S3DataSegmentPusher( + RestS3Service s3Client, + S3SegmentPusherConfig config, + ObjectMapper jsonMapper ) { this.s3Client = s3Client; diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java deleted file mode 100644 index f85a489b1fe..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java +++ /dev/null @@ -1,130 +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.loading; - -import com.google.inject.Inject; -import com.metamx.common.MapUtils; -import com.metamx.common.StreamUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.common.s3.S3Utils; -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; -import org.jets3t.service.model.S3Object; -import org.joda.time.DateTime; - -import java.io.File; -import java.util.Map; - -/** - */ -public class S3SegmentPuller implements SegmentPuller -{ - private static final Logger log = new Logger(S3SegmentPuller.class); - private static final long DEFAULT_TIMEOUT = 5 * 60 * 1000; - - private static final String BUCKET = "bucket"; - private static final String KEY = "key"; - - private final RestS3Service s3Client; - - @Inject - public S3SegmentPuller( - RestS3Service s3Client - ) - { - this.s3Client = s3Client; - } - - @Override - public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - S3Coords s3Coords = new S3Coords(segment); - - log.info("Loading index at path[%s]", s3Coords); - - if(!isObjectInBucket(s3Coords)){ - throw new StorageAdapterLoadingException("IndexFile[%s] does not exist.", s3Coords); - } - - long currTime = System.currentTimeMillis(); - File tmpFile = null; - S3Object s3Obj = null; - - try { - s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); - tmpFile = File.createTempFile(s3Coords.bucket, new DateTime().toString() + s3Coords.path.replace('/', '_')); - log.info("Downloading file[%s] to local tmpFile[%s] for segment[%s]", s3Coords, tmpFile, segment); - - StreamUtils.copyToFileAndClose(s3Obj.getDataInputStream(), tmpFile, DEFAULT_TIMEOUT); - final long downloadEndTime = System.currentTimeMillis(); - log.info("Download of file[%s] completed in %,d millis", tmpFile, downloadEndTime - currTime); - - return tmpFile; - } - catch (Exception e) { - if(tmpFile!=null && tmpFile.exists()){ - tmpFile.delete(); - } - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - finally { - S3Utils.closeStreamsQuietly(s3Obj); - } - } - - private boolean isObjectInBucket(S3Coords coords) throws StorageAdapterLoadingException { - try { - return s3Client.isObjectInBucket(coords.bucket, coords.path); - } catch (ServiceException e) { - throw new StorageAdapterLoadingException(e, "Problem communicating with S3 checking bucket/path[%s]", coords); - } - } - - @Override - public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { - S3Coords coords = new S3Coords(segment); - try { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); - return objDetails.getLastModifiedDate().getTime(); - } catch (S3ServiceException e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - } - - private class S3Coords { - String bucket; - String path; - - public S3Coords(DataSegment segment) { - Map loadSpec = segment.getLoadSpec(); - bucket = MapUtils.getString(loadSpec, BUCKET); - path = MapUtils.getString(loadSpec, KEY); - if(path.startsWith("/")){ - path = path.substring(1); - } - } - public String toString(){ - return String.format("s3://%s/%s", bucket, path); - } - } -} diff --git a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java deleted file mode 100644 index a3a7c724687..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java +++ /dev/null @@ -1,32 +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.loading; - -import org.jets3t.service.impl.rest.httpclient.RestS3Service; - -/** - * @deprecated - */ -public class S3ZippedSegmentPuller extends S3SegmentPuller -{ - public S3ZippedSegmentPuller(RestS3Service s3Client) { - super(s3Client); - } -} diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java index 1ca54b89106..20fa5592ac2 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java @@ -26,6 +26,6 @@ import com.metamx.druid.index.Segment; */ public interface SegmentLoader { - public Segment getSegment(DataSegment loadSpec) throws StorageAdapterLoadingException; - public void cleanup(DataSegment loadSpec) throws StorageAdapterLoadingException; + public Segment getSegment(DataSegment loadSpec) throws SegmentLoadingException; + public void cleanup(DataSegment loadSpec) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java similarity index 88% rename from server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java rename to server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java index d0f0ba4be93..d52fd6e3a82 100644 --- a/server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java @@ -21,9 +21,9 @@ package com.metamx.druid.loading; /** */ -public class StorageAdapterLoadingException extends Exception +public class SegmentLoadingException extends Exception { - public StorageAdapterLoadingException( + public SegmentLoadingException( String formatString, Object... objs ) @@ -31,7 +31,7 @@ public class StorageAdapterLoadingException extends Exception super(String.format(formatString, objs)); } - public StorageAdapterLoadingException( + public SegmentLoadingException( Throwable cause, String formatString, Object... objs diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index ae62cfda1e9..9aebfbd35f3 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -20,7 +20,6 @@ package com.metamx.druid.loading; import com.google.common.base.Joiner; -import com.google.common.io.Closeables; import com.google.inject.Inject; import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; @@ -29,12 +28,8 @@ import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; import com.metamx.druid.index.Segment; import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; import java.io.*; -import java.util.zip.GZIPInputStream; -import java.util.zip.ZipEntry; -import java.util.zip.ZipInputStream; /** */ @@ -42,24 +37,25 @@ public class SingleSegmentLoader implements SegmentLoader { private static final Logger log = new Logger(SingleSegmentLoader.class); - private final SegmentPuller segmentPuller; + private final DataSegmentPuller dataSegmentPuller; private final QueryableIndexFactory factory; private File cacheDirectory; private static final Joiner JOINER = Joiner.on("/").skipNulls(); @Inject public SingleSegmentLoader( - SegmentPuller segmentPuller, - QueryableIndexFactory factory, - File cacheDirectory) + DataSegmentPuller dataSegmentPuller, + QueryableIndexFactory factory, + File cacheDirectory + ) { - this.segmentPuller = segmentPuller; + this.dataSegmentPuller = dataSegmentPuller; this.factory = factory; this.cacheDirectory = cacheDirectory; } @Override - public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(DataSegment segment) throws SegmentLoadingException { File segmentFiles = getSegmentFiles(segment); final QueryableIndex index = factory.factorize(segmentFiles); @@ -67,43 +63,37 @@ public class SingleSegmentLoader implements SegmentLoader return new QueryableIndexSegment(segment.getIdentifier(), index); } - public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { + public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException + { File cacheFile = getCacheFile(segment); if (cacheFile.exists()) { long localLastModified = cacheFile.lastModified(); - long remoteLastModified = segmentPuller.getLastModified(segment); - if(remoteLastModified > 0 && localLastModified >= remoteLastModified){ + long remoteLastModified = dataSegmentPuller.getLastModified(segment); + if (remoteLastModified > 0 && localLastModified >= remoteLastModified) { log.info( "Found cacheFile[%s] with modified[%s], which is same or after remote[%s]. Using.", cacheFile, localLastModified, remoteLastModified ); - return cacheFile.getParentFile(); + return cacheFile; } } - File pulledFile = segmentPuller.getSegmentFiles(segment); + dataSegmentPuller.getSegmentFiles(segment, cacheFile); - if(!cacheFile.getParentFile().mkdirs()){ + if (!cacheFile.getParentFile().mkdirs()) { log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); } if (cacheFile.exists()) { cacheFile.delete(); } - if(pulledFile.getName().endsWith(".zip")){ - unzip(pulledFile, cacheFile.getParentFile()); - } else if(pulledFile.getName().endsWith(".gz")){ - gunzip(pulledFile, cacheFile); - } else { - moveToCache(pulledFile, cacheFile); - } - - return cacheFile.getParentFile(); + return cacheFile; } - private File getCacheFile(DataSegment segment) { + private File getCacheFile(DataSegment segment) + { String outputKey = JOINER.join( segment.getDataSource(), String.format( @@ -118,15 +108,22 @@ public class SingleSegmentLoader implements SegmentLoader return new File(cacheDirectory, outputKey); } - private void moveToCache(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + private void moveToCache(File pulledFile, File cacheFile) throws SegmentLoadingException + { log.info("Rename pulledFile[%s] to cacheFile[%s]", pulledFile, cacheFile); - if(!pulledFile.renameTo(cacheFile)){ + if (!pulledFile.renameTo(cacheFile)) { log.warn("Error renaming pulledFile[%s] to cacheFile[%s]. Copying instead.", pulledFile, cacheFile); try { StreamUtils.copyToFileAndClose(new FileInputStream(pulledFile), cacheFile); - } catch (IOException e) { - throw new StorageAdapterLoadingException(e,"Problem moving pulledFile[%s] to cache[%s]", pulledFile, cacheFile); + } + catch (IOException e) { + throw new SegmentLoadingException( + e, + "Problem moving pulledFile[%s] to cache[%s]", + pulledFile, + cacheFile + ); } if (!pulledFile.delete()) { log.error("Could not delete pulledFile[%s].", pulledFile); @@ -134,46 +131,8 @@ public class SingleSegmentLoader implements SegmentLoader } } - private void unzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { - log.info("Unzipping file[%s] to [%s]", pulledFile, cacheFile); - ZipInputStream zipIn = null; - OutputStream out = null; - ZipEntry entry = null; - try { - zipIn = new ZipInputStream(new BufferedInputStream(new FileInputStream(pulledFile))); - while ((entry = zipIn.getNextEntry()) != null) { - out = new FileOutputStream(new File(cacheFile, entry.getName())); - IOUtils.copy(zipIn, out); - zipIn.closeEntry(); - Closeables.closeQuietly(out); - out = null; - } - } catch(IOException e) { - throw new StorageAdapterLoadingException(e,"Problem unzipping[%s]", pulledFile); - } - finally { - Closeables.closeQuietly(out); - Closeables.closeQuietly(zipIn); - } - } - - private void gunzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { - log.info("Gunzipping file[%s] to [%s]", pulledFile, cacheFile); - try { - StreamUtils.copyToFileAndClose( - new GZIPInputStream(new FileInputStream(pulledFile)), - cacheFile - ); - } catch (IOException e) { - throw new StorageAdapterLoadingException(e,"Problem gunzipping[%s]", pulledFile); - } - if (!pulledFile.delete()) { - log.error("Could not delete tmpFile[%s].", pulledFile); - } - } - @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws SegmentLoadingException { File cacheFile = getCacheFile(segment).getParentFile(); @@ -182,8 +141,7 @@ public class SingleSegmentLoader implements SegmentLoader FileUtils.deleteDirectory(cacheFile); } catch (IOException e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); + throw new SegmentLoadingException(e, e.getMessage()); } } - } diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index 762662741a6..d10566c5da4 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -31,7 +31,6 @@ import com.metamx.common.Pair; import com.metamx.common.guava.ConcatSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Capabilities; import com.metamx.druid.Druids; import com.metamx.druid.Query; import com.metamx.druid.QueryGranularity; @@ -39,12 +38,9 @@ import com.metamx.druid.StorageAdapter; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.Segment; -import com.metamx.druid.index.brita.Filter; import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.index.v1.SegmentIdAttachedStorageAdapter; -import com.metamx.druid.index.v1.processing.Cursor; import com.metamx.druid.loading.SegmentLoader; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.ConcatQueryRunner; @@ -54,7 +50,6 @@ 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.query.search.SearchHit; import com.metamx.druid.query.search.SearchQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; @@ -62,7 +57,6 @@ import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceMetricEvent; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -72,7 +66,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.concurrent.ExecutorService; /** @@ -101,7 +94,7 @@ public class ServerManagerTest } @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws SegmentLoadingException { } @@ -245,7 +238,7 @@ public class ServerManagerTest ) ); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { throw new RuntimeException(e); } } @@ -267,7 +260,7 @@ public class ServerManagerTest ) ); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { throw new RuntimeException(e); } } diff --git a/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java index 29d784d3631..ca41c4dfec6 100644 --- a/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java +++ b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java @@ -30,7 +30,7 @@ import org.joda.time.Interval; public class NoopSegmentLoader implements SegmentLoader { @Override - public Segment getSegment(final DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(final DataSegment segment) throws SegmentLoadingException { return new Segment() { @@ -61,7 +61,7 @@ public class NoopSegmentLoader implements SegmentLoader } @Override - public void cleanup(DataSegment loadSpec) throws StorageAdapterLoadingException + public void cleanup(DataSegment loadSpec) throws SegmentLoadingException { } } From f8c54a72c2b61319ee8cbdd7b15bceb132a82703 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Tue, 19 Feb 2013 19:22:59 -0600 Subject: [PATCH 51/63] 1) Changes to allow for local storage --- .../druid/jackson/DefaultObjectMapper.java | 2 +- .../metamx/druid/utils/CompressionUtils.java | 38 ++++++- .../examples/RealtimeStandaloneMain.java | 6 +- .../druid/indexer/DeterminePartitionsJob.java | 3 +- .../indexer/HadoopDruidIndexerConfig.java | 3 +- .../druid/indexer/IndexGeneratorJob.java | 3 +- .../druid/merger/common/TaskToolbox.java | 10 +- .../common/index/YeOldePlumberSchool.java | 21 +++- .../druid/merger/common/task/DeleteTask.java | 6 +- .../druid/merger/common/task/MergeTask.java | 5 +- .../http/IndexerCoordinatorNode.java | 4 +- .../druid/merger/worker/http/WorkerNode.java | 4 +- .../druid/realtime/FireDepartmentConfig.java | 2 + .../metamx/druid/realtime/RealtimeNode.java | 36 ++++-- .../druid/realtime/RealtimePlumberSchool.java | 15 ++- .../druid/realtime/S3SegmentPusher.java | 4 +- .../com/metamx/druid/http/ComputeNode.java | 4 +- .../druid/initialization/ServerInit.java | 22 ++-- .../druid/loading/DataSegmentPusherUtil.java | 44 ++++++++ .../druid/loading/LocalDataSegmentPuller.java | 105 ++++++++++++++++++ .../druid/loading/LocalDataSegmentPusher.java | 96 ++++++++++++++++ ...java => LocalDataSegmentPusherConfig.java} | 6 +- .../loading/MMappedQueryableIndexFactory.java | 2 +- .../druid/loading/S3DataSegmentPusher.java | 51 ++------- ...ig.java => S3DataSegmentPusherConfig.java} | 2 +- ...erConfig.java => SegmentLoaderConfig.java} | 13 ++- .../druid/loading/SingleSegmentLoader.java | 46 ++++---- 27 files changed, 433 insertions(+), 120 deletions(-) create mode 100644 server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java create mode 100644 server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java create mode 100644 server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java rename server/src/main/java/com/metamx/druid/loading/{S3SegmentGetterConfig.java => LocalDataSegmentPusherConfig.java} (86%) rename server/src/main/java/com/metamx/druid/loading/{S3SegmentPusherConfig.java => S3DataSegmentPusherConfig.java} (95%) rename server/src/main/java/com/metamx/druid/loading/{QueryableLoaderConfig.java => SegmentLoaderConfig.java} (76%) diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index 84514bd6c4c..293f80d900d 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -174,7 +174,7 @@ public class DefaultObjectMapper extends ObjectMapper configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); configure(MapperFeature.AUTO_DETECT_GETTERS, false); - configure(MapperFeature.AUTO_DETECT_CREATORS, false); +// configure(MapperFeature.AUTO_DETECT_CREATORS, false); https://github.com/FasterXML/jackson-databind/issues/170 configure(MapperFeature.AUTO_DETECT_FIELDS, false); configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false); configure(MapperFeature.AUTO_DETECT_SETTERS, false); diff --git a/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java index 2b87d0a866d..c34b8e7e960 100644 --- a/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java +++ b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java @@ -21,6 +21,7 @@ package com.metamx.druid.utils; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; +import com.google.common.io.Files; import com.metamx.common.ISE; import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; @@ -29,6 +30,7 @@ import sun.misc.IOUtils; import java.io.BufferedInputStream; import java.io.File; import java.io.FileInputStream; +import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; @@ -36,6 +38,7 @@ import java.io.OutputStream; import java.util.zip.GZIPInputStream; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; +import java.util.zip.ZipOutputStream; /** */ @@ -43,10 +46,43 @@ public class CompressionUtils { private static final Logger log = new Logger(CompressionUtils.class); + public static long zip(File directory, File outputZipFile) throws IOException + { + if (!directory.isDirectory()) { + throw new IOException(String.format("directory[%s] is not a directory", directory)); + } + + if (!outputZipFile.getName().endsWith(".zip")) { + log.warn("No .zip suffix[%s], putting files from [%s] into it anyway.", outputZipFile, directory); + } + + long totalSize = 0; + ZipOutputStream zipOut = null; + try { + zipOut = new ZipOutputStream(new FileOutputStream(outputZipFile)); + File[] files = directory.listFiles(); + for (File file : files) { + log.info("Adding file[%s] with size[%,d]. Total size[%,d]", file, file.length(), totalSize); + if (file.length() >= Integer.MAX_VALUE) { + zipOut.close(); + outputZipFile.delete(); + throw new IOException(String.format("file[%s] too large [%,d]", file, file.length())); + } + zipOut.putNextEntry(new ZipEntry(file.getName())); + totalSize += ByteStreams.copy(Files.newInputStreamSupplier(file), zipOut); + } + } + finally { + Closeables.closeQuietly(zipOut); + } + + return totalSize; + } + public static void unzip(File pulledFile, File outDir) throws IOException { if (!(outDir.exists() && outDir.isDirectory())) { - throw new ISE("outDir[%s] must exist and be a directory"); + throw new ISE("outDir[%s] must exist and be a directory", outDir); } log.info("Unzipping file[%s] to [%s]", pulledFile, outDir); diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index ca5b9f64fd4..5f4d25cb95b 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -48,10 +48,12 @@ public class RealtimeStandaloneMain rn.setPhoneBook(dummyPhoneBook); MetadataUpdater dummyMetadataUpdater = - new MetadataUpdater(new DefaultObjectMapper(), + new MetadataUpdater( + new DefaultObjectMapper(), Config.createFactory(Initialization.loadProperties()).build(MetadataUpdaterConfig.class), dummyPhoneBook, - null) { + null + ) { @Override public void publishSegment(DataSegment segment) throws IOException { diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index d4ee1941396..9a72d997987 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -266,8 +266,7 @@ public class DeterminePartitionsJob implements Jobby Context context ) throws IOException, InterruptedException { - // Create group key - // TODO -- There are more efficient ways to do this + // Create group key, there are probably more efficient ways of doing this final Map> dims = Maps.newTreeMap(); for(final String dim : inputRow.getDimensions()) { final Set dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim)); diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index eed2339114b..979e2d989a4 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -662,8 +662,9 @@ public class HadoopDruidIndexerConfig return new Path( String.format( - "%s/%s_%s/%s/%s", + "%s/%s/%s_%s/%s/%s", getSegmentOutputDir(), + dataSource, bucketInterval.getStart().toString(), bucketInterval.getEnd().toString(), getVersion().toString(), diff --git a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index d8eba264c11..0620ba2bc85 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -379,7 +379,8 @@ public class IndexGeneratorJob implements Jobby ); } else if (outputFS instanceof LocalFileSystem) { loadSpec = ImmutableMap.of( - "type", "test" + "type", "local", + "path", indexOutURI.getPath() ); } else { throw new ISE("Unknown file system[%s]", outputFS.getClass()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index f77b08ce713..0ab0cf49be6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -25,6 +25,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.merger.common.task.Task; @@ -94,7 +95,14 @@ public class TaskToolbox final SingleSegmentLoader loader = new SingleSegmentLoader( new S3DataSegmentPuller(s3Client), new MMappedQueryableIndexFactory(), - new File(config.getTaskDir(task), "fetched_segments") + new SegmentLoaderConfig() + { + @Override + public File getCacheDirectory() + { + return new File(config.getTaskDir(task), "fetched_segments"); + } + } ); Map retVal = Maps.newLinkedHashMap(); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index 703dbe898f7..c26888c4485 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -45,11 +45,11 @@ import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.Sink; - - +import org.apache.commons.io.FileUtils; import org.joda.time.Interval; import java.io.File; +import java.io.IOException; import java.util.List; import java.util.Set; @@ -120,13 +120,13 @@ public class YeOldePlumberSchool implements PlumberSchool @Override public void finishJob() { + // The segment we will upload + File fileToUpload = null; + try { // User should have persisted everything by now. Preconditions.checkState(!theSink.swappable(), "All data must be persisted before fininshing the job!"); - // The segment we will upload - final File fileToUpload; - if(spilled.size() == 0) { throw new IllegalStateException("Nothing indexed?"); } else if(spilled.size() == 1) { @@ -160,6 +160,17 @@ public class YeOldePlumberSchool implements PlumberSchool log.warn(e, "Failed to merge and upload"); throw Throwables.propagate(e); } + finally { + try { + if (fileToUpload != null) { + log.info("Deleting Index File[%s]", fileToUpload); + FileUtils.deleteDirectory(fileToUpload); + } + } + catch (IOException e) { + log.warn(e, "Error deleting directory[%s]", fileToUpload); + } + } } private void spillIfSwappable() diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 5f37ad2853e..3e1bee62e5d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -22,6 +22,7 @@ package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; +import com.google.common.io.Files; import com.metamx.common.logger.Logger; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; @@ -37,7 +38,7 @@ import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.shard.NoneShardSpec; - +import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -102,6 +103,9 @@ public class DeleteTask extends AbstractTask segment.getVersion() ); + log.info("Deleting Uploaded Files[%s]", fileToUpload); + FileUtils.deleteDirectory(fileToUpload); + return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 2cfec4e5d2a..5d062b12892 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -46,7 +46,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import org.apache.commons.codec.digest.DigestUtils; - +import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -168,6 +168,9 @@ public abstract class MergeTask extends AbstractTask emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); + log.info("Deleting Uploaded Files[%s]", fileToUpload); + FileUtils.deleteDirectory(fileToUpload); + return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)); } catch (Exception e) { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index ec5404086d9..af29c7da2b2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -51,7 +51,7 @@ import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -405,7 +405,7 @@ public class IndexerCoordinatorNode extends RegisteringNode ); final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, - configFactory.build(S3SegmentPusherConfig.class), + configFactory.build(S3DataSegmentPusherConfig.class), jsonMapper ); taskToolbox = new TaskToolbox(config, emitter, s3Client, dataSegmentPusher, jsonMapper); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index a152f0f003f..e71cf7f3e98 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -37,6 +37,7 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.S3DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -45,7 +46,6 @@ 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.config.WorkerConfig; -import com.metamx.druid.loading.S3SegmentPusherConfig; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; @@ -292,7 +292,7 @@ public class WorkerNode extends RegisteringNode ); final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, - configFactory.build(S3SegmentPusherConfig.class), + configFactory.build(S3DataSegmentPusherConfig.class), jsonMapper ); taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, dataSegmentPusher, jsonMapper); diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java index d98997b5051..efc0c7a598f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java @@ -19,6 +19,7 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; @@ -31,6 +32,7 @@ public class FireDepartmentConfig private final int maxRowsInMemory; private final Period intermediatePersistPeriod; + @JsonCreator public FireDepartmentConfig( @JsonProperty("maxRowsInMemory") int maxRowsInMemory, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index 536074d6247..087e87d107f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -47,8 +47,10 @@ import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusherConfig; import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; @@ -258,20 +260,30 @@ public class RealtimeNode extends BaseServerNode { if (dataSegmentPusher == null) { final Properties props = getProps(); - final RestS3Service s3Client; - try { - s3Client = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) + if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) { + dataSegmentPusher = new LocalDataSegmentPusher( + getConfigFactory().build(LocalDataSegmentPusherConfig.class), getJsonMapper() ); } - catch (S3ServiceException e) { - throw Throwables.propagate(e); - } + else { - dataSegmentPusher = new S3DataSegmentPusher(s3Client, getConfigFactory().build(S3SegmentPusherConfig.class), getJsonMapper()); + final RestS3Service s3Client; + try { + s3Client = new RestS3Service( + new AWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ); + } + catch (S3ServiceException e) { + throw Throwables.propagate(e); + } + + dataSegmentPusher = new S3DataSegmentPusher( + s3Client, getConfigFactory().build(S3DataSegmentPusherConfig.class), getJsonMapper() + ); + } } } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java index 70c12eaaa45..775dc7d5305 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -307,7 +307,7 @@ public class RealtimePlumberSchool implements PlumberSchool } } - final File mergedFile; + File mergedFile = null; try { List indexes = Lists.newArrayList(); for (FireHydrant fireHydrant : sink) { @@ -337,6 +337,19 @@ public class RealtimePlumberSchool implements PlumberSchool .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); + } + } } } ); diff --git a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java index 2e40c398bdc..007ea188b82 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java @@ -21,7 +21,7 @@ package com.metamx.druid.realtime; import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -35,7 +35,7 @@ public class S3SegmentPusher extends com.metamx.druid.loading.S3DataSegmentPushe { public S3SegmentPusher( RestS3Service s3Client, - S3SegmentPusherConfig config, + S3DataSegmentPusherConfig config, ObjectMapper jsonMapper ) { diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index 2230932d9a1..08b3eb93da3 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -39,7 +39,7 @@ import com.metamx.druid.coordination.ZkCoordinatorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.QueryableLoaderConfig; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.metrics.ServerMonitor; import com.metamx.druid.query.MetricsEmittingExecutorService; @@ -172,7 +172,7 @@ public class ComputeNode extends BaseServerNode ); setSegmentLoader( - ServerInit.makeDefaultQueryableLoader(s3Client, getConfigFactory().build(QueryableLoaderConfig.class)) + ServerInit.makeDefaultQueryableLoader(s3Client, getConfigFactory().build(SegmentLoaderConfig.class)) ); } catch (S3ServiceException e) { diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 82a5f263608..7cd6caf3c1b 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -26,15 +26,16 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.loading.DelegatingSegmentLoader; +import com.metamx.druid.loading.LocalDataSegmentPuller; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.query.group.GroupByQueryEngine; import com.metamx.druid.query.group.GroupByQueryEngineConfig; import com.metamx.druid.Query; import com.metamx.druid.collect.StupidPool; -import com.metamx.druid.loading.QueryableLoaderConfig; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; @@ -63,26 +64,23 @@ public class ServerInit public static SegmentLoader makeDefaultQueryableLoader( RestS3Service s3Client, - QueryableLoaderConfig config + SegmentLoaderConfig config ) { DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); final S3DataSegmentPuller segmentGetter = new S3DataSegmentPuller(s3Client); + final QueryableIndexFactory factory = new MMappedQueryableIndexFactory(); - final QueryableIndexFactory factory; - if ("mmap".equals(config.getQueryableFactoryType())) { - factory = new MMappedQueryableIndexFactory(); - } else { - throw new ISE("Unknown queryableFactoryType[%s]", config.getQueryableFactoryType()); - } + SingleSegmentLoader s3segmentLoader = new SingleSegmentLoader(segmentGetter, factory, config); + SingleSegmentLoader localSegmentLoader = new SingleSegmentLoader(new LocalDataSegmentPuller(), factory, config); - SingleSegmentLoader segmentLoader = new SingleSegmentLoader(segmentGetter, factory, config.getCacheDirectory()); delegateLoader.setLoaderTypes( ImmutableMap.builder() - .put("s3", segmentLoader) - .put("s3_zip", segmentLoader) - .build() + .put("s3", s3segmentLoader) + .put("s3_zip", s3segmentLoader) + .put("local", localSegmentLoader) + .build() ); return delegateLoader; diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java new file mode 100644 index 00000000000..e72bd787bb3 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java @@ -0,0 +1,44 @@ +/* + * 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.loading; + +import com.google.common.base.Joiner; +import com.metamx.druid.client.DataSegment; + +/** + */ +public class DataSegmentPusherUtil +{ + private static final Joiner JOINER = Joiner.on("/").skipNulls(); + + public static String getStorageDir(DataSegment segment) + { + return JOINER.join( + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart(), + segment.getInterval().getEnd() + ), + segment.getVersion(), + segment.getShardSpec().getPartitionNum() + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java new file mode 100644 index 00000000000..8cdb8e0a7a2 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java @@ -0,0 +1,105 @@ +/* + * 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.loading; + +import com.google.common.io.Files; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.utils.CompressionUtils; + +import java.io.File; +import java.io.IOException; +import java.util.Map; + +/** + */ +public class LocalDataSegmentPuller implements DataSegmentPuller +{ + private static final Logger log = new Logger(LocalDataSegmentPuller.class); + + @Override + public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException + { + final File path = getFile(segment); + + if (path.isDirectory()) { + if (path.equals(dir)) { + log.info("Asked to load [%s] into itself, done!", dir); + return; + } + + log.info("Copying files from [%s] to [%s]", path, dir); + File file = null; + try { + final File[] files = path.listFiles(); + for (int i = 0; i < files.length; ++i) { + file = files[i]; + Files.copy(file, new File(dir, file.getName())); + } + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Unable to copy file[%s].", file); + } + } else { + if (!path.getName().endsWith(".zip")) { + throw new SegmentLoadingException("File is not a zip file[%s]", path); + } + + log.info("Unzipping local file[%s] to [%s]", path, dir); + try { + CompressionUtils.unzip(path, dir); + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Unable to unzip file[%s]", path); + } + } + } + + @Override + public long getLastModified(DataSegment segment) throws SegmentLoadingException + { + final File file = getFile(segment); + + long lastModified = Long.MAX_VALUE; + if (file.isDirectory()) { + for (File childFile : file.listFiles()) { + lastModified = Math.min(childFile.lastModified(), lastModified); + } + } + else { + lastModified = file.lastModified(); + } + + return lastModified; + } + + private File getFile(DataSegment segment) throws SegmentLoadingException + { + final Map loadSpec = segment.getLoadSpec(); + final File path = new File(MapUtils.getString(loadSpec, "path")); + + if (!path.exists()) { + throw new SegmentLoadingException("Asked to load path[%s], but it doesn't exist.", path); + } + + return path; + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java new file mode 100644 index 00000000000..1493b162572 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java @@ -0,0 +1,96 @@ +/* + * 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.loading; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.utils.CompressionUtils; + +import java.io.File; +import java.io.IOException; + +/** + */ +public class LocalDataSegmentPusher implements DataSegmentPusher +{ + private static final Logger log = new Logger(LocalDataSegmentPusher.class); + + private final LocalDataSegmentPusherConfig config; + private final ObjectMapper jsonMapper; + + public LocalDataSegmentPusher( + LocalDataSegmentPusherConfig config, + ObjectMapper jsonMapper + ) + { + this.config = config; + this.jsonMapper = jsonMapper; + } + + @Override + public DataSegment push(File dataSegmentFile, DataSegment segment) throws IOException + { + File outDir = new File(config.getStorageDirectory(), DataSegmentPusherUtil.getStorageDir(segment)); + + if (dataSegmentFile.equals(outDir)) { + long size = 0; + for (File file : dataSegmentFile.listFiles()) { + size += file.length(); + } + + return createDescriptorFile( + segment.withLoadSpec(makeLoadSpec(outDir)) + .withSize(size) + .withBinaryVersion(IndexIO.getVersionFromDir(dataSegmentFile)), + outDir + ); + } + + outDir.mkdirs(); + File outFile = new File(outDir, "index.zip"); + log.info("Compressing files from[%s] to [%s]", dataSegmentFile, outFile); + long size = CompressionUtils.zip(dataSegmentFile, outFile); + + return createDescriptorFile( + segment.withLoadSpec(makeLoadSpec(outFile)) + .withSize(size) + .withBinaryVersion(IndexIO.getVersionFromDir(dataSegmentFile)), + outDir + ); + } + + private DataSegment createDescriptorFile(DataSegment segment, File outDir) throws IOException + { + File descriptorFile = new File(outDir, "descriptor.json"); + log.info("Creating descriptor file at[%s]", descriptorFile); + Files.copy(ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(segment)), descriptorFile); + return segment; + } + + private ImmutableMap makeLoadSpec(File outFile) + { + return ImmutableMap.of("type", "local", "path", outFile.toString()); + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java similarity index 86% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java rename to server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java index c2a4c7f6308..d33a9a5130b 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java @@ -25,8 +25,8 @@ import java.io.File; /** */ -public abstract class S3SegmentGetterConfig +public abstract class LocalDataSegmentPusherConfig { - @Config("druid.paths.indexCache") - public abstract File getCacheDirectory(); + @Config("druid.pusher.local.storageDirectory") + public abstract File getStorageDirectory(); } diff --git a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java index 9f8594a30d2..9896c3f800b 100644 --- a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java @@ -66,7 +66,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory catch (IOException e2) { log.error(e, "Problem deleting parentDir[%s]", parentDir); } - throw new SegmentLoadingException(e, e.getMessage()); + throw new SegmentLoadingException(e, "%s", e.getMessage()); } } } diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index 89a15b056ec..273a07d36f3 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -22,24 +22,20 @@ package com.metamx.druid.loading; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; -import com.google.common.io.Closeables; -import com.metamx.common.ISE; -import com.metamx.common.StreamUtils; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.utils.CompressionUtils; import com.metamx.emitter.EmittingLogger; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; - import org.jets3t.service.S3ServiceException; import org.jets3t.service.acl.gs.GSAccessControlList; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Object; -import java.io.*; +import java.io.File; +import java.io.IOException; import java.security.NoSuchAlgorithmException; -import java.util.zip.ZipEntry; -import java.util.zip.ZipOutputStream; public class S3DataSegmentPusher implements DataSegmentPusher { @@ -47,12 +43,12 @@ public class S3DataSegmentPusher implements DataSegmentPusher private static final Joiner JOINER = Joiner.on("/").skipNulls(); private final RestS3Service s3Client; - private final S3SegmentPusherConfig config; + private final S3DataSegmentPusherConfig config; private final ObjectMapper jsonMapper; public S3DataSegmentPusher( RestS3Service s3Client, - S3SegmentPusherConfig config, + S3DataSegmentPusherConfig config, ObjectMapper jsonMapper ) { @@ -67,35 +63,11 @@ public class S3DataSegmentPusher implements DataSegmentPusher log.info("Uploading [%s] to S3", indexFilesDir); String outputKey = JOINER.join( config.getBaseKey().isEmpty() ? null : config.getBaseKey(), - segment.getDataSource(), - String.format( - "%s_%s", - segment.getInterval().getStart(), - segment.getInterval().getEnd() - ), - segment.getVersion(), - segment.getShardSpec().getPartitionNum() + DataSegmentPusherUtil.getStorageDir(segment) ); - long indexSize = 0; final File zipOutFile = File.createTempFile("druid", "index.zip"); - ZipOutputStream zipOut = null; - try { - zipOut = new ZipOutputStream(new FileOutputStream(zipOutFile)); - File[] indexFiles = indexFilesDir.listFiles(); - for (File indexFile : indexFiles) { - log.info("Adding indexFile[%s] with size[%,d]. Total size[%,d]", indexFile, indexFile.length(), indexSize); - if (indexFile.length() >= Integer.MAX_VALUE) { - throw new ISE("indexFile[%s] too large [%,d]", indexFile, indexFile.length()); - } - zipOut.putNextEntry(new ZipEntry(indexFile.getName())); - IOUtils.copy(new FileInputStream(indexFile), zipOut); - indexSize += indexFile.length(); - } - } - finally { - Closeables.closeQuietly(zipOut); - } + long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile); try { S3Object toPush = new S3Object(zipOutFile); @@ -119,7 +91,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher .withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir)); File descriptorFile = File.createTempFile("druid", "descriptor.json"); - StreamUtils.copyToFileAndClose(new ByteArrayInputStream(jsonMapper.writeValueAsBytes(segment)), descriptorFile); + Files.copy(ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(segment)), descriptorFile); S3Object descriptorObject = new S3Object(descriptorFile); descriptorObject.setBucketName(outputBucket); descriptorObject.setKey(outputKey + "/descriptor.json"); @@ -128,9 +100,6 @@ public class S3DataSegmentPusher implements DataSegmentPusher log.info("Pushing %s", descriptorObject); s3Client.putObject(outputBucket, descriptorObject); - log.info("Deleting Index File[%s]", indexFilesDir); - FileUtils.deleteDirectory(indexFilesDir); - log.info("Deleting zipped index File[%s]", zipOutFile); zipOutFile.delete(); diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java similarity index 95% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java rename to server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java index 0bd66a1a913..a2cada422fb 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java @@ -24,7 +24,7 @@ import org.skife.config.Default; /** */ -public abstract class S3SegmentPusherConfig +public abstract class S3DataSegmentPusherConfig { @Config("druid.pusher.s3.bucket") public abstract String getBucket(); diff --git a/server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java similarity index 76% rename from server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java rename to server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java index c6e2a72c931..294c91b9a38 100644 --- a/server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java @@ -21,13 +21,18 @@ package com.metamx.druid.loading; import org.skife.config.Config; +import java.io.File; + /** */ -public abstract class QueryableLoaderConfig extends S3SegmentGetterConfig +public abstract class SegmentLoaderConfig { - @Config("druid.queryable.factory") - public String getQueryableFactoryType() + @Config({"druid.paths.indexCache", "druid.segmentCache.path"}) + public abstract File getCacheDirectory(); + + @Config("druid.segmentCache.deleteOnRemove") + public boolean deleteOnRemove() { - return "mmap"; + return true; } } diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index 9aebfbd35f3..7e62f57fbf4 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -20,6 +20,7 @@ package com.metamx.druid.loading; import com.google.common.base.Joiner; +import com.google.common.base.Throwables; import com.google.inject.Inject; import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; @@ -39,19 +40,19 @@ public class SingleSegmentLoader implements SegmentLoader private final DataSegmentPuller dataSegmentPuller; private final QueryableIndexFactory factory; - private File cacheDirectory; + private final SegmentLoaderConfig config; private static final Joiner JOINER = Joiner.on("/").skipNulls(); @Inject public SingleSegmentLoader( DataSegmentPuller dataSegmentPuller, QueryableIndexFactory factory, - File cacheDirectory + SegmentLoaderConfig config ) { this.dataSegmentPuller = dataSegmentPuller; this.factory = factory; - this.cacheDirectory = cacheDirectory; + this.config = config; } @Override @@ -65,34 +66,37 @@ public class SingleSegmentLoader implements SegmentLoader public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException { - File cacheFile = getCacheFile(segment); - if (cacheFile.exists()) { - long localLastModified = cacheFile.lastModified(); + File localStorageDir = new File(config.getCacheDirectory(), DataSegmentPusherUtil.getStorageDir(segment)); + if (localStorageDir.exists()) { + long localLastModified = localStorageDir.lastModified(); long remoteLastModified = dataSegmentPuller.getLastModified(segment); if (remoteLastModified > 0 && localLastModified >= remoteLastModified) { log.info( - "Found cacheFile[%s] with modified[%s], which is same or after remote[%s]. Using.", - cacheFile, - localLastModified, - remoteLastModified + "Found localStorageDir[%s] with modified[%s], which is same or after remote[%s]. Using.", + localStorageDir, localLastModified, remoteLastModified ); - return cacheFile; + return localStorageDir; } } - dataSegmentPuller.getSegmentFiles(segment, cacheFile); - - if (!cacheFile.getParentFile().mkdirs()) { - log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); + if (localStorageDir.exists()) { + try { + FileUtils.deleteDirectory(localStorageDir); + } + catch (IOException e) { + log.warn(e, "Exception deleting previously existing local dir[%s]", localStorageDir); + } } - if (cacheFile.exists()) { - cacheFile.delete(); + if (!localStorageDir.mkdirs()) { + log.info("Unable to make parent file[%s]", localStorageDir); } - return cacheFile; + dataSegmentPuller.getSegmentFiles(segment, localStorageDir); + + return localStorageDir; } - private File getCacheFile(DataSegment segment) + private File getLocalStorageDir(DataSegment segment) { String outputKey = JOINER.join( segment.getDataSource(), @@ -105,7 +109,7 @@ public class SingleSegmentLoader implements SegmentLoader segment.getShardSpec().getPartitionNum() ); - return new File(cacheDirectory, outputKey); + return new File(config.getCacheDirectory(), outputKey); } private void moveToCache(File pulledFile, File cacheFile) throws SegmentLoadingException @@ -134,7 +138,7 @@ public class SingleSegmentLoader implements SegmentLoader @Override public void cleanup(DataSegment segment) throws SegmentLoadingException { - File cacheFile = getCacheFile(segment).getParentFile(); + File cacheFile = getLocalStorageDir(segment).getParentFile(); try { log.info("Deleting directory[%s]", cacheFile); From 606a2e4b01a60e2c0639d33a84565b839874edc8 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 15:49:55 -0600 Subject: [PATCH 52/63] [maven-release-plugin] prepare release druid-0.3.1 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 59cc716195c..22460ee95c5 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/common/pom.xml b/common/pom.xml index 710c88d7274..2642875bc5f 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 01078c607f4..1e5f30e4806 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.1-SNAPSHOT + 0.3.1 com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/examples/pom.xml b/examples/pom.xml index 4220ed243bc..75894e58a1a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index ae5e1e767e9..b39ec8fdd15 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index c557c46800b..4d714682f71 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/index-common/pom.xml b/index-common/pom.xml index c67359cb14e..f9b00d18c1c 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/indexer/pom.xml b/indexer/pom.xml index 0a32246ce06..deee2da0598 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/merger/pom.xml b/merger/pom.xml index e1c33c4b13b..cc73763841f 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/pom.xml b/pom.xml index b9b34479851..757862f4ba8 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.1-SNAPSHOT + 0.3.1 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 19553828205..c65b9f7e4db 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/server/pom.xml b/server/pom.xml index 09c44aea4b0..c10ff22b4d2 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 From 699aadd2b19f028cc4280b6807e39a3d2c45a7b2 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 15:50:01 -0600 Subject: [PATCH 53/63] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 22460ee95c5..453a44717a5 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 2642875bc5f..f813c06dca7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 1e5f30e4806..a2aa0136f94 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.1 + 0.3.2-SNAPSHOT com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 75894e58a1a..c2282b32a38 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index b39ec8fdd15..1b501a6239b 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 4d714682f71..a79dccedb98 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index f9b00d18c1c..2140c83512f 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index deee2da0598..0823e26e699 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index cc73763841f..e64b60808b1 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/pom.xml b/pom.xml index 757862f4ba8..ac1185a4a33 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.1 + 0.3.2-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index c65b9f7e4db..f336b4edf12 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index c10ff22b4d2..2bcf37d7054 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT From 5e123988a0c2b13ee1ae6dbde720c13c3e4b8398 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 20:37:51 -0600 Subject: [PATCH 54/63] 1) Specify old Jackson version to resolve version conflicts between AWS SDK and curator --- pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/pom.xml b/pom.xml index ac1185a4a33..02001be5ce0 100644 --- a/pom.xml +++ b/pom.xml @@ -192,6 +192,16 @@ jackson-jaxrs-json-provider 2.1.3 + + org.codehaus.jackson + jackson-core-asl + 1.9.11 + + + org.codehaus.jackson + jackson-mapper-asl + 1.9.11 + javax.inject javax.inject From 846bc0e4f0ee58997ef93f992b7197d20a1454a6 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 20:39:45 -0600 Subject: [PATCH 55/63] [maven-release-plugin] prepare release druid-0.3.2 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 453a44717a5..01fa2065885 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/common/pom.xml b/common/pom.xml index f813c06dca7..94c032684e4 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index a2aa0136f94..ac544f8a9eb 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.2-SNAPSHOT + 0.3.2 com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/examples/pom.xml b/examples/pom.xml index c2282b32a38..09bc6e84f53 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 1b501a6239b..ed856c5926b 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a79dccedb98..b151a08c5ad 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/index-common/pom.xml b/index-common/pom.xml index 2140c83512f..1ff0704164f 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/indexer/pom.xml b/indexer/pom.xml index 0823e26e699..ffca56c7de8 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/merger/pom.xml b/merger/pom.xml index e64b60808b1..b4c22f1944a 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/pom.xml b/pom.xml index 02001be5ce0..19f95b109b3 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.2-SNAPSHOT + 0.3.2 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index f336b4edf12..da2ed45cbdb 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/server/pom.xml b/server/pom.xml index 2bcf37d7054..62600955d21 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 From e0f6df1a5cc078d19d4ab3668bdfc0ab503c49cc Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 20:39:50 -0600 Subject: [PATCH 56/63] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 01fa2065885..7109bc96949 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 94c032684e4..10dbd719368 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index ac544f8a9eb..4022fbca433 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.2 + 0.3.3-SNAPSHOT com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 09bc6e84f53..ca22ebde9b9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index ed856c5926b..4cc3bc1514d 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index b151a08c5ad..15d7ec6b245 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 1ff0704164f..55353f21508 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index ffca56c7de8..9652c64ed78 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index b4c22f1944a..08f98b6389b 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/pom.xml b/pom.xml index 19f95b109b3..d24722c64cc 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.2 + 0.3.3-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index da2ed45cbdb..c9ed42132cf 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 62600955d21..5e73eab0396 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT From 8513a5ab2a59a925fca4763ccbc256bab7cfc0f2 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Sun, 24 Feb 2013 22:18:44 -0800 Subject: [PATCH 57/63] 1) Fix SimpleColumn to not produce NPEs when one of its parts is null. --- .../com/metamx/druid/index/column/SimpleColumn.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java index 93825a8e9f0..8179cd623e4 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java @@ -72,30 +72,30 @@ class SimpleColumn implements Column @Override public DictionaryEncodedColumn getDictionaryEncoding() { - return dictionaryEncodedColumn.get(); + return dictionaryEncodedColumn == null ? null : dictionaryEncodedColumn.get(); } @Override public RunLengthColumn getRunLengthColumn() { - return runLengthColumn.get(); + return runLengthColumn == null ? null : runLengthColumn.get(); } @Override public GenericColumn getGenericColumn() { - return genericColumn.get(); + return genericColumn == null ? null : genericColumn.get(); } @Override public ComplexColumn getComplexColumn() { - return complexColumn.get(); + return complexColumn == null ? null : complexColumn.get(); } @Override public BitmapIndex getBitmapIndex() { - return bitmapIndex.get(); + return bitmapIndex == null ? null : bitmapIndex.get(); } } From 7d7ce2b7feb98a8ec0f688e6312b0c88e6652192 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 10:57:26 -0800 Subject: [PATCH 58/63] Fix DeterminePartitionsJob ISE for dimensions not present in all rows --- .../druid/indexer/DeterminePartitionsJob.java | 30 ++++++++++++++----- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index 9a72d997987..f34ff2988f2 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -393,6 +393,9 @@ public class DeterminePartitionsJob implements Jobby final Interval interval = maybeInterval.get(); final byte[] groupKey = interval.getStart().toString().getBytes(Charsets.UTF_8); + // Emit row-counter value. + write(context, groupKey, new DimValueCount("", "", 1)); + for(final Map.Entry> dimAndValues : dims.entrySet()) { final String dim = dimAndValues.getKey(); @@ -509,9 +512,23 @@ public class DeterminePartitionsJob implements Jobby Context context, SortableBytes keyBytes, Iterable combinedIterable ) throws IOException, InterruptedException { - PeekingIterator iterator = Iterators.peekingIterator(combinedIterable.iterator()); + final DateTime bucket = new DateTime(new String(keyBytes.getGroupKey(), Charsets.UTF_8)); + final PeekingIterator iterator = Iterators.peekingIterator(combinedIterable.iterator()); - // "iterator" will take us over many candidate dimensions + log.info( + "Determining partitions for interval: %s", + config.getGranularitySpec().bucketInterval(bucket).orNull() + ); + + // First DVC should be the total row count indicator + final DimValueCount firstDvc = iterator.next(); + final int totalRows = firstDvc.numRows; + + if(!firstDvc.dim.equals("") || !firstDvc.value.equals("")) { + throw new IllegalStateException("WTF?! Expected total row indicator on first k/v pair!"); + } + + // "iterator" will now take us over many candidate dimensions DimPartitions currentDimPartitions = null; DimPartition currentDimPartition = null; String currentDimPartitionStart = null; @@ -635,8 +652,6 @@ public class DeterminePartitionsJob implements Jobby throw new ISE("No suitable partitioning dimension found!"); } - final int totalRows = dimPartitionss.values().iterator().next().getRows(); - int maxCardinality = Integer.MIN_VALUE; long minVariance = Long.MAX_VALUE; DimPartitions minVariancePartitions = null; @@ -644,12 +659,14 @@ public class DeterminePartitionsJob implements Jobby for(final DimPartitions dimPartitions : dimPartitionss.values()) { if(dimPartitions.getRows() != totalRows) { - throw new ISE( - "WTF?! Dimension[%s] row count %,d != expected row count %,d", + log.info( + "Dimension[%s] is not present in all rows (row count %,d != expected row count %,d)", dimPartitions.dim, dimPartitions.getRows(), totalRows ); + + continue; } // Make sure none of these shards are oversized @@ -683,7 +700,6 @@ public class DeterminePartitionsJob implements Jobby throw new ISE("No suitable partitioning dimension found!"); } - final DateTime bucket = new DateTime(new String(keyBytes.getGroupKey(), Charsets.UTF_8)); final OutputStream out = Utils.makePathAndOutputStream( context, config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)), config.isOverwriteFiles() ); From 99c4f9446c399b9e64bbcbb51158dd3617623b2a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 12:24:15 -0800 Subject: [PATCH 59/63] [maven-release-plugin] prepare release druid-0.3.3 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 7109bc96949..0774bd0c214 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/common/pom.xml b/common/pom.xml index 10dbd719368..17b376a2ee5 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 4022fbca433..0902c3e8064 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.3-SNAPSHOT + 0.3.3 com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/examples/pom.xml b/examples/pom.xml index ca22ebde9b9..976379c6678 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 4cc3bc1514d..179061524db 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 15d7ec6b245..344f5718bc0 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/index-common/pom.xml b/index-common/pom.xml index 55353f21508..d191045860b 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/indexer/pom.xml b/indexer/pom.xml index 9652c64ed78..2fd9c17894b 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/merger/pom.xml b/merger/pom.xml index 08f98b6389b..55fb1c76d82 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/pom.xml b/pom.xml index d24722c64cc..a1f3ed99638 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.3-SNAPSHOT + 0.3.3 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index c9ed42132cf..8ca33faab93 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/server/pom.xml b/server/pom.xml index 5e73eab0396..99f8ebcca4c 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 From 9964e7dfe446f6995a7fe8020084e5f4d07d277e Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 12:24:24 -0800 Subject: [PATCH 60/63] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 0774bd0c214..3af4d9b489a 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 17b376a2ee5..ef9c33b67e1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 0902c3e8064..b5b2a7e460c 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.3 + 0.3.4-SNAPSHOT com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 976379c6678..ae597675a62 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 179061524db..b9ff5296fde 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 344f5718bc0..88f4beae7da 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index d191045860b..34264feb745 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 2fd9c17894b..778f682a0f2 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 55fb1c76d82..e39359574a5 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/pom.xml b/pom.xml index a1f3ed99638..16d86ae8776 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.3 + 0.3.4-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 8ca33faab93..0e48bc168c0 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 99f8ebcca4c..e373da72f3d 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT From 14cf506c43d87e7c5ecb0d282a2478f7f049d5c4 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 17:05:01 -0800 Subject: [PATCH 61/63] DefaultObjectMapper: Add GuavaModule --- common/pom.xml | 8 ++++++-- .../com/metamx/druid/jackson/DefaultObjectMapper.java | 2 ++ pom.xml | 5 +++++ 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/common/pom.xml b/common/pom.xml index ef9c33b67e1..03483c15caa 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -80,8 +80,12 @@ jackson-databind - com.fasterxml.jackson.datatype - jackson-datatype-joda + com.fasterxml.jackson.datatype + jackson-datatype-guava + + + com.fasterxml.jackson.datatype + jackson-datatype-joda org.jdbi diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index 293f80d900d..12079a77959 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -36,6 +36,7 @@ import com.fasterxml.jackson.databind.SerializationFeature; import com.fasterxml.jackson.databind.SerializerProvider; import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.fasterxml.jackson.datatype.guava.GuavaModule; import com.fasterxml.jackson.datatype.joda.JodaModule; import com.google.common.base.Throwables; import com.metamx.common.Granularity; @@ -171,6 +172,7 @@ public class DefaultObjectMapper extends ObjectMapper } ); registerModule(serializerModule); + registerModule(new GuavaModule()); configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); configure(MapperFeature.AUTO_DETECT_GETTERS, false); diff --git a/pom.xml b/pom.xml index 16d86ae8776..1f1afee50e3 100644 --- a/pom.xml +++ b/pom.xml @@ -177,6 +177,11 @@ jackson-databind 2.1.4-mmx-2 + + com.fasterxml.jackson.datatype + jackson-datatype-guava + 2.1.2 + com.fasterxml.jackson.datatype jackson-datatype-joda From 2427e81874c117624fe9fc91af88e6e816d9368d Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 17:05:30 -0800 Subject: [PATCH 62/63] Merger: Feedback from code review --- .../common/actions/SegmentInsertAction.java | 23 +-------- .../common/actions/SegmentNukeAction.java | 23 +-------- .../common/actions/TaskActionToolbox.java | 48 +++++++++++++++++++ .../merger/common/config/TaskConfig.java | 2 +- .../merger/common/task/AbstractTask.java | 9 +--- .../task/IndexDeterminePartitionsTask.java | 12 ++++- .../common/task/IndexGeneratorTask.java | 14 +++++- .../druid/merger/common/task/IndexTask.java | 32 +++++++++---- .../druid/merger/common/task/MergeTask.java | 9 +++- .../merger/common/task/MergeTaskTest.java | 8 ++-- .../merger/common/task/TaskSerdeTest.java | 7 ++- .../coordinator/RemoteTaskRunnerTest.java | 2 +- .../merger/coordinator/TaskLifecycleTest.java | 8 ++-- 13 files changed, 123 insertions(+), 74 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java index 436c16e26fa..75ad4a9161f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java @@ -51,27 +51,8 @@ public class SegmentInsertAction implements TaskAction @Override public Void perform(TaskActionToolbox toolbox) { - // Verify that each of these segments-to-insert falls under some lock - // TODO: Should this be done while holding the giant lock on TaskLockbox? (To prevent anyone from grabbing - // TODO: these locks out from under us while the operation is ongoing.) Probably not necessary. - final List taskLocks = toolbox.getTaskLockbox().findLocksForTask(task); - for(final DataSegment segment : segments) { - final boolean ok = Iterables.any( - taskLocks, new Predicate() - { - @Override - public boolean apply(TaskLock taskLock) - { - return taskLock.getVersion().equals(segment.getVersion()) - && taskLock.getDataSource().equals(segment.getDataSource()) - && taskLock.getInterval().contains(segment.getInterval()); - } - } - ); - - if(!ok) { - throw new ISE("No currently-held lock covers segment: %s", segment); - } + if(!toolbox.taskLockCoversSegments(task, segments, false)) { + throw new ISE("Segments not covered by locks for task: %s", task.getId()); } try { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java index ca0d9f3a9bd..f1b61c58d9f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java @@ -51,27 +51,8 @@ public class SegmentNukeAction implements TaskAction @Override public Void perform(TaskActionToolbox toolbox) { - // Verify that each of these segments-to-nuke falls under some lock - // TODO: Should this be done while holding the giant lock on TaskLockbox? (To prevent anyone from grabbing - // TODO: these locks out from under us while the operation is ongoing.) Probably not necessary. - final List taskLocks = toolbox.getTaskLockbox().findLocksForTask(task); - for(final DataSegment segment : segments) { - final boolean ok = Iterables.any( - taskLocks, new Predicate() - { - @Override - public boolean apply(TaskLock taskLock) - { - return taskLock.getVersion().compareTo(segment.getVersion()) >= 0 - && taskLock.getDataSource().equals(segment.getDataSource()) - && taskLock.getInterval().contains(segment.getInterval()); - } - } - ); - - if(!ok) { - throw new ISE("No currently-held lock covers segment: %s", segment); - } + if(!toolbox.taskLockCoversSegments(task, segments, true)) { + throw new ISE("Segments not covered by locks for task: %s", task.getId()); } try { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java index 30f87a1f6c3..af8f6dcd40c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java @@ -1,10 +1,19 @@ package com.metamx.druid.merger.common.actions; +import com.google.common.base.Predicate; +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.druid.merger.coordinator.MergerDBCoordinator; import com.metamx.druid.merger.coordinator.TaskLockbox; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.emitter.service.ServiceEmitter; +import java.util.List; +import java.util.Set; + public class TaskActionToolbox { private final TaskQueue taskQueue; @@ -44,4 +53,43 @@ public class TaskActionToolbox { return emitter; } + + public boolean taskLockCoversSegments( + final Task task, + final Set segments, + final boolean allowOlderVersions + ) + { + // Verify that each of these segments falls under some lock + + // NOTE: It is possible for our lock to be revoked (if the task has failed and given up its locks) after we check + // NOTE: it and before we perform the segment insert, but, that should be OK since the worst that happens is we + // NOTE: insert some segments from the task but not others. + + final List taskLocks = getTaskLockbox().findLocksForTask(task); + for(final DataSegment segment : segments) { + final boolean ok = Iterables.any( + taskLocks, new Predicate() + { + @Override + public boolean apply(TaskLock taskLock) + { + final boolean versionOk = allowOlderVersions + ? taskLock.getVersion().compareTo(segment.getVersion()) >= 0 + : taskLock.getVersion().equals(segment.getVersion()); + + return versionOk + && taskLock.getDataSource().equals(segment.getDataSource()) + && taskLock.getInterval().contains(segment.getInterval()); + } + } + ); + + if (!ok) { + return false; + } + } + + return true; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java index c66009cd8ac..5b7609bd042 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java @@ -13,7 +13,7 @@ public abstract class TaskConfig @Config("druid.merger.rowFlushBoundary") @Default("500000") - public abstract long getRowFlushBoundary(); + public abstract int getDefaultRowFlushBoundary(); public File getTaskDir(final Task task) { return new File(getBaseTaskDir(), task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index 53653aa6595..bb65225a6a5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -69,20 +69,13 @@ public abstract class AbstractTask implements Task return dataSource; } + @JsonProperty("interval") @Override public Optional getFixedInterval() { return interval; } - // Awesome hack to get around lack of serde for Optional - // TODO Look into jackson-datatype-guava - @JsonProperty("interval") - private Interval getNullableIntervalForJackson() - { - return interval.orNull(); - } - @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index b94625e7ffa..569aa3e8a29 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -50,11 +50,16 @@ public class IndexDeterminePartitionsTask extends AbstractTask { @JsonProperty private final FirehoseFactory firehoseFactory; + @JsonProperty private final Schema schema; + @JsonProperty private final long targetPartitionSize; + @JsonProperty + private final int rowFlushBoundary; + private static final Logger log = new Logger(IndexTask.class); @JsonCreator @@ -63,7 +68,8 @@ public class IndexDeterminePartitionsTask extends AbstractTask @JsonProperty("interval") Interval interval, @JsonProperty("firehose") FirehoseFactory firehoseFactory, @JsonProperty("schema") Schema schema, - @JsonProperty("targetPartitionSize") long targetPartitionSize + @JsonProperty("targetPartitionSize") long targetPartitionSize, + @JsonProperty("rowFlushBoundary") int rowFlushBoundary ) { super( @@ -81,6 +87,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask this.firehoseFactory = firehoseFactory; this.schema = schema; this.targetPartitionSize = targetPartitionSize; + this.rowFlushBoundary = rowFlushBoundary; } @Override @@ -244,7 +251,8 @@ public class IndexDeterminePartitionsTask extends AbstractTask schema.getAggregators(), schema.getIndexGranularity(), shardSpec - ) + ), + rowFlushBoundary ); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index 514f6cb7461..f4daec2d90a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -58,6 +58,9 @@ public class IndexGeneratorTask extends AbstractTask @JsonProperty private final Schema schema; + @JsonProperty + private final int rowFlushBoundary; + private static final Logger log = new Logger(IndexTask.class); @JsonCreator @@ -65,7 +68,8 @@ public class IndexGeneratorTask extends AbstractTask @JsonProperty("groupId") String groupId, @JsonProperty("interval") Interval interval, @JsonProperty("firehose") FirehoseFactory firehoseFactory, - @JsonProperty("schema") Schema schema + @JsonProperty("schema") Schema schema, + @JsonProperty("rowFlushBoundary") int rowFlushBoundary ) { super( @@ -83,6 +87,7 @@ public class IndexGeneratorTask extends AbstractTask this.firehoseFactory = firehoseFactory; this.schema = schema; + this.rowFlushBoundary = rowFlushBoundary; } @Override @@ -139,6 +144,11 @@ public class IndexGeneratorTask extends AbstractTask tmpDir ).findPlumber(schema, metrics); + // rowFlushBoundary for this job + final int myRowFlushBoundary = this.rowFlushBoundary > 0 + ? rowFlushBoundary + : toolbox.getConfig().getDefaultRowFlushBoundary(); + try { while(firehose.hasMore()) { final InputRow inputRow = firehose.nextRow(); @@ -157,7 +167,7 @@ public class IndexGeneratorTask extends AbstractTask int numRows = sink.add(inputRow); metrics.incrementProcessed(); - if(numRows >= toolbox.getConfig().getRowFlushBoundary()) { + if(numRows >= myRowFlushBoundary) { plumber.persist(firehose.commit()); } } else { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 8cfaa15d300..246389470ef 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -42,11 +42,23 @@ import java.util.List; public class IndexTask extends AbstractTask { - @JsonProperty private final GranularitySpec granularitySpec; - @JsonProperty private final AggregatorFactory[] aggregators; - @JsonProperty private final QueryGranularity indexGranularity; - @JsonProperty private final long targetPartitionSize; - @JsonProperty private final FirehoseFactory firehoseFactory; + @JsonProperty + private final GranularitySpec granularitySpec; + + @JsonProperty + private final AggregatorFactory[] aggregators; + + @JsonProperty + private final QueryGranularity indexGranularity; + + @JsonProperty + private final long targetPartitionSize; + + @JsonProperty + private final FirehoseFactory firehoseFactory; + + @JsonProperty + private final int rowFlushBoundary; private static final Logger log = new Logger(IndexTask.class); @@ -57,7 +69,8 @@ public class IndexTask extends AbstractTask @JsonProperty("aggregators") AggregatorFactory[] aggregators, @JsonProperty("indexGranularity") QueryGranularity indexGranularity, @JsonProperty("targetPartitionSize") long targetPartitionSize, - @JsonProperty("firehose") FirehoseFactory firehoseFactory + @JsonProperty("firehose") FirehoseFactory firehoseFactory, + @JsonProperty("rowFlushBoundary") int rowFlushBoundary ) { super( @@ -75,6 +88,7 @@ public class IndexTask extends AbstractTask this.indexGranularity = indexGranularity; this.targetPartitionSize = targetPartitionSize; this.firehoseFactory = firehoseFactory; + this.rowFlushBoundary = rowFlushBoundary; } public List toSubtasks() @@ -95,7 +109,8 @@ public class IndexTask extends AbstractTask indexGranularity, new NoneShardSpec() ), - targetPartitionSize + targetPartitionSize, + rowFlushBoundary ) ); } else { @@ -110,7 +125,8 @@ public class IndexTask extends AbstractTask aggregators, indexGranularity, new NoneShardSpec() - ) + ), + rowFlushBoundary ) ); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 164c883d18b..f317efc7040 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -22,6 +22,7 @@ package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Objects; @@ -33,6 +34,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; +import com.google.common.hash.Hashing; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; @@ -46,7 +48,6 @@ import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import org.apache.commons.codec.digest.DigestUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -281,7 +282,11 @@ public abstract class MergeTask extends AbstractTask ) ); - return String.format("%s_%s", dataSource, DigestUtils.sha1Hex(segmentIDs).toLowerCase()); + return String.format( + "%s_%s", + dataSource, + Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase() + ); } private static Interval computeMergedInterval(final List segments) diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java index 5267b8c7983..0f1a7a66964 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java @@ -19,9 +19,10 @@ package com.metamx.druid.merger.common.task; +import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; +import com.google.common.hash.Hashing; import com.metamx.druid.client.DataSegment; -import org.apache.commons.codec.digest.DigestUtils; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -72,11 +73,12 @@ public class MergeTaskTest @Test public void testID() { - final String desiredPrefix = "merge_foo_" + DigestUtils.sha1Hex( + final String desiredPrefix = "merge_foo_" + Hashing.sha1().hashString( "2012-01-03T00:00:00.000Z_2012-01-05T00:00:00.000Z_V1_0" + "_2012-01-04T00:00:00.000Z_2012-01-06T00:00:00.000Z_V1_0" + "_2012-01-05T00:00:00.000Z_2012-01-07T00:00:00.000Z_V1_0" - ) + "_"; + , Charsets.UTF_8 + ).toString().toLowerCase() + "_"; Assert.assertEquals( desiredPrefix, testMergeTask.getId().substring(0, desiredPrefix.length()) diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java index 88b43a71667..51310bb2ef0 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java @@ -26,7 +26,8 @@ public class TaskSerdeTest new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, QueryGranularity.NONE, 10000, - null + null, + -1 ); final ObjectMapper jsonMapper = new DefaultObjectMapper(); @@ -52,7 +53,8 @@ public class TaskSerdeTest new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, QueryGranularity.NONE, new NoneShardSpec() - ) + ), + -1 ); final ObjectMapper jsonMapper = new DefaultObjectMapper(); @@ -97,6 +99,7 @@ public class TaskSerdeTest final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); + System.out.println(json); final Task task2 = jsonMapper.readValue(json, Task.class); Assert.assertEquals(task.getId(), task2.getId()); diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index b37ef435a41..5981bd8ed22 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -291,7 +291,7 @@ public class RemoteTaskRunnerTest } @Override - public long getRowFlushBoundary() + public int getDefaultRowFlushBoundary() { return 0; } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java index 3f469da6bb3..69a364e1900 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java @@ -106,7 +106,7 @@ public class TaskLifecycleTest } @Override - public long getRowFlushBoundary() + public int getDefaultRowFlushBoundary() { return 50000; } @@ -182,7 +182,8 @@ public class TaskLifecycleTest IR("2010-01-02T01", "a", "b", 2), IR("2010-01-02T01", "a", "c", 1) ) - ) + ), + -1 ); final TaskStatus mergedStatus = runTask(indexTask); @@ -216,7 +217,8 @@ public class TaskLifecycleTest new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, QueryGranularity.NONE, 10000, - newMockExceptionalFirehoseFactory() + newMockExceptionalFirehoseFactory(), + -1 ); final TaskStatus mergedStatus = runTask(indexTask); From d8fbddb9d418f71c316c9006a4b068d78788caaf Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 26 Feb 2013 11:36:07 -0800 Subject: [PATCH 63/63] Merger: Service discovery for worker -> master communication --- .../actions/RemoteTaskActionClient.java | 40 ++++++++++--- .../http/IndexerCoordinatorResource.java | 6 +- .../merger/worker/config/WorkerConfig.java | 3 + .../druid/merger/worker/http/WorkerNode.java | 58 ++++++++++++++++--- 4 files changed, 90 insertions(+), 17 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java index 2dc86ab3115..26900e29942 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java @@ -1,25 +1,31 @@ package com.metamx.druid.merger.common.actions; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; 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 java.net.URI; import java.net.URISyntaxException; +import java.util.Map; public class RemoteTaskActionClient implements TaskActionClient { private final HttpClient httpClient; + private final ServiceProvider serviceProvider; private final ObjectMapper jsonMapper; private static final Logger log = new Logger(RemoteTaskActionClient.class); - public RemoteTaskActionClient(HttpClient httpClient, ObjectMapper jsonMapper) + public RemoteTaskActionClient(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) { this.httpClient = httpClient; + this.serviceProvider = serviceProvider; this.jsonMapper = jsonMapper; } @@ -34,20 +40,36 @@ public class RemoteTaskActionClient implements TaskActionClient .go(new ToStringResponseHandler(Charsets.UTF_8)) .get(); - // TODO Figure out how to check HTTP status code - if(response.equals("")) { - return null; - } else { - return jsonMapper.readValue(response, taskAction.getReturnTypeReference()); - } + final Map responseDict = jsonMapper.readValue( + response, + new TypeReference>() {} + ); + + return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference()); } catch (Exception e) { throw Throwables.propagate(e); } } - public URI getServiceUri() throws URISyntaxException + private URI getServiceUri() throws Exception { - return new URI("http://localhost:8087/mmx/merger/v1/action"); + final ServiceInstance instance = serviceProvider.getInstance(); + final String scheme; + final String host; + final int port; + final String path = "/mmx/merger/v1/action"; + + host = instance.getAddress(); + + if (instance.getSslPort() != null && instance.getSslPort() > 0) { + scheme = "https"; + port = instance.getSslPort(); + } else { + scheme = "http"; + port = instance.getPort(); + } + + return new URI(scheme, null, host, port, path, null, null); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index b01e27a9316..fd3e02bcad3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -21,6 +21,7 @@ package com.metamx.druid.merger.coordinator.http; import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; @@ -176,6 +177,9 @@ public class IndexerCoordinatorResource public Response doAction(final TaskAction action) { final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action); - return Response.ok().entity(ret).build(); + final Map retMap = Maps.newHashMap(); + retMap.put("result", ret); + + return Response.ok().entity(retMap).build(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java index 5b5f3a0a6e7..51a11546d24 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java @@ -39,6 +39,9 @@ public abstract class WorkerConfig @Config("druid.worker.version") public abstract String getVersion(); + @Config("druid.worker.masterService") + public abstract String getMasterService(); + public int getCapacity() { return Runtime.getRuntime().availableProcessors() - 1; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 06ba0f73a32..d02ffa5d9e3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -35,6 +35,7 @@ import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; +import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3DataSegmentPusher; @@ -64,6 +65,8 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; +import com.netflix.curator.x.discovery.ServiceDiscovery; +import com.netflix.curator.x.discovery.ServiceProvider; import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; @@ -105,6 +108,8 @@ public class WorkerNode extends RegisteringNode private WorkerConfig workerConfig = null; private TaskToolbox taskToolbox = null; private CuratorFramework curatorFramework = null; + private ServiceDiscovery serviceDiscovery = null; + private ServiceProvider coordinatorServiceProvider = null; private WorkerCuratorCoordinator workerCuratorCoordinator = null; private TaskMonitor taskMonitor = null; private Server server = null; @@ -156,6 +161,18 @@ public class WorkerNode extends RegisteringNode return this; } + public WorkerNode setCoordinatorServiceProvider(ServiceProvider coordinatorServiceProvider) + { + this.coordinatorServiceProvider = coordinatorServiceProvider; + return this; + } + + public WorkerNode setServiceDiscovery(ServiceDiscovery serviceDiscovery) + { + this.serviceDiscovery = serviceDiscovery; + return this; + } + public WorkerNode setWorkerCuratorCoordinator(WorkerCuratorCoordinator workerCuratorCoordinator) { this.workerCuratorCoordinator = workerCuratorCoordinator; @@ -175,10 +192,12 @@ public class WorkerNode extends RegisteringNode initializeS3Service(); initializeMonitors(); initializeMergerConfig(); + initializeCuratorFramework(); + initializeServiceDiscovery(); + initializeCoordinatorServiceProvider(); initializeTaskToolbox(); initializeJacksonInjections(); initializeJacksonSubtypes(); - initializeCuratorFramework(); initializeCuratorCoordinator(); initializeTaskMonitor(); initializeServer(); @@ -328,7 +347,7 @@ public class WorkerNode extends RegisteringNode ); taskToolbox = new TaskToolbox( taskConfig, - new RemoteTaskActionClient(httpClient, jsonMapper), + new RemoteTaskActionClient(httpClient, coordinatorServiceProvider, jsonMapper), emitter, s3Service, dataSegmentPusher, @@ -340,11 +359,36 @@ public class WorkerNode extends RegisteringNode public void initializeCuratorFramework() throws IOException { - final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class); - curatorFramework = Initialization.makeCuratorFrameworkClient( - curatorConfig, - lifecycle - ); + if (curatorFramework == null) { + final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class); + curatorFramework = Initialization.makeCuratorFrameworkClient( + curatorConfig, + lifecycle + ); + } + } + + public void initializeServiceDiscovery() throws Exception + { + if (serviceDiscovery == null) { + final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class); + this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( + curatorFramework, + config, + lifecycle + ); + } + } + + public void initializeCoordinatorServiceProvider() + { + if (coordinatorServiceProvider == null) { + this.coordinatorServiceProvider = Initialization.makeServiceProvider( + workerConfig.getMasterService(), + serviceDiscovery, + lifecycle + ); + } } public void initializeCuratorCoordinator()