From c969b37bac2aeb9da6a70b55fd2b9115bc49484b Mon Sep 17 00:00:00 2001 From: Ray Sayre Date: Wed, 18 Dec 2013 13:01:36 -0800 Subject: [PATCH 01/31] Remove unused payload column frosub-select that fails in postgres. Use boolean true/false for 'used' column so that queries run in postres --- .../main/java/io/druid/db/DatabaseRuleManager.java | 2 +- .../java/io/druid/db/DatabaseSegmentManager.java | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/io/druid/db/DatabaseRuleManager.java b/server/src/main/java/io/druid/db/DatabaseRuleManager.java index 0ba44c52c85..036acda23d6 100644 --- a/server/src/main/java/io/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/io/druid/db/DatabaseRuleManager.java @@ -207,7 +207,7 @@ public class DatabaseRuleManager String.format( "SELECT r.dataSource, r.payload " + "FROM %1$s r " - + "INNER JOIN(SELECT dataSource, max(version) as version, payload FROM %1$s GROUP BY dataSource) ds " + + "INNER JOIN(SELECT dataSource, max(version) as version FROM %1$s GROUP BY dataSource) ds " + "ON r.datasource = ds.datasource and r.version = ds.version", getRulesTable() ) diff --git a/server/src/main/java/io/druid/db/DatabaseSegmentManager.java b/server/src/main/java/io/druid/db/DatabaseSegmentManager.java index 203d082b806..3d68ad46978 100644 --- a/server/src/main/java/io/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/io/druid/db/DatabaseSegmentManager.java @@ -213,7 +213,7 @@ public class DatabaseSegmentManager for (DataSegment segment : segments) { batch.add( String.format( - "UPDATE %s SET used=1 WHERE id = '%s'", + "UPDATE %s SET used=true WHERE id = '%s'", getSegmentsTable(), segment.getIdentifier() ) @@ -244,7 +244,7 @@ public class DatabaseSegmentManager public Void withHandle(Handle handle) throws Exception { handle.createStatement( - String.format("UPDATE %s SET used=1 WHERE id = :id", getSegmentsTable()) + String.format("UPDATE %s SET used=true WHERE id = :id", getSegmentsTable()) ) .bind("id", segmentId) .execute(); @@ -278,7 +278,7 @@ public class DatabaseSegmentManager public Void withHandle(Handle handle) throws Exception { handle.createStatement( - String.format("UPDATE %s SET used=0 WHERE dataSource = :dataSource", getSegmentsTable()) + String.format("UPDATE %s SET used=false WHERE dataSource = :dataSource", getSegmentsTable()) ) .bind("dataSource", ds) .execute(); @@ -308,7 +308,7 @@ public class DatabaseSegmentManager public Void withHandle(Handle handle) throws Exception { handle.createStatement( - String.format("UPDATE %s SET used=0 WHERE id = :segmentID", getSegmentsTable()) + String.format("UPDATE %s SET used=false WHERE id = :segmentID", getSegmentsTable()) ).bind("segmentID", segmentID) .execute(); @@ -408,7 +408,7 @@ public class DatabaseSegmentManager public List> withHandle(Handle handle) throws Exception { return handle.createQuery( - String.format("SELECT payload FROM %s WHERE used=1", getSegmentsTable()) + String.format("SELECT payload FROM %s WHERE used=true", getSegmentsTable()) ).list(); } } @@ -465,4 +465,4 @@ public class DatabaseSegmentManager private String getSegmentsTable() { return dbTables.get().getSegmentsTable(); } -} \ No newline at end of file +} From 6fbe67eeeafd7887b5959becd1977834b6178212 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 19 Dec 2013 13:04:59 -0800 Subject: [PATCH 02/31] IndexerDBCoordinator: Work around SELECT -> INSERT races when adding segments --- .../overlord/IndexerDBCoordinator.java | 71 +++++++++++-------- 1 file changed, 43 insertions(+), 28 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java index f9db89b3fc9..7e5f3ef48dd 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java @@ -42,6 +42,7 @@ 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.exceptions.CallbackFailedException; import org.skife.jdbi.v2.tweak.HandleCallback; import java.io.IOException; @@ -169,39 +170,39 @@ public class IndexerDBCoordinator private boolean announceHistoricalSegment(final Handle handle, final DataSegment segment) throws IOException { try { - final List> exists = handle.createQuery( - String.format( - "SELECT id FROM %s WHERE id = :identifier", - dbTables.getSegmentsTable() - ) - ).bind( - "identifier", - segment.getIdentifier() - ).list(); - - if (!exists.isEmpty()) { + if (segmentExists(handle, segment)) { log.info("Found [%s] in DB, not updating DB", segment.getIdentifier()); return false; } - 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)", - dbTables.getSegmentsTable() - ) - ) - .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(); + // Try/catch to work around races due to SELECT -> INSERT. Avoid ON DUPLICATE KEY since it's not portable. + try { + 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)", + dbTables.getSegmentsTable() + ) + ) + .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()); + log.info("Published segment [%s] to DB", segment.getIdentifier()); + } catch (Exception e) { + if (e.getCause() instanceof SQLException && segmentExists(handle, segment)) { + log.info("Found [%s] in DB, not updating DB", segment.getIdentifier()); + } else { + throw e; + } + } } catch (IOException e) { log.error(e, "Exception inserting into DB"); @@ -211,6 +212,20 @@ public class IndexerDBCoordinator return true; } + private boolean segmentExists(final Handle handle, final DataSegment segment) { + final List> exists = handle.createQuery( + String.format( + "SELECT id FROM %s WHERE id = :identifier", + dbTables.getSegmentsTable() + ) + ).bind( + "identifier", + segment.getIdentifier() + ).list(); + + return !exists.isEmpty(); + } + public void updateSegmentMetadata(final Set segments) throws IOException { dbi.inTransaction( From 2d2fa319bd7ec58005ece620ec7c83c6ab4bab07 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 19 Dec 2013 13:05:10 -0800 Subject: [PATCH 03/31] pom.xml: Update emitter --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9849c18e1bd..f0b96500d4a 100644 --- a/pom.xml +++ b/pom.xml @@ -73,7 +73,7 @@ com.metamx emitter - 0.2.6 + 0.2.7 com.metamx From 566a3a6112381205b1883408e677ecbce0ef0d7d Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 19 Dec 2013 13:10:18 -0800 Subject: [PATCH 04/31] Indexing service: Break up segment actions Each one now one operates on at most a collection of segments that comprise a single partition. The main purpose of this change is to prevent audit log payload sizes from getting out of control. --- .../io/druid/indexing/common/TaskToolbox.java | 30 ++++++++++++++- .../common/actions/LocalTaskActionClient.java | 9 +++-- .../common/actions/SegmentInsertAction.java | 5 +-- .../actions/SegmentMetadataUpdateAction.java | 6 +-- .../common/actions/SegmentNukeAction.java | 6 +-- .../common/actions/TaskActionToolbox.java | 38 +++++++++++++++++++ .../indexing/common/task/DeleteTask.java | 3 +- .../indexing/common/task/HadoopIndexTask.java | 17 +++++---- .../druid/indexing/common/task/IndexTask.java | 2 +- .../druid/indexing/common/task/KillTask.java | 4 +- .../indexing/common/task/MergeTaskBase.java | 5 ++- .../druid/indexing/common/task/MoveTask.java | 10 +---- .../common/task/RealtimeIndexTask.java | 2 +- .../common/task/VersionConverterTask.java | 3 +- 14 files changed, 97 insertions(+), 43 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index eb34f4e4c1b..44f3c600f55 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -20,10 +20,15 @@ package io.druid.indexing.common; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; +import com.google.common.collect.Multimaps; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.ServerView; +import io.druid.indexing.common.actions.SegmentInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; @@ -37,10 +42,14 @@ import io.druid.segment.loading.SegmentLoader; import io.druid.segment.loading.SegmentLoadingException; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; +import org.joda.time.Interval; import java.io.File; +import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutorService; /** @@ -167,7 +176,7 @@ public class TaskToolbox return objectMapper; } - public Map getSegments(List segments) + public Map fetchSegments(List segments) throws SegmentLoadingException { Map retVal = Maps.newLinkedHashMap(); @@ -178,6 +187,25 @@ public class TaskToolbox return retVal; } + public void pushSegments(Iterable segments) throws IOException { + // Request segment pushes for each set + final Multimap segmentMultimap = Multimaps.index( + segments, + new Function() + { + @Override + public Interval apply(DataSegment segment) + { + return segment.getInterval(); + } + } + ); + for (final Collection segmentCollection : segmentMultimap.asMap().values()) { + getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(segmentCollection))); + } + + } + public File getTaskWorkDir() { return taskWorkDir; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java index 8bb23918b01..4dd445df80d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java @@ -19,6 +19,7 @@ package io.druid.indexing.common.actions; +import com.metamx.common.ISE; import com.metamx.emitter.EmittingLogger; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.TaskStorage; @@ -45,21 +46,21 @@ public class LocalTaskActionClient implements TaskActionClient { log.info("Performing action for task[%s]: %s", task.getId(), taskAction); - final RetType ret = taskAction.perform(task, toolbox); - if (taskAction.isAudited()) { // Add audit log try { storage.addAuditLog(task, taskAction); } catch (Exception e) { + final String actionClass = taskAction.getClass().getName(); log.makeAlert(e, "Failed to record action in audit log") .addData("task", task.getId()) - .addData("actionClass", taskAction.getClass().getName()) + .addData("actionClass", actionClass) .emit(); + throw new ISE(e, "Failed to record action [%s] in audit log", actionClass); } } - return ret; + return taskAction.perform(task, toolbox); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java index aaad73b8a9f..5280e394f6f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; -import com.metamx.common.ISE; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.task.Task; import io.druid.timeline.DataSegment; @@ -80,9 +79,7 @@ public class SegmentInsertAction implements TaskAction> @Override public Set perform(Task task, TaskActionToolbox toolbox) throws IOException { - if(!toolbox.taskLockCoversSegments(task, segments, allowOlderVersions)) { - throw new ISE("Segments not covered by locks for task[%s]: %s", task.getId(), segments); - } + toolbox.verifyTaskLocksAndSinglePartitionSettitude(task, segments, true); final Set retVal = toolbox.getIndexerDBCoordinator().announceHistoricalSegments(segments); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java index f996a2c6ab0..4356c80dc59 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java @@ -5,7 +5,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; -import com.metamx.common.ISE; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.task.Task; import io.druid.timeline.DataSegment; @@ -42,10 +41,7 @@ public class SegmentMetadataUpdateAction implements TaskAction Task task, TaskActionToolbox toolbox ) throws IOException { - if(!toolbox.taskLockCoversSegments(task, segments, true)) { - throw new ISE("Segments not covered by locks for task: %s", task.getId()); - } - + toolbox.verifyTaskLocksAndSinglePartitionSettitude(task, segments, true); toolbox.getIndexerDBCoordinator().updateSegmentMetadata(segments); // Emit metrics diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java index 6ac8dd1ccc4..54258df1c2d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; -import com.metamx.common.ISE; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.task.Task; import io.druid.timeline.DataSegment; @@ -59,10 +58,7 @@ public class SegmentNukeAction implements TaskAction @Override public Void perform(Task task, TaskActionToolbox toolbox) throws IOException { - if(!toolbox.taskLockCoversSegments(task, segments, true)) { - throw new ISE("Segments not covered by locks for task: %s", task.getId()); - } - + toolbox.verifyTaskLocksAndSinglePartitionSettitude(task, segments, true); toolbox.getIndexerDBCoordinator().deleteSegments(segments); // Emit metrics diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java index a0b41e58a63..10c4d627462 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java @@ -19,9 +19,11 @@ package io.druid.indexing.common.actions; +import com.google.api.client.repackaged.com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.inject.Inject; +import com.metamx.common.ISE; import com.metamx.emitter.service.ServiceEmitter; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.task.Task; @@ -65,6 +67,42 @@ public class TaskActionToolbox return emitter; } + public boolean segmentsAreFromSamePartitionSet( + final Set segments + ) + { + // Verify that these segments are all in the same partition set + + Preconditions.checkArgument(!segments.isEmpty(), "segments nonempty"); + final DataSegment firstSegment = segments.iterator().next(); + for (final DataSegment segment : segments) { + if (!segment.getDataSource().equals(firstSegment.getDataSource())) { + return false; + } + if (!segment.getInterval().equals(firstSegment.getInterval())) { + return false; + } + if (!segment.getVersion().equals(firstSegment.getVersion())) { + return false; + } + } + return true; + } + + public void verifyTaskLocksAndSinglePartitionSettitude( + final Task task, + final Set segments, + final boolean allowOlderVersions + ) + { + if (!taskLockCoversSegments(task, segments, allowOlderVersions)) { + throw new ISE("Segments not covered by locks for task: %s", task.getId()); + } + if (!segmentsAreFromSamePartitionSet(segments)) { + throw new ISE("Segments are not in the same partition set: %s", segments); + } + } + public boolean taskLockCoversSegments( final Task task, final Set segments, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java index 872ac3507bd..970818a6e9d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java @@ -22,6 +22,7 @@ package io.druid.indexing.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.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -103,7 +104,7 @@ public class DeleteTask extends AbstractFixedIntervalTask segment.getVersion() ); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); + toolbox.pushSegments(ImmutableList.of(uploadedSegment)); return TaskStatus.success(getId()); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index 400b088a693..233714f5c71 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -24,10 +24,14 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.api.client.util.Lists; +import com.google.common.base.Function; import com.google.common.base.Joiner; 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.google.common.collect.Multimap; +import com.google.common.collect.Multimaps; import com.metamx.common.logger.Logger; import io.druid.common.utils.JodaUtils; import io.druid.indexer.HadoopDruidIndexerConfig; @@ -47,12 +51,15 @@ import io.tesla.aether.internal.DefaultTeslaAether; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.File; import java.lang.reflect.Method; import java.net.URL; import java.net.URLClassLoader; import java.util.Arrays; +import java.util.Collection; import java.util.List; +import java.util.Map; public class HadoopIndexTask extends AbstractFixedIntervalTask { @@ -180,14 +187,10 @@ public class HadoopIndexTask extends AbstractFixedIntervalTask if (segments != null) { List publishedSegments = toolbox.getObjectMapper().readValue( - segments, new TypeReference>() - { - } + segments, + new TypeReference>() {} ); - // Request segment pushes - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments))); - - // Done + toolbox.pushSegments(publishedSegments); return TaskStatus.success(getId()); } else { return TaskStatus.failure(getId()); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 19cb791b77e..6e09e46373c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -156,7 +156,7 @@ public class IndexTask extends AbstractFixedIntervalTask segments.add(segment); } } - toolbox.getTaskActionClient().submit(new SegmentInsertAction(segments)); + toolbox.pushSegments(segments); return TaskStatus.success(getId()); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java index c77ddb21d96..b4858342981 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java @@ -97,11 +97,9 @@ public class KillTask extends AbstractFixedIntervalTask // Kill segments for (DataSegment segment : unusedSegments) { toolbox.getDataSegmentKiller().kill(segment); + toolbox.getTaskActionClient().submit(new SegmentNukeAction(ImmutableSet.of(segment))); } - // Remove metadata for these segments - toolbox.getTaskActionClient().submit(new SegmentNukeAction(ImmutableSet.copyOf(unusedSegments))); - return TaskStatus.success(getId()); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java index d49d74b355b..40b07f72d71 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java @@ -27,6 +27,7 @@ import com.google.common.base.Joiner; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -142,7 +143,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask ); // download segments to merge - final Map gettedSegments = toolbox.getSegments(segments); + final Map gettedSegments = toolbox.fetchSegments(segments); // merge files together final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); @@ -165,7 +166,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); + toolbox.pushSegments(ImmutableList.of(uploadedSegment)); return TaskStatus.success(getId()); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java index 8e628b93188..da82ffa6608 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java @@ -98,18 +98,12 @@ public class MoveTask extends AbstractFixedIntervalTask log.info("OK to move segment: %s", unusedSegment.getIdentifier()); } - List movedSegments = Lists.newLinkedList(); - // Move segments for (DataSegment segment : unusedSegments) { - movedSegments.add(toolbox.getDataSegmentMover().move(segment, targetLoadSpec)); + final DataSegment movedSegment = toolbox.getDataSegmentMover().move(segment, targetLoadSpec); + toolbox.getTaskActionClient().submit(new SegmentMetadataUpdateAction(ImmutableSet.of(movedSegment))); } - // Update metadata for moved segments - toolbox.getTaskActionClient().submit(new SegmentMetadataUpdateAction( - ImmutableSet.copyOf(movedSegments) - )); - return TaskStatus.success(getId()); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index f28d83c1caf..6ddc523140b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -418,7 +418,7 @@ public class RealtimeIndexTask extends AbstractTask @Override public void publishSegment(DataSegment segment) throws IOException { - taskToolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); + taskToolbox.pushSegments(ImmutableList.of(segment)); } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java index 52807861a12..75561f2408e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; @@ -248,7 +249,7 @@ public class VersionConverterTask extends AbstractFixedIntervalTask } } - final Map localSegments = toolbox.getSegments(Arrays.asList(segment)); + final Map localSegments = toolbox.fetchSegments(Arrays.asList(segment)); final File location = localSegments.get(segment); final File outLocation = new File(location, "v9_out"); From 846c3da4ab2e94c17c39e636cb2ba10f174ec3e2 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 19 Dec 2013 13:21:41 -0800 Subject: [PATCH 05/31] Empty task intervals, and empty lock intervals, aren't useful. So prevent them from being created, through checks in AbstractFixedIntervalTask and TaskLockbox.tryLock. --- .../indexing/common/task/AbstractFixedIntervalTask.java | 4 +++- .../main/java/io/druid/indexing/overlord/TaskLockbox.java | 7 +++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractFixedIntervalTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractFixedIntervalTask.java index d1d494f5c83..2d6687ed920 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractFixedIntervalTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractFixedIntervalTask.java @@ -21,6 +21,7 @@ package io.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; import io.druid.indexing.common.actions.LockTryAcquireAction; import io.druid.indexing.common.actions.TaskActionClient; import org.joda.time.Interval; @@ -58,7 +59,8 @@ public abstract class AbstractFixedIntervalTask extends AbstractTask ) { super(id, groupId, taskResource, dataSource); - this.interval = interval; + this.interval = Preconditions.checkNotNull(interval, "interval"); + Preconditions.checkArgument(interval.toDurationMillis() > 0, "interval empty"); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index d486f37c0fc..7b3ffb09087 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -19,6 +19,7 @@ package io.druid.indexing.overlord; +import com.google.api.client.repackaged.com.google.common.base.Preconditions; import com.google.common.base.Function; import com.google.common.base.Objects; import com.google.common.base.Optional; @@ -109,6 +110,11 @@ public class TaskLockbox for (final Pair taskAndLock : byVersionOrdering.sortedCopy(storedLocks)) { final Task task = taskAndLock.lhs; final TaskLock savedTaskLock = taskAndLock.rhs; + if (savedTaskLock.getInterval().toDurationMillis() <= 0) { + // "Impossible", but you never know what crazy stuff can be restored from storage. + log.warn("WTF?! Got lock with empty interval for task: %s", task.getId()); + continue; + } uniqueTaskIds.add(task.getId()); final Optional acquiredTaskLock = tryLock( task, @@ -205,6 +211,7 @@ public class TaskLockbox giant.lock(); try { + Preconditions.checkArgument(interval.toDurationMillis() > 0, "interval empty"); final String dataSource = task.getDataSource(); final List foundPosses = findLockPossesForInterval(dataSource, interval); final TaskLockPosse posseToUse; From 1f4b99634fcd4dd17de55da7c1d56956d9cc6785 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 19 Dec 2013 16:11:30 -0800 Subject: [PATCH 06/31] Autoscaling: Move target count independent of actual count. This should let us grow and shrink the worker pool in chunks when necessary (like when a bunch of them go offline, or when there is a worker version change). --- .../indexing/overlord/RemoteTaskRunner.java | 9 +- .../overlord/scaling/AutoScalingData.java | 10 +- .../scaling/EC2AutoScalingStrategy.java | 8 +- .../SimpleResourceManagementStrategy.java | 399 ++++++++++-------- .../scaling/EC2AutoScalingStrategyTest.java | 2 - .../SimpleResourceManagementStrategyTest.java | 10 +- 6 files changed, 244 insertions(+), 194 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index 8b823444319..2c482b775f9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -27,6 +27,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; 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.Sets; @@ -251,26 +252,26 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer @Override public Collection getWorkers() { - return zkWorkers.values(); + return ImmutableList.copyOf(zkWorkers.values()); } @Override public Collection getRunningTasks() { - return runningTasks.values(); + return ImmutableList.copyOf(runningTasks.values()); } @Override public Collection getPendingTasks() { - return pendingTasks.values(); + return ImmutableList.copyOf(pendingTasks.values()); } @Override public Collection getKnownTasks() { // Racey, since there is a period of time during assignment when a task is neither pending nor running - return Lists.newArrayList(Iterables.concat(pendingTasks.values(), runningTasks.values(), completeTasks.values())); + return ImmutableList.copyOf(Iterables.concat(pendingTasks.values(), runningTasks.values(), completeTasks.values())); } public ZkWorker findWorkerRunningTask(String taskId) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/AutoScalingData.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/AutoScalingData.java index 7a0ab258310..34c45d66e71 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/AutoScalingData.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/AutoScalingData.java @@ -28,12 +28,10 @@ import java.util.List; public class AutoScalingData { private final List nodeIds; - private final List nodes; - public AutoScalingData(List nodeIds, List nodes) + public AutoScalingData(List nodeIds) { this.nodeIds = nodeIds; - this.nodes = nodes; } @JsonProperty @@ -42,17 +40,11 @@ public class AutoScalingData return nodeIds; } - public List getNodes() - { - return nodes; - } - @Override public String toString() { return "AutoScalingData{" + "nodeIds=" + nodeIds + - ", nodes=" + nodes + '}'; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java index 57c2d875ac5..b59f3d1e74e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java @@ -125,8 +125,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy return input.getInstanceId(); } } - ), - result.getReservation().getInstances() + ) ); } catch (Exception e) { @@ -140,7 +139,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy public AutoScalingData terminate(List ips) { if (ips.isEmpty()) { - return new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()); + return new AutoScalingData(Lists.newArrayList()); } DescribeInstancesResult result = amazonEC2Client.describeInstances( @@ -184,8 +183,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy return String.format("%s:%s", input, config.getWorkerPort()); } } - ), - instances + ) ); } catch (Exception e) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java index 50d612bd908..6b6fc613a7b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java @@ -19,14 +19,17 @@ package io.druid.indexing.overlord.scaling; +import com.google.api.client.repackaged.com.google.common.base.Joiner; import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.base.Supplier; +import com.google.common.collect.Collections2; +import com.google.common.collect.FluentIterable; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.inject.Inject; -import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.ISE; import com.metamx.emitter.EmittingLogger; import io.druid.indexing.overlord.RemoteTaskRunnerWorkItem; import io.druid.indexing.overlord.TaskRunnerWorkItem; @@ -38,7 +41,6 @@ import org.joda.time.Duration; import java.util.Collection; import java.util.List; import java.util.Set; -import java.util.concurrent.ConcurrentSkipListSet; /** */ @@ -48,211 +50,188 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat private final AutoScalingStrategy autoScalingStrategy; private final SimpleResourceManagementConfig config; - private final Supplier workerSetupdDataRef; + private final Supplier workerSetupDataRef; private final ScalingStats scalingStats; - private final ConcurrentSkipListSet currentlyProvisioning = new ConcurrentSkipListSet(); - private final ConcurrentSkipListSet currentlyTerminating = new ConcurrentSkipListSet(); + private final Object lock = new Object(); + private final Set currentlyProvisioning = Sets.newHashSet(); + private final Set currentlyTerminating = Sets.newHashSet(); + private final Predicate isLazyWorker = new Predicate() + { + @Override + public boolean apply(ZkWorker input) + { + return input.getRunningTasks().isEmpty() + && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() + >= config.getWorkerIdleTimeout().toStandardDuration().getMillis(); + } + }; - private volatile DateTime lastProvisionTime = new DateTime(); - private volatile DateTime lastTerminateTime = new DateTime(); + private int targetWorkerCount = -1; + private DateTime lastProvisionTime = new DateTime(); + private DateTime lastTerminateTime = new DateTime(); @Inject public SimpleResourceManagementStrategy( AutoScalingStrategy autoScalingStrategy, SimpleResourceManagementConfig config, - Supplier workerSetupdDataRef + Supplier workerSetupDataRef ) { this.autoScalingStrategy = autoScalingStrategy; this.config = config; - this.workerSetupdDataRef = workerSetupdDataRef; + this.workerSetupDataRef = workerSetupDataRef; this.scalingStats = new ScalingStats(config.getNumEventsToTrack()); } @Override public boolean doProvision(Collection pendingTasks, Collection zkWorkers) { - final WorkerSetupData workerSetupData = workerSetupdDataRef.get(); + synchronized (lock) { + boolean didProvision = false; + final WorkerSetupData workerSetupData = workerSetupDataRef.get(); + final Predicate isValidWorker = createValidWorkerPredicate(workerSetupData); + final int currValidWorkers = Collections2.filter(zkWorkers, isValidWorker).size(); - final String minVersion = workerSetupData.getMinVersion() == null - ? config.getWorkerVersion() - : workerSetupData.getMinVersion(); - int maxNumWorkers = workerSetupData.getMaxNumWorkers(); - - int currValidWorkers = 0; - for (ZkWorker zkWorker : zkWorkers) { - if (zkWorker.isValidVersion(minVersion)) { - currValidWorkers++; - } - } - - if (currValidWorkers >= maxNumWorkers) { - log.debug( - "Cannot scale anymore. Num workers = %d, Max num workers = %d", - zkWorkers.size(), - workerSetupdDataRef.get().getMaxNumWorkers() - ); - return false; - } - - List workerNodeIds = autoScalingStrategy.ipToIdLookup( - Lists.newArrayList( - Iterables.transform( - zkWorkers, - new Function() - { - @Override - public String apply(ZkWorker input) + final List workerNodeIds = autoScalingStrategy.ipToIdLookup( + Lists.newArrayList( + Iterables.transform( + zkWorkers, + new Function() { - return input.getWorker().getIp(); + @Override + public String apply(ZkWorker input) + { + return input.getWorker().getIp(); + } } - } - ) - ) - ); + ) + ) + ); + currentlyProvisioning.removeAll(workerNodeIds); - currentlyProvisioning.removeAll(workerNodeIds); - boolean nothingProvisioning = currentlyProvisioning.isEmpty(); + updateTargetWorkerCount(pendingTasks, zkWorkers); - if (nothingProvisioning) { - if (hasTaskPendingBeyondThreshold(pendingTasks)) { - AutoScalingData provisioned = autoScalingStrategy.provision(); + if (currentlyProvisioning.isEmpty()) { + int want = targetWorkerCount - (currValidWorkers + currentlyProvisioning.size()); + while (want > 0) { + final AutoScalingData provisioned = autoScalingStrategy.provision(); + if (provisioned == null) { + break; + } else { + currentlyProvisioning.addAll(provisioned.getNodeIds()); + lastProvisionTime = new DateTime(); + scalingStats.addProvisionEvent(provisioned); + want -= provisioned.getNodeIds().size(); + didProvision = true; + } + } + } else { + Duration durSinceLastProvision = new Duration(lastProvisionTime, new DateTime()); - if (provisioned != null) { - currentlyProvisioning.addAll(provisioned.getNodeIds()); - lastProvisionTime = new DateTime(); - scalingStats.addProvisionEvent(provisioned); + log.info("%s provisioning. Current wait time: %s", currentlyProvisioning, durSinceLastProvision); - return true; + if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { + log.makeAlert("Worker node provisioning taking too long!") + .addData("millisSinceLastProvision", durSinceLastProvision.getMillis()) + .addData("provisioningCount", currentlyProvisioning.size()) + .emit(); + + List nodeIps = autoScalingStrategy.idToIpLookup(Lists.newArrayList(currentlyProvisioning)); + autoScalingStrategy.terminate(nodeIps); + currentlyProvisioning.clear(); } } - } else { - Duration durSinceLastProvision = new Duration(lastProvisionTime, new DateTime()); - log.info( - "%s still provisioning. Wait for all provisioned nodes to complete before requesting new worker. Current wait time: %s", - currentlyProvisioning, - durSinceLastProvision - ); - - if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { - log.makeAlert("Worker node provisioning taking too long!") - .addData("millisSinceLastProvision", durSinceLastProvision.getMillis()) - .addData("provisioningCount", currentlyProvisioning.size()) - .emit(); - - List nodeIps = autoScalingStrategy.idToIpLookup(Lists.newArrayList(currentlyProvisioning)); - autoScalingStrategy.terminate(nodeIps); - currentlyProvisioning.clear(); - } + return didProvision; } - - return false; } @Override public boolean doTerminate(Collection pendingTasks, Collection zkWorkers) { - Set workerNodeIds = Sets.newHashSet( - autoScalingStrategy.ipToIdLookup( - Lists.newArrayList( - Iterables.transform( - zkWorkers, - new Function() - { - @Override - public String apply(ZkWorker input) + synchronized (lock) { + boolean didTerminate = false; + final Set workerNodeIds = Sets.newHashSet( + autoScalingStrategy.ipToIdLookup( + Lists.newArrayList( + Iterables.transform( + zkWorkers, + new Function() { - return input.getWorker().getIp(); + @Override + public String apply(ZkWorker input) + { + return input.getWorker().getIp(); + } } - } - ) - ) - ) - ); - - Set stillExisting = Sets.newHashSet(); - for (String s : currentlyTerminating) { - if (workerNodeIds.contains(s)) { - stillExisting.add(s); - } - } - currentlyTerminating.clear(); - currentlyTerminating.addAll(stillExisting); - boolean nothingTerminating = currentlyTerminating.isEmpty(); - - if (nothingTerminating) { - final int minNumWorkers = workerSetupdDataRef.get().getMinNumWorkers(); - if (zkWorkers.size() <= minNumWorkers) { - log.info("Only [%d <= %d] nodes in the cluster, not terminating anything.", zkWorkers.size(), minNumWorkers); - return false; - } - - List thoseLazyWorkers = Lists.newArrayList( - FunctionalIterable - .create(zkWorkers) - .filter( - new Predicate() - { - @Override - public boolean apply(ZkWorker input) - { - return input.getRunningTasks().isEmpty() - && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() - >= config.getWorkerIdleTimeout().toStandardDuration().getMillis(); - } - } + ) ) - ); - - int maxPossibleNodesTerminated = zkWorkers.size() - minNumWorkers; - int numNodesToTerminate = Math.min(maxPossibleNodesTerminated, thoseLazyWorkers.size()); - if (numNodesToTerminate <= 0) { - log.info("Found no nodes to terminate."); - return false; - } - - AutoScalingData terminated = autoScalingStrategy.terminate( - Lists.transform( - thoseLazyWorkers.subList(0, numNodesToTerminate), - new Function() - { - @Override - public String apply(ZkWorker input) - { - return input.getWorker().getIp(); - } - } ) ); - if (terminated != null) { - currentlyTerminating.addAll(terminated.getNodeIds()); - lastTerminateTime = new DateTime(); - scalingStats.addTerminateEvent(terminated); - - return true; + final Set stillExisting = Sets.newHashSet(); + for (String s : currentlyTerminating) { + if (workerNodeIds.contains(s)) { + stillExisting.add(s); + } } - } else { - Duration durSinceLastTerminate = new Duration(lastTerminateTime, new DateTime()); + currentlyTerminating.clear(); + currentlyTerminating.addAll(stillExisting); - log.info( - "%s still terminating. Wait for all nodes to terminate before trying again.", - currentlyTerminating - ); + updateTargetWorkerCount(pendingTasks, zkWorkers); - if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { - log.makeAlert("Worker node termination taking too long!") - .addData("millisSinceLastTerminate", durSinceLastTerminate.getMillis()) - .addData("terminatingCount", currentlyTerminating.size()) - .emit(); + if (currentlyTerminating.isEmpty()) { + final int want = zkWorkers.size() - targetWorkerCount; + if (want > 0) { + final List laziestWorkerIps = + FluentIterable.from(zkWorkers) + .filter(isLazyWorker) + .limit(want) + .transform( + new Function() + { + @Override + public String apply(ZkWorker zkWorker) + { + return zkWorker.getWorker().getIp(); + } + } + ) + .toList(); - currentlyTerminating.clear(); + log.info( + "Terminating %,d workers (wanted %,d): %s", + laziestWorkerIps.size(), + want, + Joiner.on(", ").join(laziestWorkerIps) + ); + + final AutoScalingData terminated = autoScalingStrategy.terminate(laziestWorkerIps); + if (terminated != null) { + currentlyTerminating.addAll(terminated.getNodeIds()); + lastTerminateTime = new DateTime(); + scalingStats.addTerminateEvent(terminated); + didTerminate = true; + } + } + } else { + Duration durSinceLastTerminate = new Duration(lastTerminateTime, new DateTime()); + + log.info("%s terminating. Current wait time: ", currentlyTerminating, durSinceLastTerminate); + + if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { + log.makeAlert("Worker node termination taking too long!") + .addData("millisSinceLastTerminate", durSinceLastTerminate.getMillis()) + .addData("terminatingCount", currentlyTerminating.size()) + .emit(); + + currentlyTerminating.clear(); + } } + + return didTerminate; } - - return false; } @Override @@ -261,16 +240,98 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat return scalingStats; } - private boolean hasTaskPendingBeyondThreshold(Collection pendingTasks) + private Predicate createValidWorkerPredicate(final WorkerSetupData workerSetupData) { - long now = System.currentTimeMillis(); - for (TaskRunnerWorkItem pendingTask : pendingTasks) { - final Duration durationSinceInsertion = new Duration(pendingTask.getQueueInsertionTime().getMillis(), now); - final Duration timeoutDuration = config.getPendingTaskTimeout().toStandardDuration(); - if (durationSinceInsertion.isEqual(timeoutDuration) || durationSinceInsertion.isLongerThan(timeoutDuration)) { - return true; + return new Predicate() + { + @Override + public boolean apply(ZkWorker zkWorker) + { + final String minVersion = workerSetupData.getMinVersion() != null + ? workerSetupData.getMinVersion() + : config.getWorkerVersion(); + if (minVersion == null) { + throw new ISE("No minVersion found! It should be set in your runtime properties or configuration database."); + } + return zkWorker.isValidVersion(minVersion); + } + }; + } + + private void updateTargetWorkerCount( + final Collection pendingTasks, + final Collection zkWorkers + ) + { + synchronized (lock) { + final WorkerSetupData workerSetupData = workerSetupDataRef.get(); + + if (targetWorkerCount < 0) { + // Initialize to size of current worker pool + targetWorkerCount = zkWorkers.size(); + log.info( + "Starting with %,d workers (min = %,d, max = %,d).", + targetWorkerCount, + workerSetupData.getMinNumWorkers(), + workerSetupData.getMaxNumWorkers() + ); + } + + final Collection validWorkers = Collections2.filter( + zkWorkers, + createValidWorkerPredicate(workerSetupData) + ); + final boolean atSteadyState = currentlyProvisioning.isEmpty() + && currentlyTerminating.isEmpty() + && validWorkers.size() == targetWorkerCount; + final boolean shouldScaleUp = atSteadyState + && hasTaskPendingBeyondThreshold(pendingTasks) + && targetWorkerCount < workerSetupData.getMaxNumWorkers(); + final boolean shouldScaleDown = atSteadyState + && Iterables.any(validWorkers, isLazyWorker) + && targetWorkerCount > workerSetupData.getMinNumWorkers(); + if (shouldScaleUp) { + targetWorkerCount++; + log.info( + "I think we should scale up to %,d workers (current = %,d, min = %,d, max = %,d).", + targetWorkerCount, + validWorkers.size(), + workerSetupData.getMinNumWorkers(), + workerSetupData.getMaxNumWorkers() + ); + } else if (shouldScaleDown) { + targetWorkerCount--; + log.info( + "I think we should scale down to %,d workers (current = %,d, min = %,d, max = %,d).", + targetWorkerCount, + validWorkers.size(), + workerSetupData.getMinNumWorkers(), + workerSetupData.getMaxNumWorkers() + ); + } else { + log.info( + "Our target is %,d workers, and I'm okay with that (current = %,d, min = %,d, max = %,d).", + targetWorkerCount, + validWorkers.size(), + workerSetupData.getMinNumWorkers(), + workerSetupData.getMaxNumWorkers() + ); } } - return false; + } + + private boolean hasTaskPendingBeyondThreshold(Collection pendingTasks) + { + synchronized (lock) { + long now = System.currentTimeMillis(); + for (TaskRunnerWorkItem pendingTask : pendingTasks) { + final Duration durationSinceInsertion = new Duration(pendingTask.getQueueInsertionTime().getMillis(), now); + final Duration timeoutDuration = config.getPendingTaskTimeout().toStandardDuration(); + if (durationSinceInsertion.isEqual(timeoutDuration) || durationSinceInsertion.isLongerThan(timeoutDuration)) { + return true; + } + } + return false; + } } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java index d0b5edb5ca1..e6cd52c80ac 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java @@ -126,13 +126,11 @@ public class EC2AutoScalingStrategyTest AutoScalingData created = strategy.provision(); Assert.assertEquals(created.getNodeIds().size(), 1); - Assert.assertEquals(created.getNodes().size(), 1); Assert.assertEquals("theInstance", created.getNodeIds().get(0)); AutoScalingData deleted = strategy.terminate(Arrays.asList("dummyIP")); Assert.assertEquals(deleted.getNodeIds().size(), 1); - Assert.assertEquals(deleted.getNodes().size(), 1); Assert.assertEquals(String.format("%s:8080", IP), deleted.getNodeIds().get(0)); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java index 1f3f4a44eee..10c7bf77882 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java @@ -105,7 +105,7 @@ public class SimpleResourceManagementStrategyTest EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList()); EasyMock.expect(autoScalingStrategy.provision()).andReturn( - new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()) + new AutoScalingData(Lists.newArrayList("aNode")) ); EasyMock.replay(autoScalingStrategy); @@ -133,7 +133,7 @@ public class SimpleResourceManagementStrategyTest EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList()).times(2); EasyMock.expect(autoScalingStrategy.provision()).andReturn( - new AutoScalingData(Lists.newArrayList("fake"), Lists.newArrayList("faker")) + new AutoScalingData(Lists.newArrayList("fake")) ); EasyMock.replay(autoScalingStrategy); @@ -190,7 +190,7 @@ public class SimpleResourceManagementStrategyTest EasyMock.expect(autoScalingStrategy.terminate(EasyMock.>anyObject())) .andReturn(null); EasyMock.expect(autoScalingStrategy.provision()).andReturn( - new AutoScalingData(Lists.newArrayList("fake"), Lists.newArrayList("faker")) + new AutoScalingData(Lists.newArrayList("fake")) ); EasyMock.replay(autoScalingStrategy); @@ -242,7 +242,7 @@ public class SimpleResourceManagementStrategyTest EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList()); EasyMock.expect(autoScalingStrategy.terminate(EasyMock.>anyObject())).andReturn( - new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()) + new AutoScalingData(Lists.newArrayList()) ); EasyMock.replay(autoScalingStrategy); @@ -272,7 +272,7 @@ public class SimpleResourceManagementStrategyTest EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList("ip")).times(2); EasyMock.expect(autoScalingStrategy.terminate(EasyMock.>anyObject())).andReturn( - new AutoScalingData(Lists.newArrayList("ip"), Lists.newArrayList("ip")) + new AutoScalingData(Lists.newArrayList("ip")) ); EasyMock.replay(autoScalingStrategy); From f86342f7dc9cefdc74936707af9258e5e33342a5 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 19 Dec 2013 16:16:20 -0800 Subject: [PATCH 07/31] DbTaskStorage: Protect against invalid lock_payload --- .../io/druid/indexing/overlord/DbTaskStorage.java | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/DbTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/DbTaskStorage.java index 5dc6e1c6fff..cf0fb4f3e24 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/DbTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/DbTaskStorage.java @@ -489,7 +489,18 @@ public class DbTaskStorage implements TaskStorage final Map retMap = Maps.newHashMap(); for (final Map row : dbTaskLocks) { - retMap.put((Long) row.get("id"), jsonMapper.readValue((byte[]) row.get("lock_payload"), TaskLock.class)); + try { + retMap.put( + (Long) row.get("id"), + jsonMapper.readValue((byte[]) row.get("lock_payload"), TaskLock.class) + ); + } + catch (Exception e) { + log.makeAlert(e, "Failed to deserialize TaskLock") + .addData("task", taskid) + .addData("lockPayload", row) + .emit(); + } } return retMap; } From 0ff7f0e8e025ea92d954214ff7077b1f259703c4 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 19 Dec 2013 16:16:34 -0800 Subject: [PATCH 08/31] TaskActionToolbox: Combine adjacent ifs --- .../indexing/common/actions/TaskActionToolbox.java | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java index 10c4d627462..d9b0520f40b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java @@ -76,13 +76,9 @@ public class TaskActionToolbox Preconditions.checkArgument(!segments.isEmpty(), "segments nonempty"); final DataSegment firstSegment = segments.iterator().next(); for (final DataSegment segment : segments) { - if (!segment.getDataSource().equals(firstSegment.getDataSource())) { - return false; - } - if (!segment.getInterval().equals(firstSegment.getInterval())) { - return false; - } - if (!segment.getVersion().equals(firstSegment.getVersion())) { + if (!segment.getDataSource().equals(firstSegment.getDataSource()) + || !segment.getInterval().equals(firstSegment.getInterval()) + || !segment.getVersion().equals(firstSegment.getVersion())) { return false; } } From 3dd9a255469c6512526f8788fce8f2dc2e8523b7 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 19 Dec 2013 16:18:16 -0800 Subject: [PATCH 09/31] Fix import --- .../overlord/scaling/SimpleResourceManagementStrategy.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java index 6b6fc613a7b..68733fed9cf 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java @@ -19,8 +19,8 @@ package io.druid.indexing.overlord.scaling; -import com.google.api.client.repackaged.com.google.common.base.Joiner; import com.google.common.base.Function; +import com.google.common.base.Joiner; import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.collect.Collections2; From 0ee6136ea3ca5481e951bafa7d594b365862d026 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 08:56:17 -0800 Subject: [PATCH 10/31] NoopTask: Fix things that should be static. Add simple factory method. --- .../java/io/druid/indexing/common/task/NoopTask.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java index d45f66377b7..b4de3512fbe 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java @@ -35,9 +35,9 @@ import org.joda.time.DateTime; public class NoopTask extends AbstractTask { private static final Logger log = new Logger(NoopTask.class); - private static int defaultRunTime = 2500; - private static int defaultIsReadyTime = 0; - private static IsReadyResult defaultIsReadyResult = IsReadyResult.YES; + private static final int defaultRunTime = 2500; + private static final int defaultIsReadyTime = 0; + private static final IsReadyResult defaultIsReadyResult = IsReadyResult.YES; enum IsReadyResult { @@ -139,4 +139,9 @@ public class NoopTask extends AbstractTask log.info("Woke up!"); return TaskStatus.success(getId()); } + + public static NoopTask create() + { + return new NoopTask(null, 0, 0, null, null); + } } From 4a722c0a6d14cb172835c6f862d8c9b2c9053046 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 08:59:35 -0800 Subject: [PATCH 11/31] Autoscaling changes from code review. - Log and return immediately when workerSetupData is null - Allow provisioning more nodes while other nodes are still provisioning - Add tests for bumping up the minimum version --- .../SimpleResourceManagementStrategy.java | 51 +++--- .../SimpleResourceManagementStrategyTest.java | 167 +++++++++++++++++- 2 files changed, 195 insertions(+), 23 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java index 68733fed9cf..4f541f113c7 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java @@ -90,6 +90,10 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat synchronized (lock) { boolean didProvision = false; final WorkerSetupData workerSetupData = workerSetupDataRef.get(); + if (workerSetupData == null) { + log.warn("No workerSetupData available, cannot provision new workers."); + return false; + } final Predicate isValidWorker = createValidWorkerPredicate(workerSetupData); final int currValidWorkers = Collections2.filter(zkWorkers, isValidWorker).size(); @@ -112,21 +116,22 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat updateTargetWorkerCount(pendingTasks, zkWorkers); - if (currentlyProvisioning.isEmpty()) { - int want = targetWorkerCount - (currValidWorkers + currentlyProvisioning.size()); - while (want > 0) { - final AutoScalingData provisioned = autoScalingStrategy.provision(); - if (provisioned == null) { - break; - } else { - currentlyProvisioning.addAll(provisioned.getNodeIds()); - lastProvisionTime = new DateTime(); - scalingStats.addProvisionEvent(provisioned); - want -= provisioned.getNodeIds().size(); - didProvision = true; - } + int want = targetWorkerCount - (currValidWorkers + currentlyProvisioning.size()); + while (want > 0) { + final AutoScalingData provisioned = autoScalingStrategy.provision(); + final List newNodes; + if (provisioned == null || (newNodes = provisioned.getNodeIds()).isEmpty()) { + break; + } else { + currentlyProvisioning.addAll(newNodes); + lastProvisionTime = new DateTime(); + scalingStats.addProvisionEvent(provisioned); + want -= provisioned.getNodeIds().size(); + didProvision = true; } - } else { + } + + if (!currentlyProvisioning.isEmpty()) { Duration durSinceLastProvision = new Duration(lastProvisionTime, new DateTime()); log.info("%s provisioning. Current wait time: %s", currentlyProvisioning, durSinceLastProvision); @@ -151,6 +156,11 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat public boolean doTerminate(Collection pendingTasks, Collection zkWorkers) { synchronized (lock) { + if (workerSetupDataRef.get() == null) { + log.warn("No workerSetupData available, cannot terminate workers."); + return false; + } + boolean didTerminate = false; final Set workerNodeIds = Sets.newHashSet( autoScalingStrategy.ipToIdLookup( @@ -218,7 +228,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat } else { Duration durSinceLastTerminate = new Duration(lastTerminateTime, new DateTime()); - log.info("%s terminating. Current wait time: ", currentlyTerminating, durSinceLastTerminate); + log.info("%s terminating. Current wait time: %s", currentlyTerminating, durSinceLastTerminate); if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { log.makeAlert("Worker node termination taking too long!") @@ -265,22 +275,23 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat { synchronized (lock) { final WorkerSetupData workerSetupData = workerSetupDataRef.get(); + final Collection validWorkers = Collections2.filter( + zkWorkers, + createValidWorkerPredicate(workerSetupData) + ); if (targetWorkerCount < 0) { // Initialize to size of current worker pool targetWorkerCount = zkWorkers.size(); log.info( - "Starting with %,d workers (min = %,d, max = %,d).", + "Starting with a target of %,d workers (current = %,d, min = %,d, max = %,d).", targetWorkerCount, + validWorkers.size(), workerSetupData.getMinNumWorkers(), workerSetupData.getMaxNumWorkers() ); } - final Collection validWorkers = Collections2.filter( - zkWorkers, - createValidWorkerPredicate(workerSetupData) - ); final boolean atSteadyState = currentlyProvisioning.isEmpty() && currentlyTerminating.isEmpty() && validWorkers.size() == targetWorkerCount; diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java index 10c7bf77882..6ffc6ae6222 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java @@ -19,6 +19,7 @@ package io.druid.indexing.overlord.scaling; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -28,6 +29,7 @@ import com.metamx.emitter.service.ServiceEventBuilder; import io.druid.common.guava.DSuppliers; import io.druid.indexing.common.TestMergeTask; import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.RemoteTaskRunnerWorkItem; import io.druid.indexing.overlord.ZkWorker; @@ -63,7 +65,7 @@ public class SimpleResourceManagementStrategyTest public void setUp() throws Exception { autoScalingStrategy = EasyMock.createMock(AutoScalingStrategy.class); - workerSetupData = new AtomicReference( + workerSetupData = new AtomicReference<>( new WorkerSetupData( "0", 0, 2, null, null, null ) @@ -309,15 +311,174 @@ public class SimpleResourceManagementStrategyTest EasyMock.verify(autoScalingStrategy); } + @Test + public void testNoActionNeeded() throws Exception + { + EasyMock.reset(autoScalingStrategy); + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList("ip")); + EasyMock.replay(autoScalingStrategy); + + boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + ), + Arrays.asList( + new TestZkWorker(NoopTask.create()), + new TestZkWorker(NoopTask.create()) + ) + ); + + Assert.assertFalse(terminatedSomething); + EasyMock.verify(autoScalingStrategy); + + EasyMock.reset(autoScalingStrategy); + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList("ip")); + EasyMock.replay(autoScalingStrategy); + + boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + ), + Arrays.asList( + new TestZkWorker(NoopTask.create()), + new TestZkWorker(NoopTask.create()) + ) + ); + + Assert.assertFalse(provisionedSomething); + EasyMock.verify(autoScalingStrategy); + } + + @Test + public void testMinVersionIncrease() throws Exception + { + // Don't terminate anything + EasyMock.reset(autoScalingStrategy); + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList("ip")); + EasyMock.replay(autoScalingStrategy); + boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( + Arrays.asList(), + Arrays.asList( + new TestZkWorker(NoopTask.create(), "h1", "i1", "0"), + new TestZkWorker(NoopTask.create(), "h1", "i2", "0") + ) + ); + Assert.assertFalse(terminatedSomething); + EasyMock.verify(autoScalingStrategy); + + // Don't provision anything + EasyMock.reset(autoScalingStrategy); + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList("ip")); + EasyMock.replay(autoScalingStrategy); + boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( + Arrays.asList(), + Arrays.asList( + new TestZkWorker(NoopTask.create()), + new TestZkWorker(NoopTask.create()) + ) + ); + Assert.assertFalse(provisionedSomething); + EasyMock.verify(autoScalingStrategy); + + // Increase minVersion + workerSetupData.set(new WorkerSetupData("1", 0, 2, null, null, null)); + + // Provision two new workers + EasyMock.reset(autoScalingStrategy); + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList("ip")); + EasyMock.expect(autoScalingStrategy.provision()).andReturn( + new AutoScalingData(Lists.newArrayList("h3")) + ); + EasyMock.expect(autoScalingStrategy.provision()).andReturn( + new AutoScalingData(Lists.newArrayList("h4")) + ); + EasyMock.replay(autoScalingStrategy); + provisionedSomething = simpleResourceManagementStrategy.doProvision( + Arrays.asList(), + Arrays.asList( + new TestZkWorker(NoopTask.create(), "h1", "i1", "0"), + new TestZkWorker(NoopTask.create(), "h2", "i2", "0") + ) + ); + Assert.assertTrue(provisionedSomething); + EasyMock.verify(autoScalingStrategy); + + // Terminate old workers + EasyMock.reset(autoScalingStrategy); + EasyMock.expect(autoScalingStrategy.ipToIdLookup(ImmutableList.of("i1", "i2", "i3", "i4"))).andReturn( + ImmutableList.of("h1", "h2", "h3", "h4") + ); + EasyMock.expect(autoScalingStrategy.terminate(ImmutableList.of("i1", "i2"))).andReturn( + new AutoScalingData(ImmutableList.of("h1", "h2")) + ); + EasyMock.replay(autoScalingStrategy); + terminatedSomething = simpleResourceManagementStrategy.doTerminate( + Arrays.asList(), + Arrays.asList( + new TestZkWorker(null, "h1", "i1", "0"), + new TestZkWorker(null, "h2", "i2", "0"), + new TestZkWorker(NoopTask.create(), "h3", "i3", "1"), + new TestZkWorker(NoopTask.create(), "h4", "i4", "1") + ) + ); + Assert.assertTrue(terminatedSomething); + EasyMock.verify(autoScalingStrategy); + } + + @Test + public void testNullWorkerSetupData() throws Exception + { + workerSetupData.set(null); + EasyMock.replay(autoScalingStrategy); + + boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + ), + Arrays.asList( + new TestZkWorker(null) + ) + ); + + boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + ), + Arrays.asList( + new TestZkWorker(null) + ) + ); + + Assert.assertFalse(terminatedSomething); + Assert.assertFalse(provisionedSomething); + + EasyMock.verify(autoScalingStrategy); + } + private static class TestZkWorker extends ZkWorker { private final Task testTask; - private TestZkWorker( + public TestZkWorker( Task testTask ) { - super(new Worker("host", "ip", 3, "version"), null, new DefaultObjectMapper()); + this(testTask, "host", "ip", "0"); + } + + public TestZkWorker( + Task testTask, + String host, + String ip, + String version + ) + { + super(new Worker(host, ip, 3, version), null, new DefaultObjectMapper()); this.testTask = testTask; } From 6224577ed12141c1542ac5ccfc202be4862fd48c Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 10:01:32 -0800 Subject: [PATCH 12/31] Autoscaling: Terminate obsolete workers faster --- .../SimpleResourceManagementStrategy.java | 43 +++++++++++++------ 1 file changed, 29 insertions(+), 14 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java index 4f541f113c7..80f7aef94c8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java @@ -56,16 +56,6 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat private final Object lock = new Object(); private final Set currentlyProvisioning = Sets.newHashSet(); private final Set currentlyTerminating = Sets.newHashSet(); - private final Predicate isLazyWorker = new Predicate() - { - @Override - public boolean apply(ZkWorker input) - { - return input.getRunningTasks().isEmpty() - && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() - >= config.getWorkerIdleTimeout().toStandardDuration().getMillis(); - } - }; private int targetWorkerCount = -1; private DateTime lastProvisionTime = new DateTime(); @@ -94,7 +84,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat log.warn("No workerSetupData available, cannot provision new workers."); return false; } - final Predicate isValidWorker = createValidWorkerPredicate(workerSetupData); + final Predicate isValidWorker = createValidWorkerPredicate(config, workerSetupData); final int currValidWorkers = Collections2.filter(zkWorkers, isValidWorker).size(); final List workerNodeIds = autoScalingStrategy.ipToIdLookup( @@ -156,7 +146,8 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat public boolean doTerminate(Collection pendingTasks, Collection zkWorkers) { synchronized (lock) { - if (workerSetupDataRef.get() == null) { + final WorkerSetupData workerSetupData = workerSetupDataRef.get(); + if (workerSetupData == null) { log.warn("No workerSetupData available, cannot terminate workers."); return false; } @@ -191,6 +182,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat updateTargetWorkerCount(pendingTasks, zkWorkers); + final Predicate isLazyWorker = createLazyWorkerPredicate(config, workerSetupData); if (currentlyTerminating.isEmpty()) { final int want = zkWorkers.size() - targetWorkerCount; if (want > 0) { @@ -250,7 +242,29 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat return scalingStats; } - private Predicate createValidWorkerPredicate(final WorkerSetupData workerSetupData) + private static Predicate createLazyWorkerPredicate( + final SimpleResourceManagementConfig config, + final WorkerSetupData workerSetupData + ) + { + final Predicate isValidWorker = createValidWorkerPredicate(config, workerSetupData); + + return new Predicate() + { + @Override + public boolean apply(ZkWorker worker) + { + final boolean itHasBeenAWhile = System.currentTimeMillis() - worker.getLastCompletedTaskTime().getMillis() + >= config.getWorkerIdleTimeout().toStandardDuration().getMillis(); + return worker.getRunningTasks().isEmpty() && (itHasBeenAWhile || !isValidWorker.apply(worker)); + } + }; + } + + private static Predicate createValidWorkerPredicate( + final SimpleResourceManagementConfig config, + final WorkerSetupData workerSetupData + ) { return new Predicate() { @@ -277,8 +291,9 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat final WorkerSetupData workerSetupData = workerSetupDataRef.get(); final Collection validWorkers = Collections2.filter( zkWorkers, - createValidWorkerPredicate(workerSetupData) + createValidWorkerPredicate(config, workerSetupData) ); + final Predicate isLazyWorker = createLazyWorkerPredicate(config, workerSetupData); if (targetWorkerCount < 0) { // Initialize to size of current worker pool From 97095ee3dbad22fb5ab4fbc39624c8f0e88cda9a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 10:13:30 -0800 Subject: [PATCH 13/31] [maven-release-plugin] prepare release druid-0.6.43 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 5bade4d64d1..cff2dc10fc0 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/common/pom.xml b/common/pom.xml index 7cdef58b97a..660cd2caa3e 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/examples/pom.xml b/examples/pom.xml index eb880023fb2..f22364d88a1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 9085d8af3d2..63497f9bcc8 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 7e68c822fb2..2644e8d3bf5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 8b0a1708f53..a3520136f96 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 9295c043a24..1bf14b92e72 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 9356a120ce6..156a4c3240d 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/pom.xml b/pom.xml index f0b96500d4a..8156c5c3313 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.43-SNAPSHOT + 0.6.43 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - ${project.artifactId}-${project.version} + druid-0.6.43 diff --git a/processing/pom.xml b/processing/pom.xml index d821fdd2572..74079cbd0b1 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index eebd245923f..cfc0f8cd68a 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 817fc5307e6..d9fd9d547c1 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/server/pom.xml b/server/pom.xml index c88b542ce21..1cfa0beb151 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 diff --git a/services/pom.xml b/services/pom.xml index 1d9cf5a4257..f1dcc4dffda 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.43-SNAPSHOT + 0.6.43 From c503f5e9c591c810c4a3137ee84d2b7378e5f34a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 10:13:35 -0800 Subject: [PATCH 14/31] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index cff2dc10fc0..c8d01d914c7 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 660cd2caa3e..cc804ade0d1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index f22364d88a1..2c260de9b26 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 63497f9bcc8..7a86fafd1dc 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 2644e8d3bf5..35acca14246 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index a3520136f96..352fc1943ae 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 1bf14b92e72..441f92187f1 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 156a4c3240d..713779e4c6b 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/pom.xml b/pom.xml index 8156c5c3313..f7eacb0e7d9 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.43 + 0.6.44-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.43 + ${project.artifactId}-${project.version} diff --git a/processing/pom.xml b/processing/pom.xml index 74079cbd0b1..a63c90f9670 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index cfc0f8cd68a..a1d565571a9 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index d9fd9d547c1..657661c4acd 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 1cfa0beb151..cdffd9871fc 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f1dcc4dffda..df239e1684d 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.43 + 0.6.44-SNAPSHOT From e5b8546d197785ed6fec975bd40f61b8c8b96736 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 11:04:54 -0800 Subject: [PATCH 15/31] Autoscaling fixes. - Initial targetWorkerCount must be subject to pool size limits - Use consistent workerSetupData for the entire autoscaling run - Don't call terminate() when we have nothing to terminate - Terminate obsolete workers even faster --- .../SimpleResourceManagementStrategy.java | 50 +++++++++++-------- 1 file changed, 30 insertions(+), 20 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java index 80f7aef94c8..fe08fd648e1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java @@ -104,7 +104,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat ); currentlyProvisioning.removeAll(workerNodeIds); - updateTargetWorkerCount(pendingTasks, zkWorkers); + updateTargetWorkerCount(workerSetupData, pendingTasks, zkWorkers); int want = targetWorkerCount - (currValidWorkers + currentlyProvisioning.size()); while (want > 0) { @@ -180,16 +180,16 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat currentlyTerminating.clear(); currentlyTerminating.addAll(stillExisting); - updateTargetWorkerCount(pendingTasks, zkWorkers); + updateTargetWorkerCount(workerSetupData, pendingTasks, zkWorkers); final Predicate isLazyWorker = createLazyWorkerPredicate(config, workerSetupData); if (currentlyTerminating.isEmpty()) { - final int want = zkWorkers.size() - targetWorkerCount; - if (want > 0) { + final int excessWorkers = (zkWorkers.size() + currentlyProvisioning.size()) - targetWorkerCount; + if (excessWorkers > 0) { final List laziestWorkerIps = FluentIterable.from(zkWorkers) .filter(isLazyWorker) - .limit(want) + .limit(excessWorkers) .transform( new Function() { @@ -202,19 +202,23 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat ) .toList(); - log.info( - "Terminating %,d workers (wanted %,d): %s", - laziestWorkerIps.size(), - want, - Joiner.on(", ").join(laziestWorkerIps) - ); + if (laziestWorkerIps.isEmpty()) { + log.info("Wanted to terminate %,d workers, but couldn't find any lazy ones!"); + } else { + log.info( + "Terminating %,d workers (wanted %,d): %s", + laziestWorkerIps.size(), + excessWorkers, + Joiner.on(", ").join(laziestWorkerIps) + ); - final AutoScalingData terminated = autoScalingStrategy.terminate(laziestWorkerIps); - if (terminated != null) { - currentlyTerminating.addAll(terminated.getNodeIds()); - lastTerminateTime = new DateTime(); - scalingStats.addTerminateEvent(terminated); - didTerminate = true; + final AutoScalingData terminated = autoScalingStrategy.terminate(laziestWorkerIps); + if (terminated != null) { + currentlyTerminating.addAll(terminated.getNodeIds()); + lastTerminateTime = new DateTime(); + scalingStats.addTerminateEvent(terminated); + didTerminate = true; + } } } } else { @@ -283,12 +287,12 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat } private void updateTargetWorkerCount( + final WorkerSetupData workerSetupData, final Collection pendingTasks, final Collection zkWorkers ) { synchronized (lock) { - final WorkerSetupData workerSetupData = workerSetupDataRef.get(); final Collection validWorkers = Collections2.filter( zkWorkers, createValidWorkerPredicate(config, workerSetupData) @@ -296,8 +300,14 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat final Predicate isLazyWorker = createLazyWorkerPredicate(config, workerSetupData); if (targetWorkerCount < 0) { - // Initialize to size of current worker pool - targetWorkerCount = zkWorkers.size(); + // Initialize to size of current worker pool, subject to pool size limits + targetWorkerCount = Math.max( + Math.min( + zkWorkers.size(), + workerSetupData.getMaxNumWorkers() + ), + workerSetupData.getMinNumWorkers() + ); log.info( "Starting with a target of %,d workers (current = %,d, min = %,d, max = %,d).", targetWorkerCount, From 3b15f6a8340e7d72659997241284a1c9d4fc27c8 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 11:07:57 -0800 Subject: [PATCH 16/31] [maven-release-plugin] prepare release druid-0.6.44 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index c8d01d914c7..02e5ee045d7 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/common/pom.xml b/common/pom.xml index cc804ade0d1..868b62fadc1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/examples/pom.xml b/examples/pom.xml index 2c260de9b26..972135858b9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 7a86fafd1dc..eca98739a06 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 35acca14246..a8ebe215cc5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 352fc1943ae..d06cb03442d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 441f92187f1..2ad3de37811 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 713779e4c6b..edbb37bef12 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/pom.xml b/pom.xml index f7eacb0e7d9..fac88736c8a 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.44-SNAPSHOT + 0.6.44 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - ${project.artifactId}-${project.version} + druid-0.6.44 diff --git a/processing/pom.xml b/processing/pom.xml index a63c90f9670..fba8175bfa2 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index a1d565571a9..fe2a22432c4 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 657661c4acd..5280c1e7281 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/server/pom.xml b/server/pom.xml index cdffd9871fc..03fc1a1307b 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 diff --git a/services/pom.xml b/services/pom.xml index df239e1684d..d099fe720f5 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.44-SNAPSHOT + 0.6.44 From 5711ac4aff5d2fc721a4ef247b3178d9bd3980b5 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 11:08:02 -0800 Subject: [PATCH 17/31] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 02e5ee045d7..8b6073e811d 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 868b62fadc1..6583512c97e 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 972135858b9..f1c2b1c24ca 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index eca98739a06..37f41e73261 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index a8ebe215cc5..89fdfa4df4e 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index d06cb03442d..fbdcf1822cd 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 2ad3de37811..8c8141e0cd2 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index edbb37bef12..657c7df3640 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/pom.xml b/pom.xml index fac88736c8a..ab711e0783f 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.44 + 0.6.45-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.44 + ${project.artifactId}-${project.version} diff --git a/processing/pom.xml b/processing/pom.xml index fba8175bfa2..d7facc2e3f9 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index fe2a22432c4..1655311bf78 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 5280c1e7281..6678ca0b6f7 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 03fc1a1307b..3654decc6a2 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index d099fe720f5..151f533a92e 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.44 + 0.6.45-SNAPSHOT From 4bc8ff4d6a679b5a2b52685de6228f876d0ad431 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 11:19:36 -0800 Subject: [PATCH 18/31] Update versions --- build.sh | 2 +- docs/content/Booting-a-production-cluster.md | 2 +- docs/content/Examples.md | 4 ++-- docs/content/Realtime.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-2.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.textile | 2 +- examples/config/historical/runtime.properties | 2 +- examples/config/realtime/runtime.properties | 2 +- services/src/main/java/io/druid/cli/CliBroker.java | 2 +- services/src/main/java/io/druid/cli/CliCoordinator.java | 2 +- services/src/main/java/io/druid/cli/CliHadoopIndexer.java | 2 +- services/src/main/java/io/druid/cli/CliHistorical.java | 2 +- services/src/main/java/io/druid/cli/CliOverlord.java | 2 +- services/src/main/java/io/druid/cli/CliRealtime.java | 2 +- services/src/main/java/io/druid/cli/CliRealtimeExample.java | 2 +- 18 files changed, 23 insertions(+), 23 deletions(-) diff --git a/build.sh b/build.sh index 9b7478e592e..95ec936e1ef 100755 --- a/build.sh +++ b/build.sh @@ -30,4 +30,4 @@ echo "For examples, see: " echo " " ls -1 examples/*/*sh echo " " -echo "See also http://druid.io/docs/0.6.40" +echo "See also http://druid.io/docs/0.6.45" diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index bb00506700f..3d048ff6b1b 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -3,7 +3,7 @@ layout: doc_page --- # Booting a Single Node Cluster # -[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.40-bin.tar.gz). +[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.45-bin.tar.gz). The [ec2 run script](https://github.com/metamx/druid/blob/master/examples/bin/run_ec2.sh), run_ec2.sh, is located at 'examples/bin' if you have checked out the code, or at the root of the project if you've downloaded a tarball. The scripts rely on the [Amazon EC2 API Tools](http://aws.amazon.com/developertools/351), and you will need to set three environment variables: diff --git a/docs/content/Examples.md b/docs/content/Examples.md index 7aff89ea644..e94c11bcf0d 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.40 +git checkout druid-0.6.45 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.40-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.45-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Realtime.md b/docs/content/Realtime.md index 980e1dfb773..978bf4121d4 100644 --- a/docs/content/Realtime.md +++ b/docs/content/Realtime.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.40"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.45"] druid.zk.service.host=localhost diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 1b35c0979e2..0d4bb480947 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.40-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.45-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.40 +cd druid-services-0.6.45 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index 50fef2985f6..5c6c665de11 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -44,7 +44,7 @@ With real-world data, we recommend having a message bus such as [Apache Kafka](h #### Setting up Kafka -[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.40/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. +[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.45/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html). diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 348dfd1df53..62cf2fbcbdd 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes as well as and exter If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.40-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.45-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.40"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.45"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -238,7 +238,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.40","io.druid.extensions:druid-kafka-seven:0.6.40"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.45","io.druid.extensions:druid-kafka-seven:0.6.45"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index a9605f2fea4..e3607d75503 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.40-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.45-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.40 +cd druid-services-0.6.45 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.textile b/docs/content/Twitter-Tutorial.textile index c82ea981ec4..e66cd62ce34 100644 --- a/docs/content/Twitter-Tutorial.textile +++ b/docs/content/Twitter-Tutorial.textile @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.40-bin.tar.gz. +We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.45-bin.tar.gz. Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index fa2593c63a1..2c7950d4112 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -4,7 +4,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.40"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.45"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index eae65e057f9..0946d6753e1 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -4,7 +4,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.40","io.druid.extensions:druid-kafka-seven:0.6.40","io.druid.extensions:druid-rabbitmq:0.6.40"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.45","io.druid.extensions:druid-kafka-seven:0.6.45","io.druid.extensions:druid-rabbitmq:0.6.45"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 3a7c88d4426..2feda4e5e03 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -53,7 +53,7 @@ import java.util.List; */ @Command( name = "broker", - description = "Runs a broker node, see http://druid.io/docs/0.6.40/Broker.html for a description" + description = "Runs a broker node, see http://druid.io/docs/0.6.45/Broker.html for a description" ) public class CliBroker extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 486511b1104..3e6da7fbb42 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -60,7 +60,7 @@ import java.util.List; */ @Command( name = "coordinator", - description = "Runs the Coordinator, see http://druid.io/docs/0.6.40/Coordinator.html for a description." + description = "Runs the Coordinator, see http://druid.io/docs/0.6.45/Coordinator.html for a description." ) public class CliCoordinator extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java index 2cdc9706cef..b654e8de86b 100644 --- a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java @@ -41,7 +41,7 @@ import java.util.List; */ @Command( name = "hadoop", - description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.40/Batch-ingestion.html for a description." + description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.45/Batch-ingestion.html for a description." ) public class CliHadoopIndexer implements Runnable { diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index e9c5ece9889..d53ddbf0a89 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -42,7 +42,7 @@ import java.util.List; */ @Command( name = "historical", - description = "Runs a Historical node, see http://druid.io/docs/0.6.40/Historical.html for a description" + description = "Runs a Historical node, see http://druid.io/docs/0.6.45/Historical.html for a description" ) public class CliHistorical extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index aa68c14c6bb..47777163726 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -93,7 +93,7 @@ import java.util.List; */ @Command( name = "overlord", - description = "Runs an Overlord node, see http://druid.io/docs/0.6.40/Indexing-Service.html for a description" + description = "Runs an Overlord node, see http://druid.io/docs/0.6.45/Indexing-Service.html for a description" ) public class CliOverlord extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index 70baabbc3c2..ade1069c899 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -30,7 +30,7 @@ import java.util.List; */ @Command( name = "realtime", - description = "Runs a realtime node, see http://druid.io/docs/0.6.40/Realtime.html for a description" + description = "Runs a realtime node, see http://druid.io/docs/0.6.45/Realtime.html for a description" ) public class CliRealtime extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index f4b5c4ab5d8..e5cc2249bca 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -42,7 +42,7 @@ import java.util.concurrent.Executor; */ @Command( name = "realtime", - description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.40/Realtime.html for a description" + description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.45/Realtime.html for a description" ) public class CliRealtimeExample extends ServerRunnable { From 17ad4ee2f0f4561ae0d1c5f4dc81389a23e988e9 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 11:23:28 -0800 Subject: [PATCH 19/31] Fix RemoteTaskRunnerTest --- .../src/test/java/io/druid/indexing/common/TestMergeTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java b/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java index 906e6e6c1e6..05f3118d3c9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java @@ -44,7 +44,7 @@ public class TestMergeTask extends MergeTask Lists.newArrayList( new DataSegment( "dummyDs", - new Interval(new DateTime(), new DateTime()), + new Interval(new DateTime(), new DateTime().plus(1)), new DateTime().toString(), null, null, From 29c0fa1df985cc508894586fad1fc0cab9c164ef Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 11:25:31 -0800 Subject: [PATCH 20/31] [maven-release-plugin] prepare release druid-0.6.45 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 8b6073e811d..602509d9f99 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/common/pom.xml b/common/pom.xml index 6583512c97e..71e0ce7bc33 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/examples/pom.xml b/examples/pom.xml index f1c2b1c24ca..bd7aa5fd319 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 37f41e73261..ff252e38307 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 89fdfa4df4e..b11508baef9 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index fbdcf1822cd..9211f6307ca 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 8c8141e0cd2..757102ba2a7 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 657c7df3640..11730d348a9 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/pom.xml b/pom.xml index ab711e0783f..0671b8ed865 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.45-SNAPSHOT + 0.6.45 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - ${project.artifactId}-${project.version} + druid-0.6.45 diff --git a/processing/pom.xml b/processing/pom.xml index d7facc2e3f9..b2171bb5ccf 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 1655311bf78..36b0708dae5 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 6678ca0b6f7..e4e9fba065e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/server/pom.xml b/server/pom.xml index 3654decc6a2..6065d97a15c 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 diff --git a/services/pom.xml b/services/pom.xml index 151f533a92e..d464f50beb5 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.45-SNAPSHOT + 0.6.45 From 9043c211b2ef0ca0c7efeca56075ab9b09aa3352 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 11:25:36 -0800 Subject: [PATCH 21/31] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 602509d9f99..9879e2b100f 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 71e0ce7bc33..9ca631cae9d 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index bd7aa5fd319..fa55220a573 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index ff252e38307..05bab3edf76 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index b11508baef9..4adea254904 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9211f6307ca..1d828b6b005 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 757102ba2a7..692121cdbec 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 11730d348a9..084eea93d83 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/pom.xml b/pom.xml index 0671b8ed865..c6450b7e428 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.45 + 0.6.46-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.45 + ${project.artifactId}-${project.version} diff --git a/processing/pom.xml b/processing/pom.xml index b2171bb5ccf..be81434b96b 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 36b0708dae5..9f3985b2336 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index e4e9fba065e..bfab2042108 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 6065d97a15c..8b9636b2fde 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index d464f50beb5..04f6f8fd63f 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.45 + 0.6.46-SNAPSHOT From 4d83837e88f6008cb040fef377a118c2432377e1 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 11:37:16 -0800 Subject: [PATCH 22/31] RealtimeIndexTask: Clean up imports and comments --- .../io/druid/indexing/common/task/RealtimeIndexTask.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 6ddc523140b..c9235a045ad 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import com.google.common.io.Closeables; import com.metamx.common.exception.FormattedException; import com.metamx.emitter.EmittingLogger; @@ -35,9 +34,7 @@ import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockAcquireAction; -import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.LockReleaseAction; -import io.druid.indexing.common.actions.SegmentInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -212,7 +209,7 @@ public class RealtimeIndexTask extends AbstractTask @Override public void announceSegment(final DataSegment segment) throws IOException { - // NOTE: Side effect: Calling announceSegment causes a lock to be acquired + // Side effect: Calling announceSegment causes a lock to be acquired toolbox.getTaskActionClient().submit(new LockAcquireAction(segment.getInterval())); toolbox.getSegmentAnnouncer().announceSegment(segment); } @@ -231,6 +228,7 @@ public class RealtimeIndexTask extends AbstractTask @Override public void announceSegments(Iterable segments) throws IOException { + // Side effect: Calling announceSegments causes locks to be acquired for (DataSegment segment : segments) { toolbox.getTaskActionClient().submit(new LockAcquireAction(segment.getInterval())); } @@ -263,7 +261,7 @@ public class RealtimeIndexTask extends AbstractTask public String getVersion(final Interval interval) { try { - // NOTE: Side effect: Calling getVersion causes a lock to be acquired + // Side effect: Calling getVersion causes a lock to be acquired final TaskLock myLock = toolbox.getTaskActionClient() .submit(new LockAcquireAction(interval)); From 837dee1934b7d94dc44154a82aec450507a27e88 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 11:38:42 -0800 Subject: [PATCH 23/31] RealtimePlumberSchool: Alert and continue on any exception when abandoning segments --- .../druid/segment/realtime/plumber/RealtimePlumberSchool.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java index 35abfcaf866..9dc18257b68 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -712,7 +712,7 @@ public class RealtimePlumberSchool implements PlumberSchool handoffCondition.notifyAll(); } } - catch (IOException e) { + catch (Exception e) { log.makeAlert(e, "Unable to abandon old segment for dataSource[%s]", schema.getDataSource()) .addData("interval", sink.getInterval()) .emit(); From 26991b5a2a33bc91cefdf958ffa2973e27eeebb3 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 12:05:42 -0800 Subject: [PATCH 24/31] Indexing service: Fix termination related log message --- .../overlord/scaling/SimpleResourceManagementStrategy.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java index fe08fd648e1..10e084b3c9e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java @@ -203,7 +203,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat .toList(); if (laziestWorkerIps.isEmpty()) { - log.info("Wanted to terminate %,d workers, but couldn't find any lazy ones!"); + log.info("Wanted to terminate %,d workers, but couldn't find any lazy ones!", excessWorkers); } else { log.info( "Terminating %,d workers (wanted %,d): %s", From bbe1b82347fc8cf0c126a60d7e8b12673a27ba5a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 12:26:27 -0800 Subject: [PATCH 25/31] Update versions --- build.sh | 2 +- docs/content/Booting-a-production-cluster.md | 2 +- docs/content/Examples.md | 4 ++-- docs/content/Realtime.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-2.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.textile | 2 +- examples/config/historical/runtime.properties | 2 +- examples/config/realtime/runtime.properties | 2 +- services/src/main/java/io/druid/cli/CliBroker.java | 2 +- services/src/main/java/io/druid/cli/CliCoordinator.java | 2 +- services/src/main/java/io/druid/cli/CliHadoopIndexer.java | 2 +- services/src/main/java/io/druid/cli/CliHistorical.java | 2 +- services/src/main/java/io/druid/cli/CliOverlord.java | 2 +- services/src/main/java/io/druid/cli/CliRealtime.java | 2 +- services/src/main/java/io/druid/cli/CliRealtimeExample.java | 2 +- 18 files changed, 23 insertions(+), 23 deletions(-) diff --git a/build.sh b/build.sh index 95ec936e1ef..50bcefbfe7b 100755 --- a/build.sh +++ b/build.sh @@ -30,4 +30,4 @@ echo "For examples, see: " echo " " ls -1 examples/*/*sh echo " " -echo "See also http://druid.io/docs/0.6.45" +echo "See also http://druid.io/docs/0.6.46" diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index 3d048ff6b1b..3f57ce13d1c 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -3,7 +3,7 @@ layout: doc_page --- # Booting a Single Node Cluster # -[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.45-bin.tar.gz). +[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.46-bin.tar.gz). The [ec2 run script](https://github.com/metamx/druid/blob/master/examples/bin/run_ec2.sh), run_ec2.sh, is located at 'examples/bin' if you have checked out the code, or at the root of the project if you've downloaded a tarball. The scripts rely on the [Amazon EC2 API Tools](http://aws.amazon.com/developertools/351), and you will need to set three environment variables: diff --git a/docs/content/Examples.md b/docs/content/Examples.md index e94c11bcf0d..f0771408c75 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.45 +git checkout druid-0.6.46 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.45-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.46-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Realtime.md b/docs/content/Realtime.md index 978bf4121d4..a6111a8734b 100644 --- a/docs/content/Realtime.md +++ b/docs/content/Realtime.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.45"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.46"] druid.zk.service.host=localhost diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 0d4bb480947..1f6f43ac97e 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.45-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.46-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.45 +cd druid-services-0.6.46 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index 5c6c665de11..cffb288f5b3 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -44,7 +44,7 @@ With real-world data, we recommend having a message bus such as [Apache Kafka](h #### Setting up Kafka -[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.45/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. +[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.46/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html). diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 62cf2fbcbdd..b2f5b6975ec 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes as well as and exter If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.45-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.46-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.45"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.46"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -238,7 +238,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.45","io.druid.extensions:druid-kafka-seven:0.6.45"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.46","io.druid.extensions:druid-kafka-seven:0.6.46"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index e3607d75503..cdbbc5f7ee7 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.45-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.46-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.45 +cd druid-services-0.6.46 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.textile b/docs/content/Twitter-Tutorial.textile index e66cd62ce34..68780006068 100644 --- a/docs/content/Twitter-Tutorial.textile +++ b/docs/content/Twitter-Tutorial.textile @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.45-bin.tar.gz. +We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.46-bin.tar.gz. Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index 2c7950d4112..8d9f1f35096 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -4,7 +4,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.45"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.46"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index 0946d6753e1..6ebf2e5dcdf 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -4,7 +4,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.45","io.druid.extensions:druid-kafka-seven:0.6.45","io.druid.extensions:druid-rabbitmq:0.6.45"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.46","io.druid.extensions:druid-kafka-seven:0.6.46","io.druid.extensions:druid-rabbitmq:0.6.46"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 2feda4e5e03..ec37aefbe82 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -53,7 +53,7 @@ import java.util.List; */ @Command( name = "broker", - description = "Runs a broker node, see http://druid.io/docs/0.6.45/Broker.html for a description" + description = "Runs a broker node, see http://druid.io/docs/0.6.46/Broker.html for a description" ) public class CliBroker extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 3e6da7fbb42..27c3f8021ab 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -60,7 +60,7 @@ import java.util.List; */ @Command( name = "coordinator", - description = "Runs the Coordinator, see http://druid.io/docs/0.6.45/Coordinator.html for a description." + description = "Runs the Coordinator, see http://druid.io/docs/0.6.46/Coordinator.html for a description." ) public class CliCoordinator extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java index b654e8de86b..a667b967b30 100644 --- a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java @@ -41,7 +41,7 @@ import java.util.List; */ @Command( name = "hadoop", - description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.45/Batch-ingestion.html for a description." + description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.46/Batch-ingestion.html for a description." ) public class CliHadoopIndexer implements Runnable { diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index d53ddbf0a89..2be3c013820 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -42,7 +42,7 @@ import java.util.List; */ @Command( name = "historical", - description = "Runs a Historical node, see http://druid.io/docs/0.6.45/Historical.html for a description" + description = "Runs a Historical node, see http://druid.io/docs/0.6.46/Historical.html for a description" ) public class CliHistorical extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 47777163726..020178f7b33 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -93,7 +93,7 @@ import java.util.List; */ @Command( name = "overlord", - description = "Runs an Overlord node, see http://druid.io/docs/0.6.45/Indexing-Service.html for a description" + description = "Runs an Overlord node, see http://druid.io/docs/0.6.46/Indexing-Service.html for a description" ) public class CliOverlord extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index ade1069c899..bfc05eb748c 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -30,7 +30,7 @@ import java.util.List; */ @Command( name = "realtime", - description = "Runs a realtime node, see http://druid.io/docs/0.6.45/Realtime.html for a description" + description = "Runs a realtime node, see http://druid.io/docs/0.6.46/Realtime.html for a description" ) public class CliRealtime extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index e5cc2249bca..dd2605140c2 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -42,7 +42,7 @@ import java.util.concurrent.Executor; */ @Command( name = "realtime", - description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.45/Realtime.html for a description" + description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.46/Realtime.html for a description" ) public class CliRealtimeExample extends ServerRunnable { From bd58def933cabf6e3ae48d3b2eb3304e045a6e27 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 12:28:49 -0800 Subject: [PATCH 26/31] [maven-release-plugin] prepare release druid-0.6.46 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 9879e2b100f..e66feb4a3a9 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/common/pom.xml b/common/pom.xml index 9ca631cae9d..ba3d89c7f38 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/examples/pom.xml b/examples/pom.xml index fa55220a573..28dc6af0e2c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 05bab3edf76..7e4b6b562b9 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 4adea254904..c6f7a1ae1e4 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 1d828b6b005..74be68f6991 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 692121cdbec..a8cea059613 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 084eea93d83..f110c2828f4 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/pom.xml b/pom.xml index c6450b7e428..250588c7809 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.46-SNAPSHOT + 0.6.46 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - ${project.artifactId}-${project.version} + druid-0.6.46 diff --git a/processing/pom.xml b/processing/pom.xml index be81434b96b..3459c90532a 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 9f3985b2336..ec8fd567106 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index bfab2042108..c6402ee47eb 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/server/pom.xml b/server/pom.xml index 8b9636b2fde..731d079aab9 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 diff --git a/services/pom.xml b/services/pom.xml index 04f6f8fd63f..3cc43f3599f 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.46-SNAPSHOT + 0.6.46 From 668cf009a07a0318f32aa008db454b966e1a38a5 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Dec 2013 12:28:54 -0800 Subject: [PATCH 27/31] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index e66feb4a3a9..57ad7d79e58 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index ba3d89c7f38..6a481899484 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 28dc6af0e2c..0ec8f06142c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 7e4b6b562b9..3d6343c7183 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index c6f7a1ae1e4..1562f24a5e5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 74be68f6991..78447097bbe 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index a8cea059613..e378ffa75ef 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index f110c2828f4..acc488d3c26 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/pom.xml b/pom.xml index 250588c7809..fb0dd02b1bd 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.46 + 0.6.47-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.46 + ${project.artifactId}-${project.version} diff --git a/processing/pom.xml b/processing/pom.xml index 3459c90532a..6344bb65d91 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index ec8fd567106..bf9653b5fb1 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c6402ee47eb..5880eea1657 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 731d079aab9..e58278a173a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 3cc43f3599f..4c462e95139 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.46 + 0.6.47-SNAPSHOT From 95d92915bf2f78122751f0f8ad57afc560909691 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 27 Dec 2013 16:13:38 -0800 Subject: [PATCH 28/31] more tests for Initialization + dependency cleanup --- pom.xml | 2 +- processing/pom.xml | 2 - server/pom.xml | 58 ------- .../java/io/druid/server/StatusResource.java | 19 ++- .../initialization/InitializationTest.java | 152 ++++++++++++++++++ .../io/druid/server/StatusResourceTest.java | 52 +----- .../io.druid.initialization.DruidModule | 2 +- .../src/main/java/io/druid/cli/Version.java | 4 +- 8 files changed, 172 insertions(+), 119 deletions(-) create mode 100644 server/src/test/java/io/druid/initialization/InitializationTest.java diff --git a/pom.xml b/pom.xml index fb0dd02b1bd..1114e9238b1 100644 --- a/pom.xml +++ b/pom.xml @@ -409,7 +409,7 @@ junit junit - 4.8.1 + 4.11 test diff --git a/processing/pom.xml b/processing/pom.xml index 6344bb65d91..8e67e03671a 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -49,8 +49,6 @@ com.metamx emitter - - com.ning compress-lzf diff --git a/server/pom.xml b/server/pom.xml index e58278a173a..c79a29e38b3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -37,24 +37,14 @@ druid-processing ${project.parent.version} - - - com.metamx - emitter - com.metamx http-client - - com.metamx - java-util - com.metamx server-metrics - commons-cli commons-cli @@ -63,22 +53,10 @@ commons-lang commons-lang - - commons-io - commons-io - com.amazonaws aws-java-sdk - - com.ning - compress-lzf - - - org.skife.config - config-magic - org.apache.curator curator-framework @@ -87,38 +65,14 @@ org.apache.curator curator-x-discovery - - it.uniroma3.mat - extendedset - - - com.google.guava - guava - - - com.google.inject - guice - - - com.fasterxml.jackson.core - jackson-core - com.fasterxml.jackson.jaxrs jackson-jaxrs-json-provider - - com.fasterxml.jackson.core - jackson-databind - com.fasterxml.jackson.dataformat jackson-dataformat-smile - - javax.inject - javax.inject - org.jdbi jdbi @@ -139,22 +93,10 @@ org.eclipse.jetty jetty-server - - joda-time - joda-time - com.google.code.findbugs jsr305 - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - io.tesla.aether tesla-aether diff --git a/server/src/main/java/io/druid/server/StatusResource.java b/server/src/main/java/io/druid/server/StatusResource.java index 643d4d4fd74..44aa50a0ee7 100644 --- a/server/src/main/java/io/druid/server/StatusResource.java +++ b/server/src/main/java/io/druid/server/StatusResource.java @@ -28,8 +28,8 @@ import javax.ws.rs.GET; import javax.ws.rs.Path; import javax.ws.rs.Produces; import java.util.ArrayList; +import java.util.Collection; import java.util.List; -import java.util.Set; /** */ @@ -40,7 +40,7 @@ public class StatusResource @Produces("application/json") public Status doGet() { - return new Status(); + return new Status(Initialization.getLoadedModules(DruidModule.class)); } public static class Status @@ -49,13 +49,18 @@ public class StatusResource final List modules; final Memory memory; - public Status() + public Status(Collection modules) { - this.version = Status.class.getPackage().getImplementationVersion(); - this.modules = getExtensionVersions(); + this.version = getDruidVersion(); + this.modules = getExtensionVersions(modules); this.memory = new Memory(Runtime.getRuntime()); } + private String getDruidVersion() + { + return Status.class.getPackage().getImplementationVersion(); + } + @JsonProperty public String getVersion() { @@ -98,9 +103,8 @@ public class StatusResource * * @return map of extensions loaded with their respective implementation versions. */ - private List getExtensionVersions() + private List getExtensionVersions(Collection druidModules) { - final Set druidModules = Initialization.getLoadedModules(DruidModule.class); List moduleVersions = new ArrayList<>(); for (DruidModule module : druidModules) { String artifact = module.getClass().getPackage().getImplementationTitle(); @@ -110,7 +114,6 @@ public class StatusResource } return moduleVersions; } - } @JsonInclude(JsonInclude.Include.NON_NULL) diff --git a/server/src/test/java/io/druid/initialization/InitializationTest.java b/server/src/test/java/io/druid/initialization/InitializationTest.java new file mode 100644 index 00000000000..d1cb0afd0c5 --- /dev/null +++ b/server/src/test/java/io/druid/initialization/InitializationTest.java @@ -0,0 +1,152 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.initialization; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.collect.Collections2; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import io.druid.server.initialization.ExtensionsConfig; +import junit.framework.Assert; +import org.junit.After; +import org.junit.Before; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.List; +import java.util.Set; + +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class InitializationTest +{ + private String oldService; + private String oldHost; + private String oldPort; + + @Before + public void messWithSystemProperties() + { + // required to test Initialization.makeInjectorWithModules + oldService = System.setProperty("druid.service", "test-service"); + oldHost = System.setProperty("druid.host", "test-host"); + oldPort = System.setProperty("druid.port", "8080"); + } + + @After + public void cleanup() + { + System.setProperty("druid.service", oldService == null ? "" : oldService); + System.setProperty("druid.host", oldHost == null ? "" : oldHost); + System.setProperty("druid.port", oldPort == null ? "" : oldPort); + } + + @Test + public void test01InitialModulesEmpty() throws Exception + { + Assert.assertEquals( + "Initial set of loaded modules must be empty", + 0, + Initialization.getLoadedModules(DruidModule.class).size() + ); + } + + @Test + public void test02MakeStartupInjector() throws Exception + { + Injector startupInjector = Initialization.makeStartupInjector(); + Assert.assertNotNull(startupInjector); + Assert.assertNotNull(startupInjector.getInstance(ObjectMapper.class)); + } + + @Test + public void test03ClassLoaderExtensionsLoading() + { + Injector startupInjector = Initialization.makeStartupInjector(); + + Function fnClassName = new Function() + { + @Nullable + @Override + public String apply(@Nullable DruidModule input) + { + return input.getClass().getCanonicalName(); + } + }; + + Assert.assertFalse( + "modules does not contain TestDruidModule", + Collections2.transform(Initialization.getLoadedModules(DruidModule.class), fnClassName) + .contains("io.druid.initialization.InitializationTest.TestDruidModule") + ); + + Collection modules = Initialization.getFromExtensions( + startupInjector.getInstance(ExtensionsConfig.class), + DruidModule.class + ); + + Assert.assertTrue( + "modules contains TestDruidModule", + Collections2.transform(modules, fnClassName) + .contains("io.druid.initialization.InitializationTest.TestDruidModule") + ); + } + + @Test + public void test04MakeInjectorWithModules() throws Exception + { + Injector startupInjector = Initialization.makeStartupInjector(); + Injector injector = Initialization.makeInjectorWithModules(startupInjector, ImmutableList.of()); + Assert.assertNotNull(injector); + } + + @Test + public void testGetLoadedModules() + { + + Set modules = Initialization.getLoadedModules(DruidModule.class); + + Set loadedModules = Initialization.getLoadedModules(DruidModule.class); + Assert.assertEquals("Set from loaded modules #1 should be same!", modules, loadedModules); + + Set loadedModules2 = Initialization.getLoadedModules(DruidModule.class); + Assert.assertEquals("Set from loaded modules #2 should be same!", modules, loadedModules2); + } + + public static class TestDruidModule implements DruidModule + { + @Override + public List getJacksonModules() + { + return ImmutableList.of(); + } + + @Override + public void configure(Binder binder) + { + // Do nothing + } + } +} diff --git a/server/src/test/java/io/druid/server/StatusResourceTest.java b/server/src/test/java/io/druid/server/StatusResourceTest.java index cff0fb4618e..9075f97ce81 100644 --- a/server/src/test/java/io/druid/server/StatusResourceTest.java +++ b/server/src/test/java/io/druid/server/StatusResourceTest.java @@ -19,19 +19,14 @@ package io.druid.server; -import com.fasterxml.jackson.databind.Module; import com.google.common.collect.ImmutableList; -import com.google.inject.Binder; -import com.google.inject.Injector; import io.druid.initialization.DruidModule; -import io.druid.initialization.Initialization; -import io.druid.server.initialization.ExtensionsConfig; +import io.druid.initialization.InitializationTest; import junit.framework.Assert; import org.junit.Test; import java.util.Collection; import java.util.List; -import java.util.Set; import static io.druid.server.StatusResource.ModuleVersion; @@ -39,29 +34,14 @@ import static io.druid.server.StatusResource.ModuleVersion; */ public class StatusResourceTest { - - private Collection loadTestModule() - { - Injector baseInjector = Initialization.makeStartupInjector(); - return Initialization.getFromExtensions(baseInjector.getInstance(ExtensionsConfig.class), DruidModule.class); - } - @Test public void testLoadedModules() { - final StatusResource resource = new StatusResource(); - List statusResourceModuleList; - statusResourceModuleList = resource.doGet().getModules(); - Assert.assertEquals( - "No Modules should be loaded currently! " + statusResourceModuleList, - statusResourceModuleList.size(), 0 - ); + Collection modules = ImmutableList.of((DruidModule)new InitializationTest.TestDruidModule()); + List statusResourceModuleList = new StatusResource.Status(modules).getModules(); - Collection modules = loadTestModule(); - statusResourceModuleList = resource.doGet().getModules(); - - Assert.assertEquals("Status should have all modules loaded!", statusResourceModuleList.size(), modules.size()); + Assert.assertEquals("Status should have all modules loaded!", modules.size(), statusResourceModuleList.size()); for (DruidModule module : modules) { String moduleName = module.getClass().getCanonicalName(); @@ -74,30 +54,6 @@ public class StatusResourceTest } Assert.assertTrue("Status resource should contain module " + moduleName, contains); } - - /* - * StatusResource only uses Initialization.getLoadedModules - */ - for (int i = 0; i < 5; i++) { - Set loadedModules = Initialization.getLoadedModules(DruidModule.class); - Assert.assertEquals("Set from loaded module should be same!", loadedModules, modules); - } } - - public static class TestDruidModule implements DruidModule - { - @Override - public List getJacksonModules() - { - return ImmutableList.of(); - } - - @Override - public void configure(Binder binder) - { - // Do nothing - } - } - } diff --git a/server/src/test/resources/META-INF/services/io.druid.initialization.DruidModule b/server/src/test/resources/META-INF/services/io.druid.initialization.DruidModule index 09678a839eb..b5bc03d5265 100644 --- a/server/src/test/resources/META-INF/services/io.druid.initialization.DruidModule +++ b/server/src/test/resources/META-INF/services/io.druid.initialization.DruidModule @@ -1 +1 @@ -io.druid.server.StatusResourceTest$TestDruidModule +io.druid.initialization.InitializationTest$TestDruidModule diff --git a/services/src/main/java/io/druid/cli/Version.java b/services/src/main/java/io/druid/cli/Version.java index 210591e81e6..45c258de803 100644 --- a/services/src/main/java/io/druid/cli/Version.java +++ b/services/src/main/java/io/druid/cli/Version.java @@ -20,6 +20,8 @@ package io.druid.cli; import io.airlift.command.Command; +import io.druid.initialization.DruidModule; +import io.druid.initialization.Initialization; import io.druid.server.StatusResource; @Command( @@ -31,6 +33,6 @@ public class Version implements Runnable @Override public void run() { - System.out.println(new StatusResource.Status()); + System.out.println(new StatusResource.Status(Initialization.getLoadedModules(DruidModule.class))); } } From ef05312c97da0848b0e03638a0d3baf5ee037593 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 30 Dec 2013 15:26:46 -0800 Subject: [PATCH 29/31] cleanup dependencies --- common/pom.xml | 22 ---------- indexing-service/pom.xml | 91 ++-------------------------------------- pom.xml | 10 +++++ processing/pom.xml | 37 +++------------- server/pom.xml | 8 ++-- 5 files changed, 22 insertions(+), 146 deletions(-) diff --git a/common/pom.xml b/common/pom.xml index 6a481899484..b46929f30d2 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -59,14 +59,6 @@ org.skife.config config-magic - - org.apache.curator - curator-recipes - - - org.apache.curator - curator-x-discovery - org.hibernate hibernate-validator @@ -75,10 +67,6 @@ javax.validation validation-api - - it.uniroma3.mat - extendedset - com.google.guava guava @@ -127,16 +115,6 @@ log4j log4j - - mysql - mysql-connector-java - 5.1.18 - - - org.mozilla - rhino - 1.7R4 - diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 78447097bbe..82166796035 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -47,95 +47,10 @@ druid-indexing-hadoop ${project.parent.version} - - com.metamx - emitter - - - com.metamx - http-client - - - com.metamx - java-util - - - com.metamx - server-metrics - - - - commons-codec - commons-codec - - - commons-io - commons-io - - - org.skife.config - config-magic - - - org.apache.curator - curator-framework - - - org.apache.curator - curator-recipes - - - com.google.guava - guava - - - com.google.inject - guice - - - com.google.inject.extensions - guice-servlet - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-json-provider - - - com.fasterxml.jackson.core - jackson-databind - - - javax.inject - javax.inject - - - org.jdbi - jdbi - - - com.sun.jersey - jersey-core - - - com.sun.jersey.contribs - jersey-guice - - - org.eclipse.jetty - jetty-server - - - joda-time - joda-time - - - com.google.code.findbugs - jsr305 + mysql + mysql-connector-java + 5.1.18 diff --git a/pom.xml b/pom.xml index 1114e9238b1..2bae36d0f52 100644 --- a/pom.xml +++ b/pom.xml @@ -289,6 +289,16 @@ com.sun.jersey.contribs jersey-guice 1.17.1 + + + com.google.inject + guice + + + com.google.inject.extensions + guice-servlet + + com.sun.jersey diff --git a/processing/pom.xml b/processing/pom.xml index 8e67e03671a..21f35318569 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -37,10 +37,6 @@ druid-common ${project.parent.version} - - com.metamx - java-util - com.metamx bytebuffer-collections @@ -61,34 +57,6 @@ it.uniroma3.mat extendedset - - com.google.guava - guava - - - com.google.inject - guice - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - - javax.inject - javax.inject - - - joda-time - joda-time - - - log4j - log4j - org.slf4j slf4j-log4j12 @@ -105,6 +73,11 @@ com.ibm.icu icu4j + + org.mozilla + rhino + 1.7R4 + diff --git a/server/pom.xml b/server/pom.xml index c79a29e38b3..f41fba0b57f 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -53,6 +53,10 @@ commons-lang commons-lang + + javax.inject + javax.inject + com.amazonaws aws-java-sdk @@ -73,10 +77,6 @@ com.fasterxml.jackson.dataformat jackson-dataformat-smile - - org.jdbi - jdbi - com.sun.jersey jersey-server From c0e7837a586fd8624b407d6d58810dee7b74c88d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 2 Jan 2014 10:18:31 -0800 Subject: [PATCH 30/31] add back guice-servlet --- server/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/server/pom.xml b/server/pom.xml index f41fba0b57f..eda8a93fcb4 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -85,6 +85,10 @@ com.sun.jersey jersey-core + + com.google.inject.extensions + guice-servlet + com.sun.jersey.contribs jersey-guice From e4bcbcf3cc71d956a433c9fdad91866572659ae7 Mon Sep 17 00:00:00 2001 From: Jae Hyeon Bae Date: Thu, 2 Jan 2014 10:35:09 -0800 Subject: [PATCH 31/31] curator 2.3.0 --- pom.xml | 2 +- .../curator/discovery/DiscoveryModule.java | 22 ++++++++++--------- .../discovery/ServerDiscoveryFactory.java | 5 +++++ 3 files changed, 18 insertions(+), 11 deletions(-) diff --git a/pom.xml b/pom.xml index fb0dd02b1bd..66ab8a1daf7 100644 --- a/pom.xml +++ b/pom.xml @@ -40,7 +40,7 @@ UTF-8 0.25.1 - 2.1.0-incubating + 2.3.0 0.1.7 diff --git a/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java index f3fc56d59a7..bc1558f60f6 100644 --- a/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java +++ b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java @@ -40,14 +40,7 @@ import io.druid.guice.annotations.Self; import io.druid.server.DruidNode; import io.druid.server.initialization.CuratorDiscoveryConfig; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.x.discovery.ProviderStrategy; -import org.apache.curator.x.discovery.ServiceCache; -import org.apache.curator.x.discovery.ServiceCacheBuilder; -import org.apache.curator.x.discovery.ServiceDiscovery; -import org.apache.curator.x.discovery.ServiceDiscoveryBuilder; -import org.apache.curator.x.discovery.ServiceInstance; -import org.apache.curator.x.discovery.ServiceProvider; -import org.apache.curator.x.discovery.ServiceProviderBuilder; +import org.apache.curator.x.discovery.*; import org.apache.curator.x.discovery.details.ServiceCacheListener; import java.io.IOException; @@ -389,8 +382,12 @@ public class DiscoveryModule implements Module } @Override - public ServiceProviderBuilder refreshPaddingMs(int refreshPaddingMs) - { + public ServiceProviderBuilder downInstancePolicy(DownInstancePolicy downInstancePolicy) { + return this; + } + + @Override + public ServiceProviderBuilder additionalFilter(InstanceFilter tInstanceFilter) { return this; } } @@ -409,6 +406,11 @@ public class DiscoveryModule implements Module return null; } + @Override + public void noteError(ServiceInstance tServiceInstance) { + + } + @Override public void close() throws IOException { diff --git a/server/src/main/java/io/druid/curator/discovery/ServerDiscoveryFactory.java b/server/src/main/java/io/druid/curator/discovery/ServerDiscoveryFactory.java index c436289e70e..0e66df0b9ed 100644 --- a/server/src/main/java/io/druid/curator/discovery/ServerDiscoveryFactory.java +++ b/server/src/main/java/io/druid/curator/discovery/ServerDiscoveryFactory.java @@ -62,6 +62,11 @@ public class ServerDiscoveryFactory return null; } + @Override + public void noteError(ServiceInstance tServiceInstance) { + // do nothing + } + @Override public void close() throws IOException {