From 63b2f502ac9ee16ff1a6e36ef21918e52f2373da Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 2 Nov 2012 11:35:23 -0500 Subject: [PATCH 01/19] 1) Make TimelineEntry in VersionedIntervalTimeline a static class with its own generic parameters to hopefully make OpenJDK happier. --- .../druid/VersionedIntervalTimeline.java | 88 ++++++++++--------- .../druid/VersionedIntervalTimelineTest.java | 6 ++ 2 files changed, 52 insertions(+), 42 deletions(-) diff --git a/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java b/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java index a45821c7dba..a8c708917eb 100644 --- a/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java +++ b/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java @@ -61,13 +61,13 @@ public class VersionedIntervalTimeline private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); - final NavigableMap completePartitionsTimeline = new TreeMap( + final NavigableMap> completePartitionsTimeline = Maps.newTreeMap( Comparators.intervalsByStartThenEnd() ); - final NavigableMap incompletePartitionsTimeline = new TreeMap( + final NavigableMap> incompletePartitionsTimeline = Maps.newTreeMap( Comparators.intervalsByStartThenEnd() ); - private final Map> allTimelineEntries = Maps.newHashMap(); + private final Map>> allTimelineEntries = Maps.newHashMap(); private final Comparator versionComparator; @@ -83,19 +83,23 @@ public class VersionedIntervalTimeline try { lock.writeLock().lock(); - Map exists = allTimelineEntries.get(interval); - TimelineEntry entry = null; + Map> exists = allTimelineEntries.get(interval); + TimelineEntry entry = null; if (exists == null) { - entry = new TimelineEntry(interval, version, new PartitionHolder(object)); - TreeMap versionEntry = Maps.newTreeMap(versionComparator); + entry = new TimelineEntry(interval, version, new PartitionHolder(object)); + TreeMap> versionEntry = Maps.newTreeMap(versionComparator); versionEntry.put(version, entry); allTimelineEntries.put(interval, versionEntry); } else { entry = exists.get(version); if (entry == null) { - entry = new TimelineEntry(interval, version, new PartitionHolder(object)); + entry = new TimelineEntry( + interval, + version, + new PartitionHolder(object) + ); exists.put(version, entry); } else { PartitionHolder partitionHolder = entry.getPartitionHolder(); @@ -119,12 +123,12 @@ public class VersionedIntervalTimeline try { lock.writeLock().lock(); - Map versionEntries = allTimelineEntries.get(interval); + Map> versionEntries = allTimelineEntries.get(interval); if (versionEntries == null) { return null; } - TimelineEntry entry = versionEntries.get(version); + TimelineEntry entry = versionEntries.get(version); if (entry == null) { return null; } @@ -152,7 +156,7 @@ public class VersionedIntervalTimeline { try { lock.readLock().lock(); - for (Map.Entry> entry : allTimelineEntries.entrySet()) { + for (Map.Entry>> entry : allTimelineEntries.entrySet()) { if (entry.getKey().equals(interval) || entry.getKey().contains(interval)) { TimelineEntry foundEntry = entry.getValue().get(version); if (foundEntry != null) { @@ -197,13 +201,14 @@ public class VersionedIntervalTimeline List> retVal = new ArrayList>(); Map> overShadowed = Maps.newHashMap(); - for (Map.Entry> versionEntry : allTimelineEntries.entrySet()) { + for (Map.Entry>> versionEntry : allTimelineEntries + .entrySet()) { Map versionCopy = Maps.newHashMap(); versionCopy.putAll(versionEntry.getValue()); overShadowed.put(versionEntry.getKey(), versionCopy); } - for (Map.Entry entry : completePartitionsTimeline.entrySet()) { + for (Map.Entry> entry : completePartitionsTimeline.entrySet()) { Map versionEntry = overShadowed.get(entry.getValue().getTrueInterval()); if (versionEntry != null) { versionEntry.remove(entry.getValue().getVersion()); @@ -213,7 +218,7 @@ public class VersionedIntervalTimeline } } - for (Map.Entry entry : incompletePartitionsTimeline.entrySet()) { + for (Map.Entry> entry : incompletePartitionsTimeline.entrySet()) { Map versionEntry = overShadowed.get(entry.getValue().getTrueInterval()); if (versionEntry != null) { versionEntry.remove(entry.getValue().getVersion()); @@ -225,7 +230,7 @@ public class VersionedIntervalTimeline for (Map.Entry> versionEntry : overShadowed.entrySet()) { for (Map.Entry entry : versionEntry.getValue().entrySet()) { - TimelineEntry object = entry.getValue(); + TimelineEntry object = entry.getValue(); retVal.add( new TimelineObjectHolder( object.getTrueInterval(), @@ -244,12 +249,12 @@ public class VersionedIntervalTimeline } private void add( - NavigableMap timeline, + NavigableMap> timeline, Interval interval, - TimelineEntry entry + TimelineEntry entry ) { - TimelineEntry existsInTimeline = timeline.get(interval); + TimelineEntry existsInTimeline = timeline.get(interval); if (existsInTimeline != null) { int compare = versionComparator.compare(entry.getVersion(), existsInTimeline.getVersion()); @@ -279,9 +284,9 @@ public class VersionedIntervalTimeline } private boolean addAtKey( - NavigableMap timeline, + NavigableMap> timeline, Interval key, - TimelineEntry entry + TimelineEntry entry ) { boolean retVal = false; @@ -315,7 +320,7 @@ public class VersionedIntervalTimeline } } } else if (versionCompare > 0) { - TimelineEntry oldEntry = timeline.remove(currKey); + TimelineEntry oldEntry = timeline.remove(currKey); if (currKey.contains(entryInterval)) { addIntervalToTimeline(new Interval(currKey.getStart(), entryInterval.getStart()), oldEntry, timeline); @@ -355,8 +360,8 @@ public class VersionedIntervalTimeline private void addIntervalToTimeline( Interval interval, - TimelineEntry entry, - NavigableMap timeline + TimelineEntry entry, + NavigableMap> timeline ) { if (interval != null && interval.toDurationMillis() > 0) { @@ -365,9 +370,9 @@ public class VersionedIntervalTimeline } private void remove( - NavigableMap timeline, + NavigableMap> timeline, Interval interval, - TimelineEntry entry, + TimelineEntry entry, boolean incompleteOk ) { @@ -375,9 +380,9 @@ public class VersionedIntervalTimeline TimelineEntry removed = timeline.get(interval); if (removed == null) { - Iterator> iter = timeline.entrySet().iterator(); + Iterator>> iter = timeline.entrySet().iterator(); while (iter.hasNext()) { - Map.Entry timelineEntry = iter.next(); + Map.Entry> timelineEntry = iter.next(); if (timelineEntry.getValue() == entry) { intervalsToRemove.add(timelineEntry.getKey()); } @@ -392,14 +397,15 @@ public class VersionedIntervalTimeline } private void remove( - NavigableMap timeline, + NavigableMap> timeline, Interval interval, boolean incompleteOk ) { timeline.remove(interval); - for (Map.Entry> versionEntry : allTimelineEntries.entrySet()) { + for (Map.Entry>> versionEntry : allTimelineEntries + .entrySet()) { if (versionEntry.getKey().overlap(interval) != null) { TimelineEntry timelineEntry = versionEntry.getValue().lastEntry().getValue(); if (timelineEntry.getPartitionHolder().isComplete() || incompleteOk) { @@ -412,20 +418,18 @@ public class VersionedIntervalTimeline private List> lookup(Interval interval, boolean incompleteOk) { List> retVal = new ArrayList>(); - NavigableMap timeline = (incompleteOk) - ? incompletePartitionsTimeline - : completePartitionsTimeline; + NavigableMap> timeline = (incompleteOk) + ? incompletePartitionsTimeline + : completePartitionsTimeline; - for (Map.Entry entry : timeline.entrySet()) { + for (Map.Entry> entry : timeline.entrySet()) { Interval timelineInterval = entry.getKey(); - TimelineEntry val = entry.getValue(); + TimelineEntry val = entry.getValue(); if (timelineInterval.overlaps(interval)) { retVal.add( new TimelineObjectHolder( - timelineInterval, - val.getVersion(), - val.getPartitionHolder() + timelineInterval, val.getVersion(), val.getPartitionHolder() ) ); } @@ -463,30 +467,30 @@ public class VersionedIntervalTimeline return retVal; } - public class TimelineEntry + private static class TimelineEntry { private final Interval trueInterval; private final VersionType version; private final PartitionHolder partitionHolder; - public TimelineEntry(Interval trueInterval, VersionType version, PartitionHolder partitionHolder) + private TimelineEntry(Interval trueInterval, VersionType version, PartitionHolder partitionHolder) { this.trueInterval = trueInterval; this.version = version; this.partitionHolder = partitionHolder; } - public Interval getTrueInterval() + private Interval getTrueInterval() { return trueInterval; } - public VersionType getVersion() + private VersionType getVersion() { return version; } - public PartitionHolder getPartitionHolder() + private PartitionHolder getPartitionHolder() { return partitionHolder; } diff --git a/common/src/test/java/com/metamx/druid/VersionedIntervalTimelineTest.java b/common/src/test/java/com/metamx/druid/VersionedIntervalTimelineTest.java index 96d0fa01ce1..90fdee34e83 100644 --- a/common/src/test/java/com/metamx/druid/VersionedIntervalTimelineTest.java +++ b/common/src/test/java/com/metamx/druid/VersionedIntervalTimelineTest.java @@ -21,6 +21,7 @@ package com.metamx.druid; import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; +import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.druid.partition.ImmutablePartitionHolder; import com.metamx.druid.partition.IntegerPartitionChunk; @@ -28,6 +29,7 @@ import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.PartitionHolder; import com.metamx.druid.partition.SingleElementPartitionChunk; import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; import org.joda.time.Days; import org.joda.time.Hours; import org.joda.time.Interval; @@ -48,6 +50,10 @@ public class VersionedIntervalTimelineTest @Before public void setUp() throws Exception { + if (! "UTC".equals(DateTimeZone.getDefault().getID())) { + throw new ISE("Tests assume default timezone of UTC, please set -Duser.timezone=UTC"); + } + timeline = makeStringIntegerTimeline(); add("2011-04-01/2011-04-03", "1", 2); From 7921be790b43aafdff450c56f62a050912dd035d Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 2 Nov 2012 14:17:50 -0500 Subject: [PATCH 02/19] 1) Bump up the maven version pre-requisite to 3.0.3 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 35ecc263c9a..cbeb399f8ed 100644 --- a/pom.xml +++ b/pom.xml @@ -34,7 +34,7 @@ - 2.2.1 + 3.0.3 From 38ed1d8b75e1d720d22acd09435814621204fdb8 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 2 Nov 2012 14:24:09 -0500 Subject: [PATCH 03/19] 1) Revert failed attempt at making OpenJDK6 happy. --- .../druid/VersionedIntervalTimeline.java | 88 +++++++++---------- .../druid/VersionedIntervalTimelineTest.java | 6 -- 2 files changed, 42 insertions(+), 52 deletions(-) diff --git a/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java b/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java index a8c708917eb..a45821c7dba 100644 --- a/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java +++ b/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java @@ -61,13 +61,13 @@ public class VersionedIntervalTimeline private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); - final NavigableMap> completePartitionsTimeline = Maps.newTreeMap( + final NavigableMap completePartitionsTimeline = new TreeMap( Comparators.intervalsByStartThenEnd() ); - final NavigableMap> incompletePartitionsTimeline = Maps.newTreeMap( + final NavigableMap incompletePartitionsTimeline = new TreeMap( Comparators.intervalsByStartThenEnd() ); - private final Map>> allTimelineEntries = Maps.newHashMap(); + private final Map> allTimelineEntries = Maps.newHashMap(); private final Comparator versionComparator; @@ -83,23 +83,19 @@ public class VersionedIntervalTimeline try { lock.writeLock().lock(); - Map> exists = allTimelineEntries.get(interval); - TimelineEntry entry = null; + Map exists = allTimelineEntries.get(interval); + TimelineEntry entry = null; if (exists == null) { - entry = new TimelineEntry(interval, version, new PartitionHolder(object)); - TreeMap> versionEntry = Maps.newTreeMap(versionComparator); + entry = new TimelineEntry(interval, version, new PartitionHolder(object)); + TreeMap versionEntry = Maps.newTreeMap(versionComparator); versionEntry.put(version, entry); allTimelineEntries.put(interval, versionEntry); } else { entry = exists.get(version); if (entry == null) { - entry = new TimelineEntry( - interval, - version, - new PartitionHolder(object) - ); + entry = new TimelineEntry(interval, version, new PartitionHolder(object)); exists.put(version, entry); } else { PartitionHolder partitionHolder = entry.getPartitionHolder(); @@ -123,12 +119,12 @@ public class VersionedIntervalTimeline try { lock.writeLock().lock(); - Map> versionEntries = allTimelineEntries.get(interval); + Map versionEntries = allTimelineEntries.get(interval); if (versionEntries == null) { return null; } - TimelineEntry entry = versionEntries.get(version); + TimelineEntry entry = versionEntries.get(version); if (entry == null) { return null; } @@ -156,7 +152,7 @@ public class VersionedIntervalTimeline { try { lock.readLock().lock(); - for (Map.Entry>> entry : allTimelineEntries.entrySet()) { + for (Map.Entry> entry : allTimelineEntries.entrySet()) { if (entry.getKey().equals(interval) || entry.getKey().contains(interval)) { TimelineEntry foundEntry = entry.getValue().get(version); if (foundEntry != null) { @@ -201,14 +197,13 @@ public class VersionedIntervalTimeline List> retVal = new ArrayList>(); Map> overShadowed = Maps.newHashMap(); - for (Map.Entry>> versionEntry : allTimelineEntries - .entrySet()) { + for (Map.Entry> versionEntry : allTimelineEntries.entrySet()) { Map versionCopy = Maps.newHashMap(); versionCopy.putAll(versionEntry.getValue()); overShadowed.put(versionEntry.getKey(), versionCopy); } - for (Map.Entry> entry : completePartitionsTimeline.entrySet()) { + for (Map.Entry entry : completePartitionsTimeline.entrySet()) { Map versionEntry = overShadowed.get(entry.getValue().getTrueInterval()); if (versionEntry != null) { versionEntry.remove(entry.getValue().getVersion()); @@ -218,7 +213,7 @@ public class VersionedIntervalTimeline } } - for (Map.Entry> entry : incompletePartitionsTimeline.entrySet()) { + for (Map.Entry entry : incompletePartitionsTimeline.entrySet()) { Map versionEntry = overShadowed.get(entry.getValue().getTrueInterval()); if (versionEntry != null) { versionEntry.remove(entry.getValue().getVersion()); @@ -230,7 +225,7 @@ public class VersionedIntervalTimeline for (Map.Entry> versionEntry : overShadowed.entrySet()) { for (Map.Entry entry : versionEntry.getValue().entrySet()) { - TimelineEntry object = entry.getValue(); + TimelineEntry object = entry.getValue(); retVal.add( new TimelineObjectHolder( object.getTrueInterval(), @@ -249,12 +244,12 @@ public class VersionedIntervalTimeline } private void add( - NavigableMap> timeline, + NavigableMap timeline, Interval interval, - TimelineEntry entry + TimelineEntry entry ) { - TimelineEntry existsInTimeline = timeline.get(interval); + TimelineEntry existsInTimeline = timeline.get(interval); if (existsInTimeline != null) { int compare = versionComparator.compare(entry.getVersion(), existsInTimeline.getVersion()); @@ -284,9 +279,9 @@ public class VersionedIntervalTimeline } private boolean addAtKey( - NavigableMap> timeline, + NavigableMap timeline, Interval key, - TimelineEntry entry + TimelineEntry entry ) { boolean retVal = false; @@ -320,7 +315,7 @@ public class VersionedIntervalTimeline } } } else if (versionCompare > 0) { - TimelineEntry oldEntry = timeline.remove(currKey); + TimelineEntry oldEntry = timeline.remove(currKey); if (currKey.contains(entryInterval)) { addIntervalToTimeline(new Interval(currKey.getStart(), entryInterval.getStart()), oldEntry, timeline); @@ -360,8 +355,8 @@ public class VersionedIntervalTimeline private void addIntervalToTimeline( Interval interval, - TimelineEntry entry, - NavigableMap> timeline + TimelineEntry entry, + NavigableMap timeline ) { if (interval != null && interval.toDurationMillis() > 0) { @@ -370,9 +365,9 @@ public class VersionedIntervalTimeline } private void remove( - NavigableMap> timeline, + NavigableMap timeline, Interval interval, - TimelineEntry entry, + TimelineEntry entry, boolean incompleteOk ) { @@ -380,9 +375,9 @@ public class VersionedIntervalTimeline TimelineEntry removed = timeline.get(interval); if (removed == null) { - Iterator>> iter = timeline.entrySet().iterator(); + Iterator> iter = timeline.entrySet().iterator(); while (iter.hasNext()) { - Map.Entry> timelineEntry = iter.next(); + Map.Entry timelineEntry = iter.next(); if (timelineEntry.getValue() == entry) { intervalsToRemove.add(timelineEntry.getKey()); } @@ -397,15 +392,14 @@ public class VersionedIntervalTimeline } private void remove( - NavigableMap> timeline, + NavigableMap timeline, Interval interval, boolean incompleteOk ) { timeline.remove(interval); - for (Map.Entry>> versionEntry : allTimelineEntries - .entrySet()) { + for (Map.Entry> versionEntry : allTimelineEntries.entrySet()) { if (versionEntry.getKey().overlap(interval) != null) { TimelineEntry timelineEntry = versionEntry.getValue().lastEntry().getValue(); if (timelineEntry.getPartitionHolder().isComplete() || incompleteOk) { @@ -418,18 +412,20 @@ public class VersionedIntervalTimeline private List> lookup(Interval interval, boolean incompleteOk) { List> retVal = new ArrayList>(); - NavigableMap> timeline = (incompleteOk) - ? incompletePartitionsTimeline - : completePartitionsTimeline; + NavigableMap timeline = (incompleteOk) + ? incompletePartitionsTimeline + : completePartitionsTimeline; - for (Map.Entry> entry : timeline.entrySet()) { + for (Map.Entry entry : timeline.entrySet()) { Interval timelineInterval = entry.getKey(); - TimelineEntry val = entry.getValue(); + TimelineEntry val = entry.getValue(); if (timelineInterval.overlaps(interval)) { retVal.add( new TimelineObjectHolder( - timelineInterval, val.getVersion(), val.getPartitionHolder() + timelineInterval, + val.getVersion(), + val.getPartitionHolder() ) ); } @@ -467,30 +463,30 @@ public class VersionedIntervalTimeline return retVal; } - private static class TimelineEntry + public class TimelineEntry { private final Interval trueInterval; private final VersionType version; private final PartitionHolder partitionHolder; - private TimelineEntry(Interval trueInterval, VersionType version, PartitionHolder partitionHolder) + public TimelineEntry(Interval trueInterval, VersionType version, PartitionHolder partitionHolder) { this.trueInterval = trueInterval; this.version = version; this.partitionHolder = partitionHolder; } - private Interval getTrueInterval() + public Interval getTrueInterval() { return trueInterval; } - private VersionType getVersion() + public VersionType getVersion() { return version; } - private PartitionHolder getPartitionHolder() + public PartitionHolder getPartitionHolder() { return partitionHolder; } diff --git a/common/src/test/java/com/metamx/druid/VersionedIntervalTimelineTest.java b/common/src/test/java/com/metamx/druid/VersionedIntervalTimelineTest.java index 90fdee34e83..96d0fa01ce1 100644 --- a/common/src/test/java/com/metamx/druid/VersionedIntervalTimelineTest.java +++ b/common/src/test/java/com/metamx/druid/VersionedIntervalTimelineTest.java @@ -21,7 +21,6 @@ package com.metamx.druid; import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; -import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.druid.partition.ImmutablePartitionHolder; import com.metamx.druid.partition.IntegerPartitionChunk; @@ -29,7 +28,6 @@ import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.PartitionHolder; import com.metamx.druid.partition.SingleElementPartitionChunk; import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; import org.joda.time.Days; import org.joda.time.Hours; import org.joda.time.Interval; @@ -50,10 +48,6 @@ public class VersionedIntervalTimelineTest @Before public void setUp() throws Exception { - if (! "UTC".equals(DateTimeZone.getDefault().getID())) { - throw new ISE("Tests assume default timezone of UTC, please set -Duser.timezone=UTC"); - } - timeline = makeStringIntegerTimeline(); add("2011-04-01/2011-04-03", "1", 2); From 88a3fb842c0a7c4e5a64ad5d53f0a76e22ba5805 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 2 Nov 2012 14:24:43 -0500 Subject: [PATCH 04/19] 1) New attempt at making OpenJDK 6 happy --- .../main/java/com/metamx/druid/VersionedIntervalTimeline.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java b/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java index a45821c7dba..bee025c8270 100644 --- a/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java +++ b/common/src/main/java/com/metamx/druid/VersionedIntervalTimeline.java @@ -88,7 +88,7 @@ public class VersionedIntervalTimeline if (exists == null) { entry = new TimelineEntry(interval, version, new PartitionHolder(object)); - TreeMap versionEntry = Maps.newTreeMap(versionComparator); + TreeMap versionEntry = new TreeMap(versionComparator); versionEntry.put(version, entry); allTimelineEntries.put(interval, versionEntry); } else { From abab35194a10a6eb1d92f30c6fbaed9cd512690b Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 2 Nov 2012 15:56:18 -0700 Subject: [PATCH 05/19] general cleanup and improvments for auto scaling --- .../merger/coordinator/RemoteTaskRunner.java | 165 +++++++++++----- .../merger/coordinator/WorkerWrapper.java | 16 +- ...java => EC2AutoScalingStrategyConfig.java} | 15 +- .../config/RemoteTaskRunnerConfig.java | 8 + .../http/IndexerCoordinatorNode.java | 19 +- .../coordinator/scaling/AutoScalingData.java | 27 +++ .../scaling/EC2AutoScalingStrategy.java | 162 ++++++++++++++++ .../scaling/NoopScalingStrategy.java | 98 +--------- .../scaling/S3AutoScalingStrategy.java | 176 ------------------ .../coordinator/scaling/ScalingStrategy.java | 11 +- .../worker/WorkerCuratorCoordinator.java | 4 +- .../merger/worker/config/WorkerConfig.java | 7 +- ...t.java => EC2AutoScalingStrategyTest.java} | 43 +++-- 13 files changed, 388 insertions(+), 363 deletions(-) rename merger/src/main/java/com/metamx/druid/merger/coordinator/config/{S3AutoScalingStrategyConfig.java => EC2AutoScalingStrategyConfig.java} (78%) create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java rename merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/{S3AutoScalingStrategyTest.java => EC2AutoScalingStrategyTest.java} (78%) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 16d0ff2c6d8..3c08b397d77 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -19,9 +19,11 @@ package com.metamx.druid.merger.coordinator; +import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Predicate; import com.google.common.base.Throwables; +import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.primitives.Ints; import com.metamx.common.ISE; @@ -34,10 +36,12 @@ import com.metamx.druid.merger.common.TaskHolder; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; +import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; import com.metamx.druid.merger.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; +import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; @@ -47,8 +51,10 @@ import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; +import javax.annotation.Nullable; import java.io.IOException; import java.util.Comparator; +import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ScheduledExecutorService; @@ -76,13 +82,18 @@ public class RemoteTaskRunner implements TaskRunner private final ObjectMapper jsonMapper; private final RemoteTaskRunnerConfig config; private final CuratorFramework cf; - private final PathChildrenCache workerListener; + private final PathChildrenCache workerPathCache; private final ScheduledExecutorService scheduledExec; private final RetryPolicyFactory retryPolicyFactory; - private final ConcurrentHashMap zkWorkers; // all workers that exist in ZK - private final ConcurrentHashMap tasks; // all tasks that are assigned or need to be assigned private final ScalingStrategy strategy; + // all workers that exist in ZK + private final ConcurrentHashMap zkWorkers = new ConcurrentHashMap(); + // all tasks that are assigned or need to be assigned + private final ConcurrentHashMap tasks = new ConcurrentHashMap(); + + private final ConcurrentSkipListSet currentlyProvisioning = new ConcurrentSkipListSet(); + private final ConcurrentSkipListSet currentlyTerminating = new ConcurrentSkipListSet(); private final Object statusLock = new Object(); private volatile boolean started = false; @@ -91,22 +102,18 @@ public class RemoteTaskRunner implements TaskRunner ObjectMapper jsonMapper, RemoteTaskRunnerConfig config, CuratorFramework cf, - PathChildrenCache workerListener, + PathChildrenCache workerPathCache, ScheduledExecutorService scheduledExec, RetryPolicyFactory retryPolicyFactory, - ConcurrentHashMap zkWorkers, - ConcurrentHashMap tasks, ScalingStrategy strategy ) { this.jsonMapper = jsonMapper; this.config = config; this.cf = cf; - this.workerListener = workerListener; + this.workerPathCache = workerPathCache; this.scheduledExec = scheduledExec; this.retryPolicyFactory = retryPolicyFactory; - this.zkWorkers = zkWorkers; - this.tasks = tasks; this.strategy = strategy; } @@ -114,27 +121,23 @@ public class RemoteTaskRunner implements TaskRunner public void start() { try { - workerListener.start(); - workerListener.getListenable().addListener( + workerPathCache.start(); + workerPathCache.getListenable().addListener( new PathChildrenCacheListener() { @Override public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception { + final Worker worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final Worker worker = jsonMapper.readValue( - cf.getData().forPath(event.getData().getPath()), - Worker.class - ); - log.info("New worker[%s] found!", worker.getHost()); addWorker(worker); } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { - // Get the worker host from the path - String workerHost = event.getData().getPath().substring(event.getData().getPath().lastIndexOf("/") + 1); - - log.info("Worker[%s] removed!", workerHost); - removeWorker(workerHost); + log.info("Worker[%s] removed!", worker.getHost()); + removeWorker(worker.getHost()); } } } @@ -157,11 +160,51 @@ public class RemoteTaskRunner implements TaskRunner @Override public void run() { - strategy.terminateIfNeeded(zkWorkers); + if (currentlyTerminating.isEmpty()) { + if (zkWorkers.size() <= config.getMinNumWorkers()) { + return; + } + + List thoseLazyWorkers = Lists.newArrayList( + FunctionalIterable + .create(zkWorkers.values()) + .filter( + new Predicate() + { + @Override + public boolean apply(@Nullable WorkerWrapper input) + { + return System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() + > config.getmaxWorkerIdleTimeMillisBeforeDeletion(); + } + } + ) + ); + + AutoScalingData terminated = strategy.terminate( + Lists.transform( + thoseLazyWorkers, + new Function() + { + @Override + public String apply(@Nullable WorkerWrapper input) + { + return input.getWorker().getHost(); + } + } + ) + ); + + currentlyTerminating.addAll(terminated.getNodeIds()); + } else { + log.info( + "[%s] still terminating. Wait for all nodes to terminate before trying again.", + currentlyTerminating + ); + } } } ); - started = true; } catch (Exception e) { @@ -174,7 +217,7 @@ public class RemoteTaskRunner implements TaskRunner { try { for (WorkerWrapper workerWrapper : zkWorkers.values()) { - workerWrapper.getWatcher().close(); + workerWrapper.close(); } } catch (Exception e) { @@ -202,16 +245,24 @@ public class RemoteTaskRunner implements TaskRunner private boolean assignTask(TaskWrapper taskWrapper) { - // If the task already exists, we don't need to announce it try { - WorkerWrapper workerWrapper; - if ((workerWrapper = findWorkerRunningTask(taskWrapper)) != null) { + WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); + // If the task already exists, we don't need to announce it + if (workerWrapper != null) { final Worker worker = workerWrapper.getWorker(); - log.info("Worker[%s] is already running task{%s].", worker.getHost(), taskWrapper.getTask().getId()); + log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskWrapper.getTask().getId()); TaskStatus taskStatus = jsonMapper.readValue( - cf.getData().forPath(JOINER.join(config.getStatusPath(), worker.getHost(), taskWrapper.getTask().getId())), + workerWrapper.getStatusCache() + .getCurrentData( + JOINER.join( + config.getStatusPath(), + worker.getHost(), + taskWrapper.getTask().getId() + ) + ) + .getData(), TaskStatus.class ); @@ -222,7 +273,7 @@ public class RemoteTaskRunner implements TaskRunner } new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()).run(); } else { - tasks.put(taskWrapper.getTask().getId(), taskWrapper); + tasks.putIfAbsent(taskWrapper.getTask().getId(), taskWrapper); } return true; } @@ -301,19 +352,36 @@ public class RemoteTaskRunner implements TaskRunner private void addWorker(final Worker worker) { try { - final String workerStatus = JOINER.join(config.getStatusPath(), worker.getHost()); + currentlyProvisioning.remove(worker.getHost()); + + final String workerStatusPath = JOINER.join(config.getStatusPath(), worker.getHost()); + final PathChildrenCache statusCache = new PathChildrenCache(cf, workerStatusPath, true); final ConcurrentSkipListSet runningTasks = new ConcurrentSkipListSet( - cf.getChildren().forPath(workerStatus) + Lists.transform( + statusCache.getCurrentData(), + new Function() + { + @Override + public String apply(@Nullable ChildData input) + { + try { + return jsonMapper.readValue(input.getData(), TaskStatus.class).getId(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ) ); - final PathChildrenCache watcher = new PathChildrenCache(cf, workerStatus, false); final WorkerWrapper workerWrapper = new WorkerWrapper( worker, runningTasks, - watcher + statusCache ); // Add status listener to the watcher for status changes - watcher.getListenable().addListener( + statusCache.getListenable().addListener( new PathChildrenCacheListener() { @Override @@ -323,9 +391,8 @@ public class RemoteTaskRunner implements TaskRunner String taskId = null; try { if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - String statusPath = event.getData().getPath(); TaskStatus taskStatus = jsonMapper.readValue( - cf.getData().forPath(statusPath), TaskStatus.class + event.getData().getData(), TaskStatus.class ); taskId = taskStatus.getId(); @@ -335,7 +402,7 @@ public class RemoteTaskRunner implements TaskRunner } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { String statusPath = event.getData().getPath(); TaskStatus taskStatus = jsonMapper.readValue( - cf.getData().forPath(statusPath), TaskStatus.class + event.getData().getData(), TaskStatus.class ); taskId = taskStatus.getId(); @@ -369,7 +436,7 @@ public class RemoteTaskRunner implements TaskRunner } ); zkWorkers.put(worker.getHost(), workerWrapper); - watcher.start(); + statusCache.start(); } catch (Exception e) { throw Throwables.propagate(e); @@ -394,6 +461,8 @@ public class RemoteTaskRunner implements TaskRunner */ private void removeWorker(final String workerId) { + currentlyTerminating.remove(workerId); + WorkerWrapper workerWrapper = zkWorkers.get(workerId); if (workerWrapper != null) { for (String taskId : workerWrapper.getRunningTasks()) { @@ -405,7 +474,7 @@ public class RemoteTaskRunner implements TaskRunner } try { - workerWrapper.getWatcher().close(); + workerWrapper.getStatusCache().close(); } catch (IOException e) { log.error("Failed to close watcher associated with worker[%s]", workerWrapper.getWorker().getHost()); @@ -441,8 +510,19 @@ public class RemoteTaskRunner implements TaskRunner ); if (workerQueue.isEmpty()) { - log.makeAlert("There are no worker nodes with capacity to run task!").emit(); - strategy.provision(zkWorkers); + log.info("Worker nodes do not have capacity to run any more tasks!"); + + if (currentlyProvisioning.isEmpty()) { + AutoScalingData provisioned = strategy.provision(); + if (provisioned != null) { + currentlyProvisioning.addAll(provisioned.getNodeIds()); + } + } else { + log.info( + "[%s] still provisioning. Wait for all provisioned nodes to complete before requesting new worker.", + currentlyProvisioning + ); + } return null; } @@ -471,7 +551,6 @@ public class RemoteTaskRunner implements TaskRunner tasks.put(task.getId(), taskWrapper); cf.create() - .creatingParentsIfNeeded() .withMode(CreateMode.EPHEMERAL) .forPath( JOINER.join( diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index 047533ae858..83440b2500b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -23,6 +23,7 @@ import com.metamx.druid.merger.worker.Worker; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import org.joda.time.DateTime; +import java.io.IOException; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; @@ -32,15 +33,15 @@ public class WorkerWrapper { private final Worker worker; private final ConcurrentSkipListSet runningTasks; - private final PathChildrenCache watcher; + private final PathChildrenCache statusCache; private volatile DateTime lastCompletedTaskTime; - public WorkerWrapper(Worker worker, ConcurrentSkipListSet runningTasks, PathChildrenCache watcher) + public WorkerWrapper(Worker worker, ConcurrentSkipListSet runningTasks, PathChildrenCache statusCache) { this.worker = worker; this.runningTasks = runningTasks; - this.watcher = watcher; + this.statusCache = statusCache; } public Worker getWorker() @@ -53,9 +54,9 @@ public class WorkerWrapper return runningTasks; } - public PathChildrenCache getWatcher() + public PathChildrenCache getStatusCache() { - return watcher; + return statusCache; } public DateTime getLastCompletedTaskTime() @@ -77,4 +78,9 @@ public class WorkerWrapper { runningTasks.remove(taskId); } + + public void close() throws IOException + { + statusCache.close(); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/S3AutoScalingStrategyConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java similarity index 78% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/config/S3AutoScalingStrategyConfig.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java index 1aa2145cdc3..3f7b9a0171f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/S3AutoScalingStrategyConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java @@ -24,7 +24,7 @@ import org.skife.config.Default; /** */ -public abstract class S3AutoScalingStrategyConfig +public abstract class EC2AutoScalingStrategyConfig { @Config("druid.indexer.amiId") public abstract String getAmiId(); @@ -36,12 +36,11 @@ public abstract class S3AutoScalingStrategyConfig @Config("druid.indexer.instanceType") public abstract String getInstanceType(); - @Config("druid.indexer.millisToWaitBeforeTerminating") - @Default("1800000") // 30 mins - public abstract long getMillisToWaitBeforeTerminating(); - - // minimum number of workers that must always be running - @Config("druid.indexer.minNumWorkers") + @Config("druid.indexer.minNumInstancesToProvision") @Default("1") - public abstract int getMinNuMWorkers(); + public abstract int getMinNumInstancesToProvision(); + + @Config("druid.indexer.maxNumInstancesToProvision") + @Default("1") + public abstract int getMaxNumInstancesToProvision(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java index 3816468250f..a9cf6b02f93 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -37,4 +37,12 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig @Config("druid.indexer.minWorkerVersion") public abstract String getMinWorkerVersion(); + + @Config("druid.indexer.minNumWorkers") + @Default("1") + public abstract int getMinNumWorkers(); + + @Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion") + @Default("1") + public abstract int getmaxWorkerIdleTimeMillisBeforeDeletion(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 30d3b4fd53e..f907f2f5add 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -59,15 +59,13 @@ import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskRunner; import com.metamx.druid.merger.coordinator.TaskRunnerFactory; import com.metamx.druid.merger.coordinator.TaskStorage; -import com.metamx.druid.merger.coordinator.TaskWrapper; -import com.metamx.druid.merger.coordinator.WorkerWrapper; +import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; -import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; +import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.NoopScalingStrategy; -import com.metamx.druid.merger.coordinator.scaling.S3AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; import com.metamx.druid.realtime.S3SegmentPusher; import com.metamx.druid.realtime.S3SegmentPusherConfig; @@ -102,7 +100,6 @@ import org.skife.config.ConfigurationObjectFactory; import java.net.URL; import java.util.List; import java.util.Properties; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -464,27 +461,25 @@ public class IndexerCoordinatorNode .build() ); - ScalingStrategy strategy = new S3AutoScalingStrategy( + ScalingStrategy strategy = new EC2AutoScalingStrategy( new AmazonEC2Client( new BasicAWSCredentials( props.getProperty("com.metamx.aws.accessKey"), props.getProperty("com.metamx.aws.secretKey") ) ), - configFactory.build(S3AutoScalingStrategyConfig.class) + configFactory.build(EC2AutoScalingStrategyConfig.class) ); - // TODO: remove this when AMI is ready - strategy = new NoopScalingStrategy(configFactory.build(S3AutoScalingStrategyConfig.class)); + // TODO: use real strategy before actual deployment + strategy = new NoopScalingStrategy(); return new RemoteTaskRunner( jsonMapper, configFactory.build(RemoteTaskRunnerConfig.class), curatorFramework, - new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), false), + new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true), retryScheduledExec, new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)), - new ConcurrentHashMap(), - new ConcurrentHashMap(), strategy ); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java new file mode 100644 index 00000000000..6cce08f8731 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java @@ -0,0 +1,27 @@ +package com.metamx.druid.merger.coordinator.scaling; + +import java.util.List; + +/** + */ +public class AutoScalingData +{ + private final List nodeIds; + private final List nodes; + + public AutoScalingData(List nodeIds, List nodes) + { + this.nodeIds = nodeIds; + this.nodes = nodes; + } + + public List getNodeIds() + { + return nodeIds; + } + + public List getNodes() + { + return nodes; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java new file mode 100644 index 00000000000..cd94b70d3ce --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -0,0 +1,162 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.merger.coordinator.scaling; + +import com.amazonaws.services.ec2.AmazonEC2Client; +import com.amazonaws.services.ec2.model.DescribeInstancesRequest; +import com.amazonaws.services.ec2.model.DescribeInstancesResult; +import com.amazonaws.services.ec2.model.Filter; +import com.amazonaws.services.ec2.model.Instance; +import com.amazonaws.services.ec2.model.InstanceType; +import com.amazonaws.services.ec2.model.Reservation; +import com.amazonaws.services.ec2.model.RunInstancesRequest; +import com.amazonaws.services.ec2.model.RunInstancesResult; +import com.amazonaws.services.ec2.model.TerminateInstancesRequest; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; +import com.metamx.emitter.EmittingLogger; + +import javax.annotation.Nullable; +import java.util.List; + +/** + */ +public class EC2AutoScalingStrategy implements ScalingStrategy +{ + private static final EmittingLogger log = new EmittingLogger(EC2AutoScalingStrategy.class); + + private final AmazonEC2Client amazonEC2Client; + private final EC2AutoScalingStrategyConfig config; + + public EC2AutoScalingStrategy( + AmazonEC2Client amazonEC2Client, + EC2AutoScalingStrategyConfig config + ) + { + this.amazonEC2Client = amazonEC2Client; + this.config = config; + } + + @Override + public AutoScalingData provision() + { + try { + log.info("Creating new instance(s)..."); + RunInstancesResult result = amazonEC2Client.runInstances( + new RunInstancesRequest( + config.getAmiId(), + config.getMinNumInstancesToProvision(), + config.getMaxNumInstancesToProvision() + ) + .withInstanceType(InstanceType.fromValue(config.getInstanceType())) + ); + + List instanceIds = Lists.transform( + result.getReservation().getInstances(), + new Function() + { + @Override + public String apply(@Nullable Instance input) + { + return input.getInstanceId(); + } + } + ); + + log.info("Created instances: %s", instanceIds); + + return new AutoScalingData( + Lists.transform( + result.getReservation().getInstances(), + new Function() + { + @Override + public String apply(@Nullable Instance input) + { + return String.format("%s:%s", input.getPrivateIpAddress(), config.getWorkerPort()); + } + } + ), + result.getReservation().getInstances() + ); + } + catch (Exception e) { + log.error(e, "Unable to provision any EC2 instances."); + } + + return null; + } + + @Override + public AutoScalingData terminate(List nodeIds) + { + DescribeInstancesResult result = amazonEC2Client.describeInstances( + new DescribeInstancesRequest() + .withFilters( + new Filter("private-ip-address", nodeIds) + ) + ); + + List instances = Lists.newArrayList(); + for (Reservation reservation : result.getReservations()) { + instances.addAll(reservation.getInstances()); + } + + try { + log.info("Terminating instance[%s]", instances); + amazonEC2Client.terminateInstances( + new TerminateInstancesRequest( + Lists.transform( + instances, + new Function() + { + @Override + public String apply(@Nullable Instance input) + { + return input.getInstanceId(); + } + } + ) + ) + ); + + return new AutoScalingData( + Lists.transform( + instances, + new Function() + { + @Override + public String apply(@Nullable Instance input) + { + return String.format("%s:%s", input.getPrivateIpAddress(), config.getWorkerPort()); + } + } + ), + instances + ); + } + catch (Exception e) { + log.error(e, "Unable to terminate any instances."); + } + + return null; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java index 81f9550a24e..0331022082c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java @@ -1,112 +1,34 @@ package com.metamx.druid.merger.coordinator.scaling; import com.amazonaws.services.ec2.model.Instance; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; import com.google.common.collect.MinMaxPriorityQueue; -import com.google.common.collect.Ordering; -import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.merger.coordinator.WorkerWrapper; -import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; +import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; import java.util.Comparator; +import java.util.List; import java.util.Map; /** * This class just logs when scaling should occur. */ -public class NoopScalingStrategy implements ScalingStrategy +public class NoopScalingStrategy implements ScalingStrategy { private static final EmittingLogger log = new EmittingLogger(NoopScalingStrategy.class); - private final S3AutoScalingStrategyConfig config; - - private final Object lock = new Object(); - - private volatile String currentlyProvisioning = null; - private volatile String currentlyTerminating = null; - - public NoopScalingStrategy( - S3AutoScalingStrategyConfig config - ) + @Override + public AutoScalingData provision() { - this.config = config; + log.info("If I were a real strategy I'd create something now"); + return null; } @Override - public void provision(Map zkWorkers) + public AutoScalingData terminate(List nodeIds) { - synchronized (lock) { - if (currentlyProvisioning != null) { - if (!zkWorkers.containsKey(currentlyProvisioning)) { - log.info( - "[%s] is still provisioning. Wait for it to finish before requesting new worker.", - currentlyProvisioning - ); - return; - } - } - - try { - log.info("If I were a real strategy I'd create something now"); - currentlyProvisioning = "willNeverBeTrue"; - } - catch (Exception e) { - log.error(e, "Unable to create instance"); - currentlyProvisioning = null; - } - } - } - - @Override - public Instance terminateIfNeeded(Map zkWorkers) - { - synchronized (lock) { - if (currentlyTerminating != null) { - if (zkWorkers.containsKey(currentlyTerminating)) { - log.info("[%s] has not terminated. Wait for it to finish before terminating again.", currentlyTerminating); - return null; - } - } - - MinMaxPriorityQueue currWorkers = MinMaxPriorityQueue.orderedBy( - new Comparator() - { - @Override - public int compare(WorkerWrapper w1, WorkerWrapper w2) - { - DateTime w1Time = (w1 == null) ? new DateTime(0) : w1.getLastCompletedTaskTime(); - DateTime w2Time = (w2 == null) ? new DateTime(0) : w2.getLastCompletedTaskTime(); - return w1Time.compareTo(w2Time); - } - } - ).create( - zkWorkers.values() - ); - - if (currWorkers.size() <= config.getMinNuMWorkers()) { - return null; - } - - WorkerWrapper thatLazyWorker = currWorkers.poll(); - - if (System.currentTimeMillis() - thatLazyWorker.getLastCompletedTaskTime().getMillis() - > config.getMillisToWaitBeforeTerminating()) { - try { - log.info("If I were a real strategy I'd terminate something now"); - currentlyTerminating = "willNeverBeTrue"; - - return null; - } - catch (Exception e) { - log.error(e, "Unable to terminate instance"); - currentlyTerminating = null; - } - } - - return null; - } + log.info("If I were a real strategy I'd terminate %s now", nodeIds); + return null; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java deleted file mode 100644 index 840dba3a2a8..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java +++ /dev/null @@ -1,176 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.merger.coordinator.scaling; - -import com.amazonaws.services.ec2.AmazonEC2Client; -import com.amazonaws.services.ec2.model.DescribeInstancesRequest; -import com.amazonaws.services.ec2.model.DescribeInstancesResult; -import com.amazonaws.services.ec2.model.Filter; -import com.amazonaws.services.ec2.model.Instance; -import com.amazonaws.services.ec2.model.InstanceType; -import com.amazonaws.services.ec2.model.RunInstancesRequest; -import com.amazonaws.services.ec2.model.RunInstancesResult; -import com.amazonaws.services.ec2.model.TerminateInstancesRequest; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; -import com.google.common.collect.MinMaxPriorityQueue; -import com.metamx.common.ISE; -import com.metamx.common.guava.FunctionalIterable; -import com.metamx.druid.merger.coordinator.WorkerWrapper; -import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; -import com.metamx.emitter.EmittingLogger; -import org.joda.time.DateTime; - -import java.util.Arrays; -import java.util.Comparator; -import java.util.Map; - -/** - */ -public class S3AutoScalingStrategy implements ScalingStrategy -{ - private static final EmittingLogger log = new EmittingLogger(S3AutoScalingStrategy.class); - - private final AmazonEC2Client amazonEC2Client; - private final S3AutoScalingStrategyConfig config; - - private final Object lock = new Object(); - - private volatile String currentlyProvisioning = null; - private volatile String currentlyTerminating = null; - - public S3AutoScalingStrategy( - AmazonEC2Client amazonEC2Client, - S3AutoScalingStrategyConfig config - ) - { - this.amazonEC2Client = amazonEC2Client; - this.config = config; - } - - @Override - public void provision(Map zkWorkers) - { - synchronized (lock) { - if (zkWorkers.containsKey(currentlyProvisioning)) { - currentlyProvisioning = null; - } - - if (currentlyProvisioning != null) { - log.info( - "[%s] is still provisioning. Wait for it to finish before requesting new worker.", - currentlyProvisioning - ); - return; - } - - try { - log.info("Creating a new instance"); - RunInstancesResult result = amazonEC2Client.runInstances( - new RunInstancesRequest(config.getAmiId(), 1, 1) - .withInstanceType(InstanceType.fromValue(config.getInstanceType())) - ); - - if (result.getReservation().getInstances().size() != 1) { - throw new ISE("Created more than one instance, WTF?!"); - } - - Instance instance = result.getReservation().getInstances().get(0); - log.info("Created instance: %s", instance.getInstanceId()); - log.debug("%s", instance); - - currentlyProvisioning = String.format("%s:%s", instance.getPrivateIpAddress(), config.getWorkerPort()); - } - catch (Exception e) { - log.error(e, "Unable to create instance"); - currentlyProvisioning = null; - } - } - } - - @Override - public Instance terminateIfNeeded(Map zkWorkers) - { - synchronized (lock) { - if (!zkWorkers.containsKey(currentlyTerminating)) { - currentlyProvisioning = null; - } - - if (currentlyTerminating != null) { - log.info("[%s] has not terminated. Wait for it to finish before terminating again.", currentlyTerminating); - return null; - } - - MinMaxPriorityQueue currWorkers = MinMaxPriorityQueue.orderedBy( - new Comparator() - { - @Override - public int compare(WorkerWrapper w1, WorkerWrapper w2) - { - DateTime w1Time = (w1 == null) ? new DateTime(0) : w1.getLastCompletedTaskTime(); - DateTime w2Time = (w2 == null) ? new DateTime(0) : w2.getLastCompletedTaskTime(); - return w1Time.compareTo(w2Time); - } - } - ).create( - zkWorkers.values() - ); - - if (currWorkers.size() <= config.getMinNuMWorkers()) { - return null; - } - - WorkerWrapper thatLazyWorker = currWorkers.poll(); - - if (System.currentTimeMillis() - thatLazyWorker.getLastCompletedTaskTime().getMillis() - > config.getMillisToWaitBeforeTerminating()) { - DescribeInstancesResult result = amazonEC2Client.describeInstances( - new DescribeInstancesRequest() - .withFilters( - new Filter("private-ip-address", Arrays.asList(thatLazyWorker.getWorker().getIp())) - ) - ); - - if (result.getReservations().size() != 1 || result.getReservations().get(0).getInstances().size() != 1) { - throw new ISE("More than one node with the same private IP[%s], WTF?!", thatLazyWorker.getWorker().getIp()); - } - - Instance instance = result.getReservations().get(0).getInstances().get(0); - - try { - log.info("Terminating instance[%s]", instance.getInstanceId()); - amazonEC2Client.terminateInstances( - new TerminateInstancesRequest(Arrays.asList(instance.getInstanceId())) - ); - - currentlyTerminating = String.format("%s:%s", instance.getPrivateIpAddress(), config.getWorkerPort()); - - return instance; - } - catch (Exception e) { - log.error(e, "Unable to terminate instance"); - currentlyTerminating = null; - } - } - - return null; - } - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java index 6a779927b87..9b7da8fb3a4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java @@ -19,16 +19,13 @@ package com.metamx.druid.merger.coordinator.scaling; -import com.amazonaws.services.ec2.model.Instance; -import com.metamx.druid.merger.coordinator.WorkerWrapper; - -import java.util.Map; +import java.util.List; /** */ -public interface ScalingStrategy +public interface ScalingStrategy { - public void provision(Map zkWorkers); + public AutoScalingData provision(); - public Instance terminateIfNeeded(Map zkWorkers); + public AutoScalingData terminate(List nodeIds); } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java index 9375ef81696..82dbd75adf8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java @@ -145,7 +145,6 @@ public class WorkerCuratorCoordinator if (curatorFramework.checkExists().forPath(path) == null) { try { curatorFramework.create() - .creatingParentsIfNeeded() .withMode(mode) .forPath(path, jsonMapper.writeValueAsBytes(data)); } @@ -201,7 +200,7 @@ public class WorkerCuratorCoordinator curatorFramework.delete().guaranteed().forPath(getTaskPathForId(taskId)); } catch (Exception e) { - log.warn("Could not delete task path for task[%s], looks like it already went away", taskId); + log.warn(e, "Could not delete task path for task[%s]", taskId); } } @@ -214,7 +213,6 @@ public class WorkerCuratorCoordinator try { curatorFramework.create() - .creatingParentsIfNeeded() .withMode(CreateMode.EPHEMERAL) .forPath( getStatusPathForId(status.getId()), diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java index e8e68749e2d..5b5f3a0a6e7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java @@ -39,7 +39,8 @@ public abstract class WorkerConfig @Config("druid.worker.version") public abstract String getVersion(); - @Config("druid.worker.capacity") - @Default("3") - public abstract int getCapacity(); + public int getCapacity() + { + return Runtime.getRuntime().availableProcessors() - 1; + } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java similarity index 78% rename from merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java rename to merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java index de486cbee57..f5d682ba781 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -29,7 +29,7 @@ import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.google.common.collect.Maps; import com.metamx.druid.merger.coordinator.WorkerWrapper; -import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; +import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.worker.Worker; import org.easymock.EasyMock; import org.joda.time.DateTime; @@ -45,10 +45,11 @@ import java.util.concurrent.ConcurrentSkipListSet; /** */ -public class S3AutoScalingStrategyTest +public class EC2AutoScalingStrategyTest { private static final String AMI_ID = "dummy"; private static final String INSTANCE_ID = "theInstance"; + private static final String IP = "dummyIP"; private AmazonEC2Client amazonEC2Client; private RunInstancesResult runInstancesResult; @@ -56,7 +57,7 @@ public class S3AutoScalingStrategyTest private Reservation reservation; private Instance instance; private WorkerWrapper worker; - private S3AutoScalingStrategy strategy; + private EC2AutoScalingStrategy strategy; @Before public void setUp() throws Exception @@ -66,16 +67,20 @@ public class S3AutoScalingStrategyTest describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class); reservation = EasyMock.createMock(Reservation.class); - instance = new Instance().withInstanceId(INSTANCE_ID).withLaunchTime(new Date()).withImageId(AMI_ID); + instance = new Instance() + .withInstanceId(INSTANCE_ID) + .withLaunchTime(new Date()) + .withImageId(AMI_ID) + .withPrivateIpAddress(IP); worker = new WorkerWrapper( - new Worker("dummyHost", "dummyIP", 2, "0"), + new Worker("dummyHost", IP, 2, "0"), new ConcurrentSkipListSet(), null ); worker.setLastCompletedTaskTime(new DateTime(0)); - strategy = new S3AutoScalingStrategy( - amazonEC2Client, new S3AutoScalingStrategyConfig() + strategy = new EC2AutoScalingStrategy( + amazonEC2Client, new EC2AutoScalingStrategyConfig() { @Override public String getAmiId() @@ -96,15 +101,15 @@ public class S3AutoScalingStrategyTest } @Override - public long getMillisToWaitBeforeTerminating() + public int getMinNumInstancesToProvision() { - return 0; + return 1; } @Override - public int getMinNuMWorkers() + public int getMaxNumInstancesToProvision() { - return 0; + return 1; } } ); @@ -140,10 +145,6 @@ public class S3AutoScalingStrategyTest EasyMock.expect(reservation.getInstances()).andReturn(Arrays.asList(instance)).atLeastOnce(); EasyMock.replay(reservation); - Map zkWorkers = Maps.newHashMap(); - - zkWorkers.put(worker.getWorker().getHost(), worker); - worker.getRunningTasks().add("task1"); Assert.assertFalse(worker.isAtCapacity()); @@ -152,13 +153,19 @@ public class S3AutoScalingStrategyTest Assert.assertTrue(worker.isAtCapacity()); - strategy.provision(zkWorkers); + AutoScalingData created = strategy.provision(); + + Assert.assertEquals(created.getNodeIds().size(), 1); + Assert.assertEquals(created.getNodes().size(), 1); + Assert.assertEquals(String.format("%s:8080", IP), created.getNodeIds().get(0)); worker.getRunningTasks().remove("task1"); worker.getRunningTasks().remove("task2"); - Instance deleted = strategy.terminateIfNeeded(zkWorkers); + AutoScalingData deleted = strategy.terminate(Arrays.asList("dummyHost")); - Assert.assertEquals(deleted.getInstanceId(), INSTANCE_ID); + Assert.assertEquals(deleted.getNodeIds().size(), 1); + Assert.assertEquals(deleted.getNodes().size(), 1); + Assert.assertEquals(String.format("%s:8080", IP), deleted.getNodeIds().get(0)); } } From a5c3eab2f1ce6dd09e97c208fe100db6973d85e1 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 2 Nov 2012 15:59:55 -0700 Subject: [PATCH 06/19] fix merge break --- client/src/main/java/com/metamx/druid/http/BrokerNode.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/http/BrokerNode.java b/client/src/main/java/com/metamx/druid/http/BrokerNode.java index 665a2fbd427..b8f1f5c4d31 100644 --- a/client/src/main/java/com/metamx/druid/http/BrokerNode.java +++ b/client/src/main/java/com/metamx/druid/http/BrokerNode.java @@ -221,7 +221,7 @@ public class BrokerNode extends BaseNode final ServiceDiscoveryConfig serviceDiscoveryConfig = getConfigFactory().build(ServiceDiscoveryConfig.class); CuratorFramework curatorFramework = Initialization.makeCuratorFrameworkClient( - serviceDiscoveryConfig.getZkHosts(), lifecycle + serviceDiscoveryConfig, lifecycle ); final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( From e9dadcae490a296f296aff80484f5ef33489dbb8 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 2 Nov 2012 16:59:49 -0700 Subject: [PATCH 07/19] more sensical logic around when to terminate worker nodes --- .../merger/coordinator/RemoteTaskRunner.java | 20 ++++++++++++------- .../config/RemoteTaskRunnerConfig.java | 14 +++++++------ 2 files changed, 21 insertions(+), 13 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 3c08b397d77..7e890385703 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -144,17 +144,17 @@ public class RemoteTaskRunner implements TaskRunner ); // Schedule termination of worker nodes periodically - Period period = new Period(config.getTerminateResourcesPeriodMs()); - PeriodGranularity granularity = new PeriodGranularity(period, null, null); - final long truncatedNow = granularity.truncate(new DateTime().getMillis()); + Period period = new Period(config.getTerminateResourcesDuration()); + PeriodGranularity granularity = new PeriodGranularity(period, config.getTerminateResourcesOriginDateTime(), null); + final long startTime = granularity.next(granularity.truncate(new DateTime().getMillis())); ScheduledExecutors.scheduleAtFixedRate( scheduledExec, new Duration( System.currentTimeMillis(), - granularity.next(truncatedNow) - config.getTerminateResourcesWindowMs() + startTime ), - new Duration(config.getTerminateResourcesPeriodMs()), + config.getTerminateResourcesDuration(), new Runnable() { @Override @@ -588,12 +588,18 @@ public class RemoteTaskRunner implements TaskRunner { try { final String statusPath = JOINER.join(config.getStatusPath(), workerId, taskId); - final String taskPath = JOINER.join(config.getTaskPath(), workerId, taskId); cf.delete().guaranteed().forPath(statusPath); + } + catch (Exception e) { + log.warn("Tried to delete a status path that didn't exist! Must've gone away already?"); + } + + try { + final String taskPath = JOINER.join(config.getTaskPath(), workerId, taskId); cf.delete().guaranteed().forPath(taskPath); } catch (Exception e) { - log.warn("Tried to delete a path that didn't exist! Must've gone away already!"); + log.warn("Tried to delete a task path that didn't exist! Must've gone away already?"); } } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java index a9cf6b02f93..5118e6e7c59 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -20,6 +20,8 @@ package com.metamx.druid.merger.coordinator.config; import com.metamx.druid.merger.common.config.IndexerZkConfig; +import org.joda.time.DateTime; +import org.joda.time.Duration; import org.skife.config.Config; import org.skife.config.Default; @@ -27,13 +29,13 @@ import org.skife.config.Default; */ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig { - @Config("druid.indexer.terminateResources.periodMs") - @Default("3600000") // 1 hr - public abstract long getTerminateResourcesPeriodMs(); + @Config("druid.indexer.terminateResources.duration") + @Default("PT1H") + public abstract Duration getTerminateResourcesDuration(); - @Config("druid.indexer.terminateResources.windowMs") - @Default("300000") // 5 mins - public abstract long getTerminateResourcesWindowMs(); + @Config("druid.indexer.terminateResources.originDateTime") + @Default("2012-01-01T00:55:00.000Z") + public abstract DateTime getTerminateResourcesOriginDateTime(); @Config("druid.indexer.minWorkerVersion") public abstract String getMinWorkerVersion(); From b4e3f98f4b539f26debc2be5f49d5b5d87d04d09 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 5 Nov 2012 11:18:11 -0800 Subject: [PATCH 08/19] bug fixes for retries without sufficient capacity --- .../merger/coordinator/RemoteTaskRunner.java | 63 +++++++++---------- .../druid/merger/coordinator/RetryPolicy.java | 5 -- .../merger/coordinator/WorkerWrapper.java | 2 +- 3 files changed, 29 insertions(+), 41 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 7e890385703..77cbc0d2f5e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -26,7 +26,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.primitives.Ints; -import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; @@ -174,6 +173,9 @@ public class RemoteTaskRunner implements TaskRunner @Override public boolean apply(@Nullable WorkerWrapper input) { + if (!input.getRunningTasks().isEmpty()) { + return false; + } return System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() > config.getmaxWorkerIdleTimeMillisBeforeDeletion(); } @@ -243,13 +245,15 @@ public class RemoteTaskRunner implements TaskRunner ); } - private boolean assignTask(TaskWrapper taskWrapper) + private void assignTask(TaskWrapper taskWrapper) { - try { - WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); - // If the task already exists, we don't need to announce it - if (workerWrapper != null) { - final Worker worker = workerWrapper.getWorker(); + tasks.putIfAbsent(taskWrapper.getTask().getId(), taskWrapper); + WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); + + // If the task already exists, we don't need to announce it + if (workerWrapper != null) { + final Worker worker = workerWrapper.getWorker(); + try { log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskWrapper.getTask().getId()); @@ -272,24 +276,20 @@ public class RemoteTaskRunner implements TaskRunner callback.notify(taskStatus); } new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()).run(); - } else { - tasks.putIfAbsent(taskWrapper.getTask().getId(), taskWrapper); } - return true; + } + catch (Exception e) { + log.error(e, "Task exists, but hit exception!"); + retryTask(new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()), taskWrapper); + } + } else { // Announce the task + workerWrapper = getWorkerForTask(); + if (workerWrapper != null) { + announceTask(workerWrapper.getWorker(), taskWrapper); + } else { + retryTask(null, taskWrapper); } } - catch (Exception e) { - throw Throwables.propagate(e); - } - - // Announce the task - WorkerWrapper workerWrapper = getWorkerForTask(); - if (workerWrapper != null) { - announceTask(workerWrapper.getWorker(), taskWrapper); - return true; - } - - return false; } /** @@ -320,20 +320,13 @@ public class RemoteTaskRunner implements TaskRunner @Override public void run() { - try { - if (pre != null) { - pre.run(); - } - - if (tasks.containsKey(task.getId())) { - log.info("Retry[%d] for task[%s]", retryPolicy.getNumRetries(), task.getId()); - if (!assignTask(taskWrapper)) { - throw new ISE("Unable to find worker to send retry request to for task[%s]", task.getId()); - } - } + if (pre != null) { + pre.run(); } - catch (Exception e) { - retryTask(null, taskWrapper); + + if (tasks.containsKey(task.getId())) { + log.info("Retry[%d] for task[%s]", retryPolicy.getNumRetries(), task.getId()); + assignTask(taskWrapper); } } }, diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java index b449ce01960..24ee54290bb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java @@ -19,12 +19,9 @@ package com.metamx.druid.merger.coordinator; -import com.google.common.collect.Lists; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.emitter.EmittingLogger; -import java.util.List; - /** */ public class RetryPolicy @@ -34,8 +31,6 @@ public class RetryPolicy private final long MAX_NUM_RETRIES; private final long MAX_RETRY_DELAY_MILLIS; - private final List runnables = Lists.newArrayList(); - private volatile long currRetryDelay; private volatile int retryCount; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index 83440b2500b..99e330c0a3a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -35,7 +35,7 @@ public class WorkerWrapper private final ConcurrentSkipListSet runningTasks; private final PathChildrenCache statusCache; - private volatile DateTime lastCompletedTaskTime; + private volatile DateTime lastCompletedTaskTime = new DateTime(); public WorkerWrapper(Worker worker, ConcurrentSkipListSet runningTasks, PathChildrenCache statusCache) { From 6b247f1599260309a3741afb12a3b470c6b7a212 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 5 Nov 2012 11:19:55 -0800 Subject: [PATCH 09/19] cleanup redundant code --- .../metamx/druid/merger/coordinator/RemoteTaskRunner.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 77cbc0d2f5e..84f71360404 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -173,11 +173,9 @@ public class RemoteTaskRunner implements TaskRunner @Override public boolean apply(@Nullable WorkerWrapper input) { - if (!input.getRunningTasks().isEmpty()) { - return false; - } - return System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() - > config.getmaxWorkerIdleTimeMillisBeforeDeletion(); + return input.getRunningTasks().isEmpty() + && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() + > config.getmaxWorkerIdleTimeMillisBeforeDeletion(); } } ) From 51cd361fbe20b73f5122244ea68fb80a41fb421f Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 5 Nov 2012 13:39:16 -0800 Subject: [PATCH 10/19] some changes to how handlers are registered --- client/src/main/java/com/metamx/druid/BaseNode.java | 4 ++-- .../src/main/java/com/metamx/druid/index/v1/IndexIO.java | 2 -- .../{ComplexMetricRegistererer.java => Registererer.java} | 4 ++-- .../merger/coordinator/http/IndexerCoordinatorNode.java | 7 +++++++ .../com/metamx/druid/merger/worker/http/WorkerNode.java | 7 +++++++ 5 files changed, 18 insertions(+), 6 deletions(-) rename index-common/src/main/java/com/metamx/druid/index/v1/serde/{ComplexMetricRegistererer.java => Registererer.java} (88%) diff --git a/client/src/main/java/com/metamx/druid/BaseNode.java b/client/src/main/java/com/metamx/druid/BaseNode.java index 7e75ea38347..dc45cd6eef8 100644 --- a/client/src/main/java/com/metamx/druid/BaseNode.java +++ b/client/src/main/java/com/metamx/druid/BaseNode.java @@ -30,7 +30,7 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.http.RequestLogger; -import com.metamx.druid.index.v1.serde.ComplexMetricRegistererer; +import com.metamx.druid.index.v1.serde.Registererer; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ZkClientConfig; @@ -174,7 +174,7 @@ public abstract class BaseNode } @SuppressWarnings("unchecked") - public T registerComplexMetric(ComplexMetricRegistererer registererer) + public T registerHandler(Registererer registererer) { registererer.register(); return (T) this; diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index 75352a25137..719b2350bbf 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -36,12 +36,10 @@ import org.joda.time.Interval; import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.FileChannel; import java.util.Map; /** diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricRegistererer.java b/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java similarity index 88% rename from index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricRegistererer.java rename to index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java index c0dcade7ed9..f560dfdc1e6 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricRegistererer.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java @@ -20,14 +20,14 @@ package com.metamx.druid.index.v1.serde; /** - * This is a "factory" interface for registering complex metrics in the system. It exists because I'm unaware of + * This is a "factory" interface for registering handlers in the system. It exists because I'm unaware of * another way to register the complex serdes in the MR jobs that run on Hadoop. As such, instances of this interface * must be instantiatable via a no argument default constructor (the MR jobs on Hadoop use reflection to instantiate * instances). * * The name is not a typo, I felt that it needed an extra "er" to make the pronunciation that much more difficult. */ -public interface ComplexMetricRegistererer +public interface Registererer { public void register(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index f907f2f5add..2b689c71706 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -41,6 +41,7 @@ import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.index.v1.serde.Registererer; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; @@ -182,6 +183,12 @@ public class IndexerCoordinatorNode this.taskRunnerFactory = taskRunnerFactory; } + public IndexerCoordinatorNode registerHandler(Registererer registererer) + { + registererer.register(); + return this; + } + public void init() throws Exception { scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 67e1e2e089f..210a55bcf5f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -29,6 +29,7 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.index.v1.serde.Registererer; import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; @@ -147,6 +148,12 @@ public class WorkerNode return this; } + public WorkerNode registerHandler(Registererer registererer) + { + registererer.register(); + return this; + } + public void init() throws Exception { initializeEmitter(); From 0a42f18cc3ed40015fbe84f4157782f31c320ac9 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 5 Nov 2012 14:33:15 -0800 Subject: [PATCH 11/19] HistogramVisual: add support for quantiles --- .../main/java/com/metamx/druid/aggregation/Histogram.java | 2 +- .../java/com/metamx/druid/aggregation/HistogramVisual.java | 7 +++++++ .../java/com/metamx/druid/histogram/HistogramTest.java | 1 + 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java b/common/src/main/java/com/metamx/druid/aggregation/Histogram.java index 8f081ca7ade..7199d098d6a 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java +++ b/common/src/main/java/com/metamx/druid/aggregation/Histogram.java @@ -135,7 +135,7 @@ public class Histogram public HistogramVisual asVisual() { float[] visualCounts = new float[bins.length - 2]; for(int i = 0; i < visualCounts.length; ++i) visualCounts[i] = (float)bins[i + 1]; - return new HistogramVisual(breaks, visualCounts, min, max); + return new HistogramVisual(breaks, visualCounts, new float[]{}, min, max); } public static Histogram fromBytes(byte[] bytes) { diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java b/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java index 5ce5217e1b2..4a88e652c7b 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java +++ b/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java @@ -29,6 +29,7 @@ public class HistogramVisual { @JsonProperty final public double[] breaks; @JsonProperty final public double[] counts; + @JsonProperty final public double[] quantiles; @JsonProperty final double min; @JsonProperty final double max; @@ -36,6 +37,7 @@ public class HistogramVisual public HistogramVisual( @JsonProperty double[] breaks, @JsonProperty double[] counts, + @JsonProperty double[] quantiles, @JsonProperty double min, @JsonProperty double max ) @@ -46,6 +48,7 @@ public class HistogramVisual this.breaks = breaks; this.counts = counts; + this.quantiles = quantiles; this.min = min; this.max = max; } @@ -53,6 +56,7 @@ public class HistogramVisual public HistogramVisual( float[] breaks, float[] counts, + float[] quantiles, float min, float max ) @@ -63,8 +67,10 @@ public class HistogramVisual this.breaks = new double[breaks.length]; this.counts = new double[counts.length]; + this.quantiles = new double[quantiles.length]; for(int i = 0; i < breaks.length; ++i) this.breaks[i] = breaks[i]; for(int i = 0; i < counts.length; ++i) this.counts[i] = counts[i]; + for(int i = 0; i < quantiles.length; ++i) this.quantiles[i] = quantiles[i]; this.min = min; this.max = max; } @@ -76,6 +82,7 @@ public class HistogramVisual "counts=" + Arrays.toString(counts) + ", breaks=" + Arrays.toString(breaks) + ", min=" + min + + ", quantiles=" + Arrays.toString(quantiles) + ", max=" + max + '}'; } diff --git a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java index e40cecaa203..7b764365f91 100644 --- a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java +++ b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java @@ -68,6 +68,7 @@ public class HistogramTest Map expectedObj = Maps.newLinkedHashMap(); expectedObj.put("breaks", Arrays.asList(visualBreaks)); expectedObj.put("counts", Arrays.asList(visualCounts)); + expectedObj.put("quantiles", Arrays.asList(new Double[]{})); expectedObj.put("min", -1.0); expectedObj.put("max", 1.0); From 86d2fb3550d4923521f8545e1e6126359749ccaf Mon Sep 17 00:00:00 2001 From: Paul Baclace Date: Tue, 6 Nov 2012 00:16:27 -0800 Subject: [PATCH 12/19] added DruidSetup.java and install/druid_setup.sh to run it for Druid ensemble setup (put properties to zk, create zk zpaths, prep db, and dump properties/paths in zk); property druid.zk.paths.base can establish a namespace for a druid ensemble allowing more than one ensemble to use the same zookeepers; this namespace is the base zpath prefix for paths with properties in the form druid.zk.paths.*Path which are normally set automatically now; if druid.zk.paths.*Path are explicitly set, then all of them must be set with absolute paths to ensure careful attention; ZkSetup is now deprecated, use DruidSetup instead; examples/twitter RealtimeStandaloneMain will exit properly now after a kill -15 (control-c or kill pid) instead of hanging on an unstopped daemon. --- .gitignore | 1 + README | 7 +- .../druid/initialization/Initialization.java | 204 ++++++++- .../druid/zk/PropertiesZkSerializer.java | 69 +++ druid-services/pom.xml | 71 +++ examples/rand/run_server.sh | 2 +- .../src/main/resources/runtime.properties | 60 ++- .../examples/RealtimeStandaloneMain.java | 16 +- .../src/main/resources/runtime.properties | 57 ++- examples/twitter/twitter_realtime.spec | 2 +- install/druid_setup.sh | 25 ++ install/log4j.xml | 18 + pom.xml | 1 + .../com/metamx/druid/utils/DruidSetup.java | 412 ++++++++++++++++++ .../java/com/metamx/druid/utils/ZkSetup.java | 7 +- 15 files changed, 908 insertions(+), 44 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/zk/PropertiesZkSerializer.java create mode 100644 druid-services/pom.xml create mode 100755 install/druid_setup.sh create mode 100644 install/log4j.xml create mode 100644 server/src/main/java/com/metamx/druid/utils/DruidSetup.java diff --git a/.gitignore b/.gitignore index 15e9d97a8f9..74b30509e3d 100644 --- a/.gitignore +++ b/.gitignore @@ -14,3 +14,4 @@ target client/.settings/org.eclipse.jdt.core.prefs common/.settings/org.eclipse.jdt.core.prefs server/.settings/org.eclipse.jdt.core.prefs +*.log diff --git a/README b/README index e39398352b4..cb1ec84a8ab 100644 --- a/README +++ b/README @@ -1 +1,6 @@ -See the "Wiki":https://github.com/metamx/druid/wiki \ No newline at end of file +See the "Wiki" https://github.com/metamx/druid/wiki + +Build with build.sh + +See examples/rand +See examples/twitter diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index f14cfecebf8..6b6f1402d02 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -29,6 +29,7 @@ import com.metamx.druid.client.ZKPhoneBook; import com.metamx.druid.http.FileRequestLogger; import com.metamx.druid.http.RequestLogger; import com.metamx.druid.zk.StringZkSerializer; +import com.metamx.druid.zk.PropertiesZkSerializer; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.CuratorFrameworkFactory; import com.netflix.curator.retry.ExponentialBackoffRetry; @@ -57,7 +58,17 @@ public class Initialization { private static final Logger log = new Logger(Initialization.class); - private static volatile Properties props = null; + private static final Properties zkProps = new Properties(); + private static final Properties fileProps = new Properties(zkProps); + private static Properties props = null; + public final static String PROP_SUBPATH = "properties"; + public final static String[] SUB_PATHS = {"announcements", "servedSegments", "loadQueue", "master"}; + public final static String[] SUB_PATH_PROPS = { + "druid.zk.paths.announcementsPath", + "druid.zk.paths.servedSegmentsPath", + "druid.zk.paths.loadQueuePath", + "druid.zk.paths.masterPath"}; + public static final String DEFAULT_ZPATH = "/druid"; public static ZkClient makeZkClient(ZkClientConfig config, Lifecycle lifecycle) { @@ -105,23 +116,33 @@ public class Initialization ); } - public static Properties loadProperties() + + /** Load properties. + * Properties are layered, high to low precedence: cmdLine -D, runtime.properties file, stored in zookeeper. + * Idempotent. Thread-safe. Properties are only loaded once. + * If property druid.zk.service.host=none then do not load properties from zookeeper. + * @return Properties ready to use. + */ + public synchronized static Properties loadProperties() { if (props != null) { return props; } - Properties loadedProps = null; + // Note that zookeeper coordinates must be either in cmdLine or in runtime.properties + Properties sp = System.getProperties(); + + Properties tmp_props = new Properties(fileProps); // the head of the 3 level Properties chain + tmp_props.putAll(sp); + final InputStream stream = ClassLoader.getSystemResourceAsStream("runtime.properties"); if (stream == null) { - log.info("runtime.properties didn't exist as a resource, loading system properties instead."); - loadedProps = System.getProperties(); + log.info("runtime.properties not found as a resource in classpath, relying only on system properties, and zookeeper now."); } else { - log.info("Loading properties from runtime.properties."); + log.info("Loading properties from runtime.properties"); try { - loadedProps = new Properties(); try { - loadedProps.load(stream); + fileProps.load(stream); } catch (IOException e) { throw Throwables.propagate(e); @@ -132,13 +153,60 @@ public class Initialization } } - for (String prop : loadedProps.stringPropertyNames()) { - log.info("Loaded Property[%s] as [%s]", prop, loadedProps.getProperty(prop)); + // log properties from file; note stringPropertyNames() will follow Properties.defaults but + // next level is empty at this point. + for (String prop : fileProps.stringPropertyNames()) { + log.info("Loaded(runtime.properties) Property[%s] as [%s]", prop, fileProps.getProperty(prop)); } - props = loadedProps; + final String zk_hosts = tmp_props.getProperty("druid.zk.service.host"); - return loadedProps; + if (zk_hosts != null) { + if (!zk_hosts.equals("none")) { // get props from zk + final ZkClient zkPropLoadingClient; + final ZkClientConfig clientConfig = new ZkClientConfig() + { + @Override + public String getZkHosts() + { + return zk_hosts; + } + }; + + zkPropLoadingClient = new ZkClient( + new ZkConnection(clientConfig.getZkHosts()), + clientConfig.getConnectionTimeout(), + new PropertiesZkSerializer() + ); + zkPropLoadingClient.waitUntilConnected(); + String propertiesZNodePath = tmp_props.getProperty("druid.zk.paths.propertiesPath"); + if (propertiesZNodePath == null) { + String zpathBase = tmp_props.getProperty("druid.zk.paths.base", DEFAULT_ZPATH); + propertiesZNodePath = makePropPath(zpathBase); + } + // get properties stored by zookeeper (lowest precedence) + if (zkPropLoadingClient.exists(propertiesZNodePath)) { + Properties p = zkPropLoadingClient.readData(propertiesZNodePath, true); + if (p != null) { + zkProps.putAll(p); + } + } + // log properties from zk + for (String prop : zkProps.stringPropertyNames()) { + log.info("Loaded(properties stored in zk) Property[%s] as [%s]", prop, zkProps.getProperty(prop)); + } + } // get props from zk + } else { // ToDo: should this be an error? + log.warn("property druid.zk.service.host is not set, so no way to contact zookeeper for coordination."); + } + // validate properties now that all levels of precedence are loaded + if (! validateResolveProps(tmp_props)) { + log.error("Properties failed to validate, cannot continue"); + throw new RuntimeException("Properties failed to validate"); + } + props = tmp_props; // publish + + return props; } public static Server makeJettyServer(ServerConfig config) @@ -281,4 +349,116 @@ public class Initialization return new FileRequestLogger(exec, new File(loggingDir)); } + + public static String makePropPath(String basePath) + { + return String.format("%s/%s", basePath, PROP_SUBPATH); + } + + /** Validate and Resolve Properties. + * Resolve zpaths with props like druid.zk.paths.*Path using druid.zk.paths.base value. + * Check validity so that if druid.zk.paths.*Path props are set, all are set, + * if none set, then construct defaults relative to druid.zk.paths.base and add these + * to the properties chain. + * @param props + * @return true if valid zpath properties. + */ + public static boolean validateResolveProps(Properties props) + { + boolean zpathValidateFailed;// validate druid.zk.paths.base + String propertyZpath = props.getProperty("druid.zk.paths.base"); + zpathValidateFailed = zpathBaseCheck(propertyZpath, "property druid.zk.paths.base"); + + String zpathEffective = DEFAULT_ZPATH; + if (propertyZpath != null) { + zpathEffective = propertyZpath; + } + + final String propertiesZpathOverride = props.getProperty("druid.zk.paths.propertiesPath"); + + if (!zpathValidateFailed) System.out.println("Effective zpath prefix=" + zpathEffective); + + // validate druid.zk.paths.*Path properties + // + // if any zpath overrides are set in properties, all must be set, and they must start with / + int zpathOverrideCount = 0; + boolean zpathOverridesNotAbs = false; + StringBuilder sbErrors = new StringBuilder(100); + for (int i = 0; i < SUB_PATH_PROPS.length; i++) { + String val = props.getProperty(SUB_PATH_PROPS[i]); + if (val != null) { + zpathOverrideCount++; + if (!val.startsWith("/")) { + zpathOverridesNotAbs = true; + sbErrors.append(SUB_PATH_PROPS[i]).append("=").append(val).append("\n"); + zpathValidateFailed = true; + } + } + } + // separately check druid.zk.paths.propertiesPath (not in SUB_PATH_PROPS since it is not a "dir") + if (propertiesZpathOverride != null) { + zpathOverrideCount++; + if (!propertiesZpathOverride.startsWith("/")) { + zpathOverridesNotAbs = true; + sbErrors.append("druid.zk.paths.propertiesPath").append("=").append(propertiesZpathOverride).append("\n"); + zpathValidateFailed = true; + } + } + if (zpathOverridesNotAbs) { + System.err.println("When overriding zk zpaths, with properties like druid.zk.paths.*Path " + + "the znode path must start with '/' (slash) ; problem overrides:"); + System.err.print(sbErrors.toString()); + } + if (zpathOverrideCount > 0) { + if (zpathOverrideCount < SUB_PATH_PROPS.length + 1) { + zpathValidateFailed = true; + System.err.println("When overriding zk zpaths, with properties of form druid.zk.paths.*Path " + + "all must be overridden together; missing overrides:"); + for (int i = 0; i < SUB_PATH_PROPS.length; i++) { + String val = props.getProperty(SUB_PATH_PROPS[i]); + if (val == null) { + System.err.println(" " + SUB_PATH_PROPS[i]); + } + } + if (propertiesZpathOverride == null) System.err.println(" " + "druid.zk.paths.propertiesPath"); + } else { // proper overrides + // do not prefix with property druid.zk.paths.base + ; // fallthru + } + } else { // no overrides + if (propertyZpath == null) { // if default base is used, store it as documentation + props.setProperty("druid.zk.paths.base", zpathEffective); + } + // + // Resolve default zpaths using zpathEffective as base + // + for (int i = 0; i < SUB_PATH_PROPS.length; i++) { + props.setProperty(SUB_PATH_PROPS[i], zpathEffective + "/" + SUB_PATHS[i]); + } + props.setProperty("druid.zk.paths.propertiesPath", zpathEffective + "/properties"); + } + return ! zpathValidateFailed; + } + + /** Check znode zpath base for proper slash, no trailing slash. + * @param zpathBase znode base path, if null then this method does nothing. + * @param errorMsgPrefix error context to use if errors are emitted, should indicate + * where the zpathBase value came from. + * @return true if validate failed. + */ + public static boolean zpathBaseCheck(String zpathBase, String errorMsgPrefix) + { + boolean zpathValidateFailed = false; + if (zpathBase != null) { + if (!zpathBase.startsWith("/")) { + zpathValidateFailed = true; + System.err.println(errorMsgPrefix + " must start with '/' (slash); found=" + zpathBase); + } + if (zpathBase.endsWith("/")) { + zpathValidateFailed = true; + System.err.println(errorMsgPrefix + " must NOT end with '/' (slash); found=" + zpathBase); + } + } + return zpathValidateFailed; + } } diff --git a/client/src/main/java/com/metamx/druid/zk/PropertiesZkSerializer.java b/client/src/main/java/com/metamx/druid/zk/PropertiesZkSerializer.java new file mode 100644 index 00000000000..036a8dc5055 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/zk/PropertiesZkSerializer.java @@ -0,0 +1,69 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.zk; + +import com.metamx.common.IAE; +import org.I0Itec.zkclient.exception.ZkMarshallingError; +import org.I0Itec.zkclient.serialize.ZkSerializer; + +import java.io.*; +import java.nio.charset.Charset; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Properties; +import java.util.TimeZone; + +/** +*/ +public class PropertiesZkSerializer implements ZkSerializer +{ + private static final SimpleDateFormat df = new SimpleDateFormat("yyyyMMdd'T'HHmmss'z'"); + static { + df.setTimeZone(TimeZone.getTimeZone("UTC")); + } + public final static String META_PROP = "__MODIFIED"; + + @Override + public byte[] serialize(Object data) throws ZkMarshallingError + { + if (data instanceof Properties) { + final Properties props = (Properties) data; + ByteArrayOutputStream bos = new ByteArrayOutputStream(props.size()*60 + 30); + try { + final String ts = df.format(new Date()); + props.setProperty("__MODIFIED", ts); + props.store(bos, "Druid"); + } catch (IOException ignored) { } + return bos.toByteArray(); + } + throw new IAE("Can only serialize Properties into ZK"); + } + + @Override + public Object deserialize(byte[] bytes) throws ZkMarshallingError + { + final Properties props = new Properties(); + try { + props.load(new ByteArrayInputStream(bytes)); + } catch (IOException ignored) { + } + return props; + } +} diff --git a/druid-services/pom.xml b/druid-services/pom.xml new file mode 100644 index 00000000000..75d476bf748 --- /dev/null +++ b/druid-services/pom.xml @@ -0,0 +1,71 @@ + + + + 4.0.0 + + com.metamx.druid + druid-services + druid-services + druid-services + 0.1.0-SNAPSHOT + + com.metamx + druid + 0.1.0-SNAPSHOT + + + + + com.metamx.druid + druid-realtime + ${project.parent.version} + + + com.metamx.druid + druid-server + ${project.parent.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.6 + + + package + + shade + + + + ${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar + + + + + + + + \ No newline at end of file diff --git a/examples/rand/run_server.sh b/examples/rand/run_server.sh index 9273500da61..2789064653f 100755 --- a/examples/rand/run_server.sh +++ b/examples/rand/run_server.sh @@ -15,7 +15,7 @@ echo "cleaning up previous run, if any" # start RealtimeNode process # -java -Xmx400m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath target/druid-examples-rand-2.5.72-SNAPSHOT-selfcontained.jar druid.examples.RealtimeStandaloneMain >RealtimeNode.out 2>&1 & +java -Xmx400m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath target/druid-examples-rand-*-selfcontained.jar druid.examples.RealtimeStandaloneMain >RealtimeNode.out 2>&1 & PID=$! trap "kill $PID" 1 2 3 15 diff --git a/examples/rand/src/main/resources/runtime.properties b/examples/rand/src/main/resources/runtime.properties index 3aafdda906a..03197ad6751 100644 --- a/examples/rand/src/main/resources/runtime.properties +++ b/examples/rand/src/main/resources/runtime.properties @@ -1,6 +1,6 @@ # Properties for demo of Realtime Node in standalone mode. -# To Use This: copy this file to runtime.properties and put directory containing it in classpath. # +comment.origin=druid/examples/rand/src/main/resources/runtime.properties # S3 access com.metamx.aws.accessKey= @@ -15,7 +15,10 @@ druid.database.user=user druid.database.password=password # time between polling for metadata database druid.database.poll.duration=PT1M + +# table for segment metadata coordination, no default druid.database.segmentTable=prod_segments + #in progress 20121010 #druid.database.taskTable= druid.emitter.period=PT60S @@ -42,23 +45,52 @@ druid.server.maxSize=300000000000 # =realtime or =historical (default) druid.server.type=realtime -# ZK path for service discovery within the cluster -druid.zk.paths.announcementsPath=/druid/announcementsPath +# +# zookeeper (zk) znode paths (zpaths) +# -# Legacy path, must be set, but can be ignored -druid.zk.paths.indexesPath=/druid/indexesPath +# base znode which establishes a unique namespace for a Druid ensemble. +# Default is /druid if not set +# This can also be set via parameter baseZkPath of the DruidSetup commandline +# druid.zk.paths.base= -druid.zk.paths.indexer.tasksPath=/druid/tasksPath -druid.zk.paths.indexer.statusPath=/druid/statusPath +# If these zpath properties like druid.zk.paths.*Path are overridden, then all must be +# overridden together for upgrade safety reasons. +# The commandline utility DruidSetup, which is used to set up properties on zookeeper, +# will validate this. Also, these zpaths must start with / because they are not relative. + +# ZK znode path for service discovery within the cluster. +# Default is value of druid.zk.paths.base + /announcements +# druid.zk.paths.announcementsPath=/druid/announcements + +# Legacy znode path, must be set, but can be ignored +#druid.zk.paths.indexesPath=/druid/indexes + +# Default is value of druid.zk.paths.base + /tasks +##druid.zk.paths.indexer.tasksPath=/druid/tasks + +# Default is value of druid.zk.paths.base + /status +#druid.zk.paths.indexer.statusPath=/druid/status # ZK path for load/drop protocol between Master/Compute -druid.zk.paths.loadQueuePath=/druid/loadQueuePath +# Default is value of druid.zk.paths.base + /loadQueue +#druid.zk.paths.loadQueuePath=/druid/loadQueue # ZK path for Master leadership election -druid.zk.paths.masterPath=/druid/masterPath +# Default is value of druid.zk.paths.base + /master +#druid.zk.paths.masterPath=/druid/master # ZK path for publishing served segments -druid.zk.paths.servedSegmentsPath=/druid/servedSegmentsPath +# Default is value of druid.zk.paths.base + /servedSegments +#druid.zk.paths.servedSegmentsPath=/druid/servedSegments + +# Default is value of druid.zk.paths.base + /leaderLatch +#druid.zk.paths.indexer.leaderLatchPath=/druid/leaderLatch + +# ZK path for properties stored in zookeeper +# Default is value of druid.zk.paths.base + /properties +#druid.zk.paths.propertiesPath=/druid/properties + druid.host=127.0.0.1 druid.port=8080 @@ -72,8 +104,10 @@ com.metamx.emitter.logging=true com.metamx.emitter.logging.level=info com.metamx.metrics.emitter.period=PT60S -# ZK quorum IPs; if empty, the use demo mode -druid.zk.service.host= +# ZK quorum IPs; ZK coordinates in the form host1:port1[,host2:port2[, ...]] +# if =none then do not contact zookeeper (only for RealtimeStandaloneMain examples) +druid.zk.service.host=none + # msec; high value means tolerate slow zk nodes, default is to wait about 3 weeks druid.zk.service.connectionTimeout=1000000 @@ -85,7 +119,7 @@ druid.processing.numThreads=3 # other properties found # druid.computation.buffer.size=10000000 -druid.zk.paths.indexer.leaderLatchPath=/druid/leaderLatchPath + druid.merger.threads=1 druid.merger.runner=remote druid.merger.whitelist.enabled=false diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index 6fabdf04452..32f355c756b 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -32,7 +32,7 @@ public class RealtimeStandaloneMain { LogLevelAdjuster.register(); - Lifecycle lifecycle = new Lifecycle(); + final Lifecycle lifecycle = new Lifecycle(); RealtimeNode rn = RealtimeNode.builder().build(); lifecycle.addManagedInstance(rn); @@ -86,6 +86,20 @@ public class RealtimeStandaloneMain } ); + Runtime.getRuntime().addShutdownHook( + new Thread( + new Runnable() + { + @Override + public void run() + { + log.info("Running shutdown hook"); + lifecycle.stop(); + } + } + ) + ); + try { lifecycle.start(); } diff --git a/examples/twitter/src/main/resources/runtime.properties b/examples/twitter/src/main/resources/runtime.properties index 3f7fdeaec50..b504f2513de 100644 --- a/examples/twitter/src/main/resources/runtime.properties +++ b/examples/twitter/src/main/resources/runtime.properties @@ -1,6 +1,7 @@ # Properties for demo of Realtime Node in standalone mode. # To Use This: copy this file to runtime.properties and put directory containing it in classpath. # +comment.origin=druid/examples/twitter/src/main/resources/runtime.properties # S3 access com.metamx.aws.accessKey= @@ -15,7 +16,10 @@ druid.database.user=user druid.database.password=password # time between polling for metadata database druid.database.poll.duration=PT1M + +# table for segment metadata coordination, no default druid.database.segmentTable=prod_segments + #in progress 20121010 #druid.database.taskTable= druid.emitter.period=PT60S @@ -47,23 +51,51 @@ druid.server.maxSize=300000000000 # =realtime or =historical (default) druid.server.type=realtime -# ZK path for service discovery within the cluster -druid.zk.paths.announcementsPath=/druid/announcementsPath +# +# zookeeper (zk) znode paths (zpaths) +# -# Legacy path, must be set, but can be ignored -druid.zk.paths.indexesPath=/druid/indexesPath +# base znode which establishes a unique namespace for a Druid ensemble. +# Default is /druid if not set +# This can also be set via parameter baseZkPath of the DruidSetup commandline +# druid.zk.paths.base= -druid.zk.paths.indexer.tasksPath=/druid/tasksPath -druid.zk.paths.indexer.statusPath=/druid/statusPath +# If these zpath properties like druid.zk.paths.*Path are overridden, then all must be +# overridden together for upgrade safety reasons. +# The commandline utility DruidSetup, which is used to set up properties on zookeeper, +# will validate this. Also, these zpaths must start with / because they are not relative. + +# ZK znode path for service discovery within the cluster. +# Default is value of druid.zk.paths.base + /announcements +# druid.zk.paths.announcementsPath=/druid/announcements + +# Legacy znode path, must be set, but can be ignored +#druid.zk.paths.indexesPath=/druid/indexes + +# Default is value of druid.zk.paths.base + /tasks +##druid.zk.paths.indexer.tasksPath=/druid/tasks + +# Default is value of druid.zk.paths.base + /status +#druid.zk.paths.indexer.statusPath=/druid/status # ZK path for load/drop protocol between Master/Compute -druid.zk.paths.loadQueuePath=/druid/loadQueuePath +# Default is value of druid.zk.paths.base + /loadQueue +#druid.zk.paths.loadQueuePath=/druid/loadQueue # ZK path for Master leadership election -druid.zk.paths.masterPath=/druid/masterPath +# Default is value of druid.zk.paths.base + /master +#druid.zk.paths.masterPath=/druid/master # ZK path for publishing served segments -druid.zk.paths.servedSegmentsPath=/druid/servedSegmentsPath +# Default is value of druid.zk.paths.base + /servedSegments +#druid.zk.paths.servedSegmentsPath=/druid/servedSegments + +# Default is value of druid.zk.paths.base + /leaderLatch +#druid.zk.paths.indexer.leaderLatchPath=/druid/leaderLatch + +# ZK path for properties stored in zookeeper +# Default is value of druid.zk.paths.base + /properties +#druid.zk.paths.propertiesPath=/druid/properties druid.host=127.0.0.1 druid.port=8080 @@ -79,8 +111,10 @@ com.metamx.emitter.http=true # unknown # com.metamx.emitter.logging.level=info # unknown # com.metamx.metrics.emitter.period=PT60S -# ZK quorum IPs; if empty, the use demo mode -druid.zk.service.host= +# ZK quorum IPs; ZK coordinates in the form host1:port1[,host2:port2[, ...]] +# if =none then do not contact zookeeper (only for RealtimeStandaloneMain examples) +druid.zk.service.host=none + # msec; high value means tolerate slow zk nodes, default is to wait about 3 weeks druid.zk.service.connectionTimeout=1000000 @@ -92,7 +126,6 @@ druid.processing.numThreads=3 # other properties found # druid.computation.buffer.size=10000000 -druid.zk.paths.indexer.leaderLatchPath=/druid/leaderLatchPath druid.merger.threads=1 druid.merger.runner=remote druid.merger.whitelist.enabled=false diff --git a/examples/twitter/twitter_realtime.spec b/examples/twitter/twitter_realtime.spec index 8ffe014822b..3381b075060 100644 --- a/examples/twitter/twitter_realtime.spec +++ b/examples/twitter/twitter_realtime.spec @@ -3,7 +3,7 @@ "aggregators":[ {"type":"count", "name":"tweets"}, {"type":"doubleSum","fieldName":"follower_count","name":"total_follower_count"}, - {"type":"doubleSum","fieldName":"retweet_count","name":"tota_retweet_count"}, + {"type":"doubleSum","fieldName":"retweet_count","name":"total_retweet_count"}, {"type":"doubleSum","fieldName":"friends_count","name":"total_friends_count"}, {"type":"doubleSum","fieldName":"statuses_count","name":"total_statuses_count"}, diff --git a/install/druid_setup.sh b/install/druid_setup.sh new file mode 100755 index 00000000000..2ab0067eb43 --- /dev/null +++ b/install/druid_setup.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash +# Script to run util DruidSetup which will initialize zookeeper locations, properties, and metadata store (MySQL or similar). +# The dump cmd of DruidSetup will dump properties stored at and zpaths of zookeeper. +# Run with no args to get usage. + +which java >/dev/null +WJ=$? +if [ "${JAVA_HOME}" ]; then + RUN_JAVA=$JAVA_HOME/bin/java +elif [ $WJ -eq 0 ]; then + RUN_JAVA=java +fi + +[ -z "${RUN_JAVA}" ] && echo "env var JAVA_HOME is not defined and java not in path" && exit 1 + +DRUID_DIR=$(cd $(dirname $0)/.. ; pwd) + +DRUID_SERVER_JAR="$(ls -1 $(find $DRUID_DIR -name 'druid-server*selfcontained.jar') |head -1)" +[ -z "${DRUID_SERVER_JAR}" ] && echo "unable to find druid server jar" && exit 2 +echo "using ${DRUID_SERVER_JAR}" +echo + +$RUN_JAVA -cp "${DRUID_SERVER_JAR}" -Dlog4j.configuration=file://${DRUID_DIR}/install/log4j.xml -Djava.net.preferIPv4Stack=true -Duser.timezone=UTC -Dfile.encoding=UTF-8 com.metamx.druid.utils.DruidSetup $* + +[ -e ${DRUID_DIR}/install/druid_setup.log ] && egrep "WARN|ERROR|FATAL" ${DRUID_DIR}/install/druid_setup.log diff --git a/install/log4j.xml b/install/log4j.xml new file mode 100644 index 00000000000..aa6eb8cc13e --- /dev/null +++ b/install/log4j.xml @@ -0,0 +1,18 @@ + + + + + + + + + + + + + + + + + + diff --git a/pom.xml b/pom.xml index f1ebad70e01..7e260f5afa5 100644 --- a/pom.xml +++ b/pom.xml @@ -45,6 +45,7 @@ merger realtime examples + druid-services diff --git a/server/src/main/java/com/metamx/druid/utils/DruidSetup.java b/server/src/main/java/com/metamx/druid/utils/DruidSetup.java new file mode 100644 index 00000000000..9b96e1d88a1 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/utils/DruidSetup.java @@ -0,0 +1,412 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.utils; + +import com.google.common.io.Closeables; +import com.metamx.common.logger.Logger; +import com.metamx.druid.db.DbConnector; +import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.zk.PropertiesZkSerializer; +import org.I0Itec.zkclient.ZkClient; +import org.I0Itec.zkclient.ZkConnection; + +import java.io.*; +import java.util.Properties; + +/** + * Set up the shared Druid ensemble space. + * This affects the Zookeeper which holds common properties, and znode paths for coordination, + * and also performs metadata table creation in the database (MySQL). + * By storing ensemble-wide properties in zookeeper, cluster administration is simplified. + * Each service instance can also have local property overrides in the file runtime.properties + * located in the classpath. + *

+ * The design rules are noted here with rationale + *

+ *

+ *

+ * Design Rule Notes:
+ * (a) Properties set on the commandline of services take precedence over  runtime.properties which
+ *       takes precedence over properties stored in zookeeper.
+ *
+ *       Rationale:  organizing principle.
+ *
+ * (a) Services load properties on startup only.
+ *
+ *       Rationale: stepwise changes are safer and easier to manage.
+ *
+ * (b) Only DruidSetup creates properties and znode paths (zpaths) on zookeeper and no other tool or service
+ *       will make ensemble-wide settings automatically.
+ *
+ *       Rationale: one place for this logic, under manual control, and avoid accidental
+ *       namespace/partition creation.
+ *
+ * (c) DruidSetup creates reasonable zpaths but supports overrides to enable tactical
+ *   version transitions (just in case).  If zpaths are overridden, then they must all be
+ *   overridden together since they are not independent.
+ *
+ *       Rationale:  convention beats configuration most of the time; sometimes configuration is needed
+ *       negotiate unusual cases.
+ *
+ * (d) Properties settings stored on zookeeper are not cumulative; previous properties are removed before
+ *   new ones are stored.
+ *       Rationale:  Keep the operations at the granularity of a file of properties, avoid
+ *       dependence on order of setup operations, enable dumping of current settings.
+ * 
+ * + * @author pbaclace + */ +public class DruidSetup +{ + private static final Logger log = new Logger(DruidSetup.class); + + public static void main(final String[] args) + { + ZkClient zkClient = null; + + if (args.length < 2 || args.length > 3) { + printUsage(); + System.exit(1); + } + String cmd = args[0]; + if ("dump".equals(cmd) && args.length == 3) { + final String zkConnect = args[1]; + zkClient = connectToZK(zkConnect); + String zpathBase = args[2]; + dumpFromZk(zkClient, zpathBase, zkConnect, System.out); + } else if ("put".equals(cmd) && args.length == 3) { + final String zkConnect = args[1]; + zkClient = connectToZK(zkConnect); + final String pfile = args[2]; + putToZk(zkClient, pfile); + } else if ("dbprep".equals(cmd) && args.length == 2) { + final String pfile = args[1]; + prepDB(pfile); + } else { + printUsage(); + System.exit(1); + } + + if (zkClient != null) { + zkClient.close(); + } + } + + /** + * Load properties from local file, validate and tweak. + *

+ * This can only be used for setup, not service run time because of some assembly here. + * + * @param pfile path to runtime.properties file to be read. + * @param props Properties object to fill, props like druid.zk.paths.*Path will always be set after + * this method either because the input file has them set (overrides) or because prop + * druid.zk.paths.base was used as a prefix to construct the default zpaths; + * druid.zk.paths.base will be set iff there is a single base for all zpaths + */ + private static void loadProperties(String pfile, Properties props) + { + InputStream is = null; + try { + is = new FileInputStream(pfile); + } catch (FileNotFoundException e) { + System.err.println("File not found: " + pfile); + System.err.println("No changes made."); + System.exit(4); + } catch (IOException ioe) { + reportErrorAndExit(pfile, ioe); + } + try { + props.load(is); + } catch (IOException e) { + reportErrorAndExit(pfile, e); + } finally { + Closeables.closeQuietly(is); + } + + if (! Initialization.validateResolveProps(props)) { // bail, errors have been emitted + System.exit(9); + } + + // emit effective zpaths to be used + System.out.println("Effective zpath properties:"); + for (String pname : Initialization.SUB_PATH_PROPS) { + System.out.println(" " + pname + "=" + props.getProperty(pname)); + } + System.out.println(" " + "druid.zk.paths.propertiesPath" + "=" + + props.getProperty("druid.zk.paths.propertiesPath")); + + } + + /** + * @param zkClient zookeeper client. + * @param zpathBase znode base path. + * @param zkConnect ZK coordinates in the form host1:port1[,host2:port2[, ...]] + * @param out + */ + private static void dumpFromZk(ZkClient zkClient, String zpathBase, String zkConnect, PrintStream out) + { + final String propPath = Initialization.makePropPath(zpathBase); + if (zkClient.exists(propPath)) { + Properties currProps = zkClient.readData(propPath, true); + if (currProps != null) { + out.println("# Begin Properties Listing for zkConnect=" + zkConnect + " zpath=" + propPath); + try { + currProps.store(out, "Druid"); + } catch (IOException ignored) { + } + out.println("# End Properties Listing for zkConnect=" + zkConnect + " zpath=" + propPath); + out.println("# NOTE: properties like druid.zk.paths.*Path are always stored in zookeeper in absolute form."); + out.println(); + } + } + out.println("Zookeeper znodes and zpaths for " + zkConnect + " (showing all zpaths)"); + // list all znodes + // (not ideal since recursive listing starts at / instead of at baseZkPath) + zkClient.showFolders(out); + } + + /** + * @param zkClient zookeeper client. + * @param pfile + */ + private static void putToZk(ZkClient zkClient, String pfile) + { + Properties props = new Properties(); + loadProperties(pfile, props); + String zpathBase = props.getProperty("druid.zk.paths.base"); + + // create znodes first + // + createZNodes(zkClient, zpathBase, System.out); + + // put props + // + updatePropertiesZK(zkClient, zpathBase, props, System.out); + } + + /** + * @param zkClient zookeeper client. + * @param zpathBase znode base path. + * @param props the properties to store. + * @param out the PrintStream for human readable update summary (usually System.out). + */ + private static void updatePropertiesZK(ZkClient zkClient, String zpathBase, Properties props, PrintStream out) + { + final String propPathOverride = props.getProperty("druid.zk.paths.propertiesPath"); + final String propPathConstructed = Initialization.makePropPath(zpathBase); + final String propPath = (propPathOverride != null) ? propPathOverride : propPathConstructed; + Properties currProps = null; + if (zkClient.exists(propPath)) { + currProps = zkClient.readData(propPath, true); + } + boolean propsDiffer = false; + if (currProps == null) { + out.println("No properties currently stored in zk"); + propsDiffer = true; + } else { // determine whether anything is different + int countNew = 0; + int countDiffer = 0; + int countRemoved = 0; + int countNoChange = 0; + String currMetaPropVal = ""; + StringBuilder changes = new StringBuilder(1024); + for (String pname : props.stringPropertyNames()) { + if (pname.equals(PropertiesZkSerializer.META_PROP)) continue; // ignore meta prop datestamp, if any + final String pvalue = props.getProperty(pname); + final String pvalueCurr = currProps.getProperty(pname); + if (pvalueCurr == null) { + countNew++; + } else { + if (pvalueCurr.equals(pvalue)) { + countNoChange++; + } else { + countDiffer++; + changes.append("CHANGED: ").append(pname).append("= PREV=").append(pvalueCurr) + .append(" NOW=").append(pvalue).append("\n"); + } + } + } + for (String pname : currProps.stringPropertyNames()) { + if (pname.equals(PropertiesZkSerializer.META_PROP)) { + currMetaPropVal = currProps.getProperty(pname); + continue; // ignore meta prop datestamp + } + if (props.getProperty(pname) == null) { + countRemoved++; + changes.append("REMOVED: ").append(pname).append("=").append(currProps.getProperty(pname)).append("\n"); + } + } + if (countNew + countRemoved + countDiffer > 0) { + out.println("Current properties differ: " + + countNew + " new, " + + countDiffer + " different values, " + + countRemoved + " removed, " + + countNoChange + " unchanged, " + + currMetaPropVal + " previously updated" + ); + out.println(changes); + propsDiffer = true; + } else { + out.println("Current properties identical to file given, entry count=" + countNoChange); + } + } + if (propsDiffer) { + if (currProps != null) { + zkClient.delete(propPath); + } + // update zookeeper + zkClient.createPersistent(propPath, props); + out.println("Properties updated, entry count=" + props.size()); + } + } + + /** + * @param zkClient zookeeper client. + * @param zpathBase znode base path. + * @param out the PrintStream for human readable update summary. + */ + private static void createZNodes(ZkClient zkClient, String zpathBase, PrintStream out) + { + zkClient.createPersistent(zpathBase, true); + for (String subPath : Initialization.SUB_PATHS) { + final String thePath = String.format("%s/%s", zpathBase, subPath); + if (zkClient.exists(thePath)) { + out.printf("Path[%s] exists already%n", thePath); + } else { + out.printf("Creating ZK path[%s]%n", thePath); + zkClient.createPersistent(thePath, true); + } + } + } + + private static void reportErrorAndExit(String pfile, IOException ioe) + { + System.err.println("Could not read file: " + pfile); + System.err.println(" because of: " + ioe); + System.err.println("No changes made."); + System.exit(4); + } + + private static ZkClient connectToZK(String zkConnect) + { + return new ZkClient( + new ZkConnection(zkConnect), + Integer.MAX_VALUE, + new PropertiesZkSerializer() + ); + } + + /** + * Connect to db and create table, if it does not exist. + * NOTE: Connection failure only shows up in log output. + * + * @param pfile path to properties file to use. + */ + private static void prepDB(final String pfile) + { + Properties tmp_props = new Properties(); + loadProperties(pfile, tmp_props); + final String tableName = tmp_props.getProperty("druid.database.segmentTable", "prod_segments"); + + final String dbConnectionUrl = tmp_props.getProperty("druid.database.connectURI"); + final String username = tmp_props.getProperty("druid.database.user"); + final String password = tmp_props.getProperty("druid.database.password"); + + // + // validation + // + if (tableName.length() == 0 || !Character.isLetter(tableName.charAt(0))) { + throw new RuntimeException("poorly formed property druid.database.segmentTable=" + tableName); + } + if (username == null || username.length() == 0) { + throw new RuntimeException("poorly formed property druid.database.user=" + username); + } + if (password == null || password.length() == 0) { + throw new RuntimeException("poorly formed property druid.database.password=" + password); + } + if (dbConnectionUrl == null || dbConnectionUrl.length() == 0) { + throw new RuntimeException("poorly formed property druid.database.connectURI=" + dbConnectionUrl); + } + + final DbConnectorConfig config = new DbConnectorConfig() + { + { + } + + @Override + public String getDatabaseConnectURI() + { + return dbConnectionUrl; + } + + @Override + public String getDatabaseUser() + { + return username; + } + + @Override + public String getDatabasePassword() + { + return password; + } + + @Override + public String getSegmentTable() + { + return tableName; + } + }; + + DbConnector dbConnector = new DbConnector(config); + + DbConnector.createSegmentTable(dbConnector.getDBI(), config); + + } + + /** + * Print usage to stdout. + */ + private static void printUsage() + { + System.out.println("Usage: CMD [args]\n" + + " Where CMD is a particular command:\n" + + " CMD choices:\n" + + " dump zkConnect baseZkPath # dump info from zk at given coordinates\n" + + " dbprep propfile # create metadata table in db\n" + + " put zkConnect propfile # store paths and propfile into zk at given coordinates\n" + + " args:\n" + + " zkConnect: ZK coordinates in the form host1:port1[,host2:port2[, ...]]\n" + + " baseZkPath: like /druid or /mydruid etc. to uniquely identify a Druid ensemble\n" + + " and should be equal to property druid.zk.paths.base\n" + + " propfile: Java properties file with common properties for all services in ensemble\n" + + " Notes:\n" + + " dump command makes no modifications and shows zk path listing and zk properties.\n" + + " put command can safely be invoked more than once, will not disturb existing queues,\n" + + " and properties are not cumulative.\n" + + " A zookeeper can service more than one Druid ensemble if baseZkPath is distinct.\n" + + " Druid services only load properties during process startup.\n" + + " Properties defined on a service command line take precedence over the runtime.properties\n" + + " file which takes precedence over properties stored in zookeeper.\n" + + "" + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/utils/ZkSetup.java b/server/src/main/java/com/metamx/druid/utils/ZkSetup.java index 95492fbeb7e..061854a7a68 100644 --- a/server/src/main/java/com/metamx/druid/utils/ZkSetup.java +++ b/server/src/main/java/com/metamx/druid/utils/ZkSetup.java @@ -21,11 +21,13 @@ package com.metamx.druid.utils; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.initialization.Initialization; import com.metamx.druid.zk.StringZkSerializer; import org.I0Itec.zkclient.ZkClient; import org.I0Itec.zkclient.ZkConnection; /** + * @deprecated see DruidSetup */ public class ZkSetup { @@ -33,13 +35,12 @@ public class ZkSetup { if (args.length != 5) { System.out.println("Usage: zkConnect baseZkPath dbConnectionUrl dbUsername:password tableName"); + System.out.println("This utility is deprecated, see DruidSetup instead."); System.exit(1); } String path = args[1]; - String[] subPaths = new String[]{"announcements", "servedSegments", "loadQueue", "master"}; - final ZkClient zkClient = new ZkClient( new ZkConnection(args[0]), Integer.MAX_VALUE, @@ -47,7 +48,7 @@ public class ZkSetup ); zkClient.createPersistent(path, true); - for (String subPath : subPaths) { + for (String subPath : Initialization.SUB_PATHS) { final String thePath = String.format("%s/%s", path, subPath); if (zkClient.exists(thePath)) { System.out.printf("Path[%s] exists already%n", thePath); From cb02c6841b7023208b63224635628521188ca24d Mon Sep 17 00:00:00 2001 From: Paul Baclace Date: Tue, 6 Nov 2012 15:30:46 -0800 Subject: [PATCH 13/19] removed recursive zpath listing from DruidSetup dump command. --- server/src/main/java/com/metamx/druid/utils/DruidSetup.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/utils/DruidSetup.java b/server/src/main/java/com/metamx/druid/utils/DruidSetup.java index 9b96e1d88a1..6ef2432e5a7 100644 --- a/server/src/main/java/com/metamx/druid/utils/DruidSetup.java +++ b/server/src/main/java/com/metamx/druid/utils/DruidSetup.java @@ -177,10 +177,10 @@ public class DruidSetup out.println(); } } - out.println("Zookeeper znodes and zpaths for " + zkConnect + " (showing all zpaths)"); + //out.println("Zookeeper znodes and zpaths for " + zkConnect + " (showing all zpaths)"); // list all znodes // (not ideal since recursive listing starts at / instead of at baseZkPath) - zkClient.showFolders(out); + //zkClient.showFolders(out); } /** @@ -399,7 +399,7 @@ public class DruidSetup + " and should be equal to property druid.zk.paths.base\n" + " propfile: Java properties file with common properties for all services in ensemble\n" + " Notes:\n" - + " dump command makes no modifications and shows zk path listing and zk properties.\n" + + " dump command makes no modifications and shows zk properties at baseZkPath.\n" + " put command can safely be invoked more than once, will not disturb existing queues,\n" + " and properties are not cumulative.\n" + " A zookeeper can service more than one Druid ensemble if baseZkPath is distinct.\n" From 217caa3a04fad7f0f95de7d178c91f16a2498f04 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Tue, 6 Nov 2012 16:55:35 -0800 Subject: [PATCH 14/19] HistogramVisual: put min. and max. in quantile array --- .../com/metamx/druid/aggregation/Histogram.java | 7 ++++++- .../druid/aggregation/HistogramVisual.java | 17 +++-------------- .../metamx/druid/histogram/HistogramTest.java | 4 +--- 3 files changed, 10 insertions(+), 18 deletions(-) diff --git a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java b/common/src/main/java/com/metamx/druid/aggregation/Histogram.java index 7199d098d6a..bb27e317eb4 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java +++ b/common/src/main/java/com/metamx/druid/aggregation/Histogram.java @@ -132,10 +132,15 @@ public class Histogram return buf.array(); } + /** + * Returns a visual representation of a histogram object. + * Initially returns an array of just the min. and max. values + * but can also support the addition of quantiles. + */ public HistogramVisual asVisual() { float[] visualCounts = new float[bins.length - 2]; for(int i = 0; i < visualCounts.length; ++i) visualCounts[i] = (float)bins[i + 1]; - return new HistogramVisual(breaks, visualCounts, new float[]{}, min, max); + return new HistogramVisual(breaks, visualCounts, new float[]{min, max}); } public static Histogram fromBytes(byte[] bytes) { diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java b/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java index 4a88e652c7b..ab4e8836fe1 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java +++ b/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java @@ -29,17 +29,14 @@ public class HistogramVisual { @JsonProperty final public double[] breaks; @JsonProperty final public double[] counts; + // an array of the quantiles including the min. and max. @JsonProperty final public double[] quantiles; - @JsonProperty final double min; - @JsonProperty final double max; @JsonCreator public HistogramVisual( @JsonProperty double[] breaks, @JsonProperty double[] counts, - @JsonProperty double[] quantiles, - @JsonProperty double min, - @JsonProperty double max + @JsonProperty double[] quantiles ) { Preconditions.checkArgument(breaks != null, "breaks must not be null"); @@ -49,16 +46,12 @@ public class HistogramVisual this.breaks = breaks; this.counts = counts; this.quantiles = quantiles; - this.min = min; - this.max = max; } public HistogramVisual( float[] breaks, float[] counts, - float[] quantiles, - float min, - float max + float[] quantiles ) { Preconditions.checkArgument(breaks != null, "breaks must not be null"); @@ -71,8 +64,6 @@ public class HistogramVisual for(int i = 0; i < breaks.length; ++i) this.breaks[i] = breaks[i]; for(int i = 0; i < counts.length; ++i) this.counts[i] = counts[i]; for(int i = 0; i < quantiles.length; ++i) this.quantiles[i] = quantiles[i]; - this.min = min; - this.max = max; } @Override @@ -81,9 +72,7 @@ public class HistogramVisual return "HistogramVisual{" + "counts=" + Arrays.toString(counts) + ", breaks=" + Arrays.toString(breaks) + - ", min=" + min + ", quantiles=" + Arrays.toString(quantiles) + - ", max=" + max + '}'; } } diff --git a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java index 7b764365f91..c280507e207 100644 --- a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java +++ b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java @@ -68,9 +68,7 @@ public class HistogramTest Map expectedObj = Maps.newLinkedHashMap(); expectedObj.put("breaks", Arrays.asList(visualBreaks)); expectedObj.put("counts", Arrays.asList(visualCounts)); - expectedObj.put("quantiles", Arrays.asList(new Double[]{})); - expectedObj.put("min", -1.0); - expectedObj.put("max", 1.0); + expectedObj.put("quantiles", Arrays.asList(new Double[]{-1.0, 1.0})); Map obj = (Map)objectMapper.readValue(json, Object.class); Assert.assertEquals(expectedObj, obj); From 52214d735578630f4af6efc397d4136a0bf3d5ee Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 7 Nov 2012 17:27:23 -0800 Subject: [PATCH 15/19] additional improvements according to code review; unit tests; bug fixes for retry policies --- merger/pom.xml | 6 +- .../druid/merger/common/task/MergeTask.java | 2 +- .../merger/coordinator/RemoteTaskRunner.java | 146 +++--- .../druid/merger/coordinator/RetryPolicy.java | 19 +- .../merger/coordinator/WorkerWrapper.java | 29 +- .../config/RemoteTaskRunnerConfig.java | 10 +- .../coordinator/config/RetryPolicyConfig.java | 9 +- .../scaling/EC2AutoScalingStrategy.java | 2 +- .../scaling/NoopScalingStrategy.java | 9 +- .../coordinator/scaling/ScalingStrategy.java | 2 +- .../druid/merger/worker/TaskMonitor.java | 5 +- .../druid/merger/worker/http/WorkerNode.java | 5 +- .../coordinator/RemoteTaskRunnerTest.java | 433 ++++++++++++++++++ .../merger/coordinator/RetryPolicyTest.java | 45 ++ .../scaling/EC2AutoScalingStrategyTest.java | 26 +- pom.xml | 7 +- 16 files changed, 622 insertions(+), 133 deletions(-) create mode 100644 merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java create mode 100644 merger/src/test/java/com/metamx/druid/merger/coordinator/RetryPolicyTest.java diff --git a/merger/pom.xml b/merger/pom.xml index e5fc6f99f69..c60bd7e2bd5 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -19,7 +19,7 @@ --> + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> 4.0.0 com.metamx.druid druid-merger @@ -178,6 +178,10 @@ easymock test + + com.netflix.curator + curator-test + diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index db5ff0594da..859352c2a7d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -88,7 +88,7 @@ public abstract class MergeTask extends AbstractTask @Override public boolean apply(@Nullable DataSegment segment) { - return segment == null || !segment.getDataSource().equals(dataSource); + return segment == null || !segment.getDataSource().equalsIgnoreCase(dataSource); } } ) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 84f71360404..a2bc0c3688e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -25,7 +25,9 @@ import com.google.common.base.Predicate; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; +import com.google.common.collect.Sets; import com.google.common.primitives.Ints; +import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; @@ -54,6 +56,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.Comparator; import java.util.List; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ScheduledExecutorService; @@ -95,6 +98,8 @@ public class RemoteTaskRunner implements TaskRunner private final ConcurrentSkipListSet currentlyTerminating = new ConcurrentSkipListSet(); private final Object statusLock = new Object(); + private volatile DateTime lastProvisionTime = new DateTime(); + private volatile DateTime lastTerminateTime = new DateTime(); private volatile boolean started = false; public RemoteTaskRunner( @@ -120,27 +125,31 @@ public class RemoteTaskRunner implements TaskRunner public void start() { try { - workerPathCache.start(); workerPathCache.getListenable().addListener( new PathChildrenCacheListener() { @Override public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception { - final Worker worker = jsonMapper.readValue( - event.getData().getData(), - Worker.class - ); if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { + final Worker worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); log.info("New worker[%s] found!", worker.getHost()); addWorker(worker); } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { + final Worker worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); log.info("Worker[%s] removed!", worker.getHost()); removeWorker(worker.getHost()); } } } ); + workerPathCache.start(); // Schedule termination of worker nodes periodically Period period = new Period(config.getTerminateResourcesDuration()); @@ -175,7 +184,7 @@ public class RemoteTaskRunner implements TaskRunner { return input.getRunningTasks().isEmpty() && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() - > config.getmaxWorkerIdleTimeMillisBeforeDeletion(); + > config.getMaxWorkerIdleTimeMillisBeforeDeletion(); } } ) @@ -195,8 +204,19 @@ public class RemoteTaskRunner implements TaskRunner ) ); - currentlyTerminating.addAll(terminated.getNodeIds()); + if (terminated != null) { + currentlyTerminating.addAll(terminated.getNodeIds()); + lastTerminateTime = new DateTime(); + } } else { + Duration durSinceLastTerminate = new Duration(new DateTime(), lastTerminateTime); + if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration())) { + log.makeAlert( + "It has been %d millis since last scheduled termination but nodes remain", + durSinceLastTerminate.getMillis() + ).emit(); + } + log.info( "[%s] still terminating. Wait for all nodes to terminate before trying again.", currentlyTerminating @@ -233,36 +253,38 @@ public class RemoteTaskRunner implements TaskRunner return started; } + public int getNumWorkers() + { + return zkWorkers.size(); + } + @Override public void run(Task task, TaskContext context, TaskCallback callback) { - assignTask( - new TaskWrapper( - task, context, callback, retryPolicyFactory.makeRetryPolicy() - ) + if (tasks.contains(task.getId())) { + throw new ISE("Assigned a task[%s] that already exists, WTF is happening?!", task.getId()); + } + TaskWrapper taskWrapper = new TaskWrapper( + task, context, callback, retryPolicyFactory.makeRetryPolicy() ); + tasks.put(taskWrapper.getTask().getId(), taskWrapper); + assignTask(taskWrapper); } private void assignTask(TaskWrapper taskWrapper) { - tasks.putIfAbsent(taskWrapper.getTask().getId(), taskWrapper); WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); // If the task already exists, we don't need to announce it if (workerWrapper != null) { final Worker worker = workerWrapper.getWorker(); try { - log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskWrapper.getTask().getId()); TaskStatus taskStatus = jsonMapper.readValue( workerWrapper.getStatusCache() .getCurrentData( - JOINER.join( - config.getStatusPath(), - worker.getHost(), - taskWrapper.getTask().getId() - ) + JOINER.join(config.getStatusPath(), worker.getHost(), taskWrapper.getTask().getId()) ) .getData(), TaskStatus.class @@ -280,8 +302,9 @@ public class RemoteTaskRunner implements TaskRunner log.error(e, "Task exists, but hit exception!"); retryTask(new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()), taskWrapper); } - } else { // Announce the task - workerWrapper = getWorkerForTask(); + } else { + // Announce the task or retry if there is not enough capacity + workerWrapper = findWorkerForTask(); if (workerWrapper != null) { announceTask(workerWrapper.getWorker(), taskWrapper); } else { @@ -328,7 +351,7 @@ public class RemoteTaskRunner implements TaskRunner } } }, - retryPolicy.getAndIncrementRetryDelay(), + retryPolicy.getAndIncrementRetryDelay().getMillis(), TimeUnit.MILLISECONDS ); } @@ -347,28 +370,22 @@ public class RemoteTaskRunner implements TaskRunner final String workerStatusPath = JOINER.join(config.getStatusPath(), worker.getHost()); final PathChildrenCache statusCache = new PathChildrenCache(cf, workerStatusPath, true); - final ConcurrentSkipListSet runningTasks = new ConcurrentSkipListSet( - Lists.transform( - statusCache.getCurrentData(), - new Function() - { - @Override - public String apply(@Nullable ChildData input) - { - try { - return jsonMapper.readValue(input.getData(), TaskStatus.class).getId(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ) - ); final WorkerWrapper workerWrapper = new WorkerWrapper( worker, - runningTasks, - statusCache + statusCache, + new Function() + { + @Override + public String apply(@Nullable ChildData input) + { + try { + return jsonMapper.readValue(input.getData(), TaskStatus.class).getId(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } ); // Add status listener to the watcher for status changes @@ -385,10 +402,7 @@ public class RemoteTaskRunner implements TaskRunner TaskStatus taskStatus = jsonMapper.readValue( event.getData().getData(), TaskStatus.class ); - taskId = taskStatus.getId(); - - log.info("New status[%s] appeared!", taskId); - runningTasks.add(taskId); + log.info("New status[%s] appeared!", taskStatus.getId()); statusLock.notify(); } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { String statusPath = event.getData().getPath(); @@ -413,13 +427,13 @@ public class RemoteTaskRunner implements TaskRunner callback.notify(taskStatus); } tasks.remove(taskId); - runningTasks.remove(taskId); - cf.delete().guaranteed().forPath(statusPath); + cf.delete().guaranteed().inBackground().forPath(statusPath); } } } } catch (Exception e) { + log.error(e, "Exception in status listener"); retryTask(new CleanupPaths(worker.getHost(), taskId), tasks.get(taskId)); } } @@ -456,25 +470,27 @@ public class RemoteTaskRunner implements TaskRunner WorkerWrapper workerWrapper = zkWorkers.get(workerId); if (workerWrapper != null) { - for (String taskId : workerWrapper.getRunningTasks()) { - TaskWrapper taskWrapper = tasks.get(taskId); - if (taskWrapper != null) { - retryTask(new CleanupPaths(workerId, taskId), tasks.get(taskId)); - } - workerWrapper.removeTask(taskId); - } - try { + Set tasksToRetry = Sets.newHashSet(workerWrapper.getRunningTasks()); + tasksToRetry.addAll(cf.getChildren().forPath(JOINER.join(config.getTaskPath(), workerId))); + + for (String taskId : tasksToRetry) { + TaskWrapper taskWrapper = tasks.get(taskId); + if (taskWrapper != null) { + retryTask(new CleanupPaths(workerId, taskId), tasks.get(taskId)); + } + } + workerWrapper.getStatusCache().close(); } - catch (IOException e) { - log.error("Failed to close watcher associated with worker[%s]", workerWrapper.getWorker().getHost()); + catch (Exception e) { + log.error(e, "Failed to cleanly remove worker[%s]"); } } zkWorkers.remove(workerId); } - private WorkerWrapper getWorkerForTask() + private WorkerWrapper findWorkerForTask() { try { final MinMaxPriorityQueue workerQueue = MinMaxPriorityQueue.orderedBy( @@ -504,11 +520,20 @@ public class RemoteTaskRunner implements TaskRunner log.info("Worker nodes do not have capacity to run any more tasks!"); if (currentlyProvisioning.isEmpty()) { - AutoScalingData provisioned = strategy.provision(); + AutoScalingData provisioned = strategy.provision(currentlyProvisioning.size()); if (provisioned != null) { currentlyProvisioning.addAll(provisioned.getNodeIds()); + lastProvisionTime = new DateTime(); } } else { + Duration durSinceLastProvision = new Duration(new DateTime(), lastProvisionTime); + if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration())) { + log.makeAlert( + "It has been %d millis since last scheduled provision but nodes remain", + durSinceLastProvision.getMillis() + ).emit(); + } + log.info( "[%s] still provisioning. Wait for all provisioned nodes to complete before requesting new worker.", currentlyProvisioning @@ -552,8 +577,9 @@ public class RemoteTaskRunner implements TaskRunner jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)) ); + // Syncing state with Zookeeper while (findWorkerRunningTask(taskWrapper) == null) { - statusLock.wait(); + statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis()); } } catch (Exception e) { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java index 24ee54290bb..fbb27d76c18 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java @@ -21,6 +21,7 @@ package com.metamx.druid.merger.coordinator; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.emitter.EmittingLogger; +import org.joda.time.Duration; /** */ @@ -29,29 +30,25 @@ public class RetryPolicy private static final EmittingLogger log = new EmittingLogger(RetryPolicy.class); private final long MAX_NUM_RETRIES; - private final long MAX_RETRY_DELAY_MILLIS; + private final Duration MAX_RETRY_DURATION; - private volatile long currRetryDelay; + private volatile Duration currRetryDelay; private volatile int retryCount; public RetryPolicy(RetryPolicyConfig config) { this.MAX_NUM_RETRIES = config.getMaxRetryCount(); - this.MAX_RETRY_DELAY_MILLIS = config.getRetryMaxMillis(); + this.MAX_RETRY_DURATION = config.getRetryMaxDuration(); - this.currRetryDelay = config.getRetryMinMillis(); + this.currRetryDelay = config.getRetryMinDuration(); this.retryCount = 0; } - public long getAndIncrementRetryDelay() + public Duration getAndIncrementRetryDelay() { - long retVal = currRetryDelay; - if (currRetryDelay < MAX_RETRY_DELAY_MILLIS) { - currRetryDelay *= 2; - } - + Duration retVal = new Duration(currRetryDelay); + currRetryDelay = new Duration(Math.min(currRetryDelay.getMillis() * 2, MAX_RETRY_DURATION.getMillis())); retryCount++; - return retVal; } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index 99e330c0a3a..68d4f0a128c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -19,29 +19,33 @@ package com.metamx.druid.merger.coordinator; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import com.metamx.druid.merger.worker.Worker; +import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import org.joda.time.DateTime; +import java.io.Closeable; import java.io.IOException; import java.util.Set; -import java.util.concurrent.ConcurrentSkipListSet; /** */ -public class WorkerWrapper +public class WorkerWrapper implements Closeable { private final Worker worker; - private final ConcurrentSkipListSet runningTasks; private final PathChildrenCache statusCache; + private final Function cacheConverter; private volatile DateTime lastCompletedTaskTime = new DateTime(); - public WorkerWrapper(Worker worker, ConcurrentSkipListSet runningTasks, PathChildrenCache statusCache) + public WorkerWrapper(Worker worker, PathChildrenCache statusCache, Function cacheConverter) { this.worker = worker; - this.runningTasks = runningTasks; this.statusCache = statusCache; + this.cacheConverter = cacheConverter; } public Worker getWorker() @@ -51,7 +55,12 @@ public class WorkerWrapper public Set getRunningTasks() { - return runningTasks; + return Sets.newHashSet( + Lists.transform( + statusCache.getCurrentData(), + cacheConverter + ) + ); } public PathChildrenCache getStatusCache() @@ -66,7 +75,7 @@ public class WorkerWrapper public boolean isAtCapacity() { - return runningTasks.size() >= worker.getCapacity(); + return statusCache.getCurrentData().size() >= worker.getCapacity(); } public void setLastCompletedTaskTime(DateTime completedTaskTime) @@ -74,11 +83,7 @@ public class WorkerWrapper lastCompletedTaskTime = completedTaskTime; } - public void removeTask(String taskId) - { - runningTasks.remove(taskId); - } - + @Override public void close() throws IOException { statusCache.close(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java index 5118e6e7c59..00b869ea6da 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -46,5 +46,13 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig @Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion") @Default("1") - public abstract int getmaxWorkerIdleTimeMillisBeforeDeletion(); + public abstract int getMaxWorkerIdleTimeMillisBeforeDeletion(); + + @Config("druid.indexer.maxScalingDuration") + @Default("PT1H") + public abstract Duration getMaxScalingDuration(); + + @Config("druid.indexer.taskAssignmentTimeoutDuration") + @Default("PT5M") + public abstract Duration getTaskAssignmentTimeoutDuration(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java index 044706b67ed..47c8eaf4d1a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator.config; +import org.joda.time.Duration; import org.skife.config.Config; import org.skife.config.Default; @@ -27,12 +28,12 @@ import org.skife.config.Default; public abstract class RetryPolicyConfig { @Config("druid.indexer.retry.minWaitMillis") - @Default("60000") // 1 minute - public abstract long getRetryMinMillis(); + @Default("PT1M") // 1 minute + public abstract Duration getRetryMinDuration(); @Config("druid.indexer.retry.maxWaitMillis") - @Default("600000") // 10 minutes - public abstract long getRetryMaxMillis(); + @Default("PT10M") // 10 minutes + public abstract Duration getRetryMaxDuration(); @Config("druid.indexer.retry.maxRetryCount") @Default("10") diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index cd94b70d3ce..a85c3ade8fd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -56,7 +56,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy } @Override - public AutoScalingData provision() + public AutoScalingData provision(long numUnassignedTasks) { try { log.info("Creating new instance(s)..."); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java index 0331022082c..923de463870 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java @@ -1,15 +1,8 @@ package com.metamx.druid.merger.coordinator.scaling; -import com.amazonaws.services.ec2.model.Instance; -import com.google.common.collect.MinMaxPriorityQueue; -import com.metamx.druid.merger.coordinator.WorkerWrapper; -import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.emitter.EmittingLogger; -import org.joda.time.DateTime; -import java.util.Comparator; import java.util.List; -import java.util.Map; /** * This class just logs when scaling should occur. @@ -19,7 +12,7 @@ public class NoopScalingStrategy implements ScalingStrategy private static final EmittingLogger log = new EmittingLogger(NoopScalingStrategy.class); @Override - public AutoScalingData provision() + public AutoScalingData provision(long numUnassignedTasks) { log.info("If I were a real strategy I'd create something now"); return null; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java index 9b7da8fb3a4..ec71d856301 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java @@ -25,7 +25,7 @@ import java.util.List; */ public interface ScalingStrategy { - public AutoScalingData provision(); + public AutoScalingData provision(long numUnassignedTasks); public AutoScalingData terminate(List nodeIds); } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index 938bda933fc..5fc49788fcd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -47,7 +47,6 @@ public class TaskMonitor private final PathChildrenCache pathChildrenCache; private final CuratorFramework cf; - private final ObjectMapper jsonMapper; private final WorkerCuratorCoordinator workerCuratorCoordinator; private final TaskToolbox toolbox; private final ExecutorService exec; @@ -55,7 +54,6 @@ public class TaskMonitor public TaskMonitor( PathChildrenCache pathChildrenCache, CuratorFramework cf, - ObjectMapper jsonMapper, WorkerCuratorCoordinator workerCuratorCoordinator, TaskToolbox toolbox, ExecutorService exec @@ -63,7 +61,6 @@ public class TaskMonitor { this.pathChildrenCache = pathChildrenCache; this.cf = cf; - this.jsonMapper = jsonMapper; this.workerCuratorCoordinator = workerCuratorCoordinator; this.toolbox = toolbox; this.exec = exec; @@ -87,7 +84,7 @@ public class TaskMonitor throws Exception { if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final TaskHolder taskHolder = jsonMapper.readValue( + final TaskHolder taskHolder = toolbox.getObjectMapper().readValue( cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()), TaskHolder.class ); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 210a55bcf5f..cc30b914367 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -287,8 +287,8 @@ public class WorkerNode if (taskToolbox == null) { final RestS3Service s3Client = new RestS3Service( new AWSCredentials( - props.getProperty("com.metamx.aws.accessKey"), - props.getProperty("com.metamx.aws.secretKey") + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ); final SegmentPusher segmentPusher = new S3SegmentPusher( @@ -334,7 +334,6 @@ public class WorkerNode taskMonitor = new TaskMonitor( pathChildrenCache, curatorFramework, - jsonMapper, workerCuratorCoordinator, taskToolbox, workerExec diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java new file mode 100644 index 00000000000..e219c2984a0 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -0,0 +1,433 @@ +package com.metamx.druid.merger.coordinator; + +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.config.IndexerZkConfig; +import com.metamx.druid.merger.common.task.DefaultMergeTask; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; +import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; +import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; +import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; +import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; +import com.metamx.druid.merger.worker.TaskMonitor; +import com.metamx.druid.merger.worker.Worker; +import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; +import com.netflix.curator.framework.CuratorFramework; +import com.netflix.curator.framework.CuratorFrameworkFactory; +import com.netflix.curator.framework.recipes.cache.PathChildrenCache; +import com.netflix.curator.retry.ExponentialBackoffRetry; +import com.netflix.curator.test.TestingCluster; +import org.apache.commons.lang.mutable.MutableBoolean; +import org.apache.zookeeper.CreateMode; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.annotate.JsonTypeName; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.map.jsontype.NamedType; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.util.List; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; + +/** + */ +public class RemoteTaskRunnerTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private static final String basePath = "/test/druid/indexer"; + private static final String announcementsPath = String.format("%s/announcements", basePath); + private static final String tasksPath = String.format("%s/tasks", basePath); + private static final String statusPath = String.format("%s/status", basePath); + + private TestingCluster testingCluster; + private CuratorFramework cf; + private PathChildrenCache pathChildrenCache; + private RemoteTaskRunner remoteTaskRunner; + private TaskMonitor taskMonitor; + + private ScheduledExecutorService scheduledExec; + + private Task task1; + + private Worker worker1; + + + @Before + public void setUp() throws Exception + { + testingCluster = new TestingCluster(1); + testingCluster.start(); + + cf = CuratorFrameworkFactory.builder() + .connectString(testingCluster.getConnectString()) + .retryPolicy(new ExponentialBackoffRetry(1, 10)) + .build(); + cf.start(); + + cf.create().creatingParentsIfNeeded().forPath(announcementsPath); + cf.create().forPath(tasksPath); + cf.create().forPath(String.format("%s/worker1", tasksPath)); + cf.create().forPath(statusPath); + cf.create().forPath(String.format("%s/worker1", statusPath)); + + pathChildrenCache = new PathChildrenCache(cf, announcementsPath, true); + + worker1 = new Worker( + "worker1", + "localhost", + 3, + "0" + ); + + makeRemoteTaskRunner(); + makeTaskMonitor(); + } + + @After + public void tearDown() throws Exception + { + testingCluster.stop(); + remoteTaskRunner.stop(); + taskMonitor.stop(); + } + + @Test + public void testRunNoExistingTask() throws Exception + { + remoteTaskRunner.run( + task1, + new TaskContext(new DateTime().toString(), Sets.newHashSet()), + null + ); + } + + @Test + public void testRunWithExistingCompletedTask() throws Exception + { + cf.create().creatingParentsIfNeeded().forPath( + String.format("%s/worker1/task1", statusPath), + jsonMapper.writeValueAsBytes( + TaskStatus.success( + "task1", + Lists.newArrayList() + ) + ) + ); + + // Really don't like this way of waiting for the task to appear + while (remoteTaskRunner.getNumWorkers() == 0) { + Thread.sleep(500); + } + + final MutableBoolean callbackCalled = new MutableBoolean(false); + remoteTaskRunner.run( + task1, + null, + new TaskCallback() + { + @Override + public void notify(TaskStatus status) + { + callbackCalled.setValue(true); + } + } + ); + + Assert.assertTrue("TaskCallback was not called!", callbackCalled.booleanValue()); + } + + private void makeTaskMonitor() throws Exception + { + WorkerCuratorCoordinator workerCuratorCoordinator = new WorkerCuratorCoordinator( + jsonMapper, + new IndexerZkConfig() + { + @Override + public String getAnnouncementPath() + { + return announcementsPath; + } + + @Override + public String getTaskPath() + { + return tasksPath; + } + + @Override + public String getStatusPath() + { + return statusPath; + } + }, + cf, + worker1 + ); + workerCuratorCoordinator.start(); + + taskMonitor = new TaskMonitor( + new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true), + cf, + workerCuratorCoordinator, + new TaskToolbox( + new IndexerCoordinatorConfig() + { + @Override + public String getServerName() + { + return "worker1"; + } + + @Override + public String getLeaderLatchPath() + { + return null; + } + + @Override + public int getNumLocalThreads() + { + return 1; + } + + @Override + public String getRunnerImpl() + { + return null; + } + + @Override + public String getStorageImpl() + { + return null; + } + + @Override + public File getBaseTaskDir() + { + try { + return File.createTempFile("billy", "yay"); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @Override + public boolean isWhitelistEnabled() + { + return false; + } + + @Override + public String getWhitelistDatasourcesString() + { + return null; + } + + @Override + public long getRowFlushBoundary() + { + return 0; + } + }, null, null, null, jsonMapper + ), + Executors.newSingleThreadExecutor() + ); + jsonMapper.registerSubtypes(new NamedType(TestTask.class, "test")); + taskMonitor.start(); + } + + private void makeRemoteTaskRunner() throws Exception + { + scheduledExec = EasyMock.createMock(ScheduledExecutorService.class); + + remoteTaskRunner = new RemoteTaskRunner( + jsonMapper, + new TestRemoteTaskRunnerConfig(), + cf, + pathChildrenCache, + scheduledExec, + new RetryPolicyFactory(new TestRetryPolicyConfig()), + new TestScalingStrategy() + ); + + task1 = new TestTask( + "task1", + "dummyDs", + Lists.newArrayList( + new DataSegment( + "dummyDs", + new Interval(new DateTime(), new DateTime()), + new DateTime().toString(), + null, + null, + null, + null, + 0 + ) + ), Lists.newArrayList() + ); + + // Create a single worker and wait for things for be ready + remoteTaskRunner.start(); + cf.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath( + String.format("%s/worker1", announcementsPath), + jsonMapper.writeValueAsBytes(worker1) + ); + while (remoteTaskRunner.getNumWorkers() == 0) { + Thread.sleep(500); + } + } + + private static class TestRetryPolicyConfig extends RetryPolicyConfig + { + @Override + public Duration getRetryMinDuration() + { + return null; + } + + @Override + public Duration getRetryMaxDuration() + { + return null; + } + + @Override + public long getMaxRetryCount() + { + return 0; + } + } + + private static class TestScalingStrategy implements ScalingStrategy + { + @Override + public AutoScalingData provision(long numUnassignedTasks) + { + return null; + } + + @Override + public AutoScalingData terminate(List nodeIds) + { + return null; + } + } + + private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig + { + @Override + public Duration getTerminateResourcesDuration() + { + return null; + } + + @Override + public DateTime getTerminateResourcesOriginDateTime() + { + return null; + } + + @Override + public String getMinWorkerVersion() + { + return "0"; + } + + @Override + public int getMinNumWorkers() + { + return 0; + } + + @Override + public int getMaxWorkerIdleTimeMillisBeforeDeletion() + { + return 0; + } + + @Override + public Duration getMaxScalingDuration() + { + return null; + } + + @Override + public String getAnnouncementPath() + { + return announcementsPath; + } + + @Override + public String getTaskPath() + { + return tasksPath; + } + + @Override + public String getStatusPath() + { + return statusPath; + } + + @Override + public Duration getTaskAssignmentTimeoutDuration() + { + return new Duration(60000); + } + } + + @JsonTypeName("test") + private static class TestTask extends DefaultMergeTask + { + private final String id; + + public TestTask( + @JsonProperty("id") String id, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segments") List segments, + @JsonProperty("aggregations") List aggregators + ) + { + super(dataSource, segments, aggregators); + + this.id = id; + } + + @Override + @JsonProperty + public String getId() + { + return id; + } + + @Override + public Type getType() + { + return Type.TEST; + } + + @Override + public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + { + return TaskStatus.success("task1", Lists.newArrayList()); + } + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RetryPolicyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RetryPolicyTest.java new file mode 100644 index 00000000000..5445c05e7dd --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RetryPolicyTest.java @@ -0,0 +1,45 @@ +package com.metamx.druid.merger.coordinator; + +import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; +import junit.framework.Assert; +import org.joda.time.Duration; +import org.junit.Test; + +/** + */ +public class RetryPolicyTest +{ + @Test + public void testGetAndIncrementRetryDelay() throws Exception + { + RetryPolicy retryPolicy = new RetryPolicy( + new RetryPolicyConfig() + { + @Override + public Duration getRetryMinDuration() + { + return new Duration("PT1S"); + } + + @Override + public Duration getRetryMaxDuration() + { + return new Duration("PT10S"); + } + + @Override + public long getMaxRetryCount() + { + return 10; + } + } + ); + + Assert.assertEquals(new Duration("PT1S"), retryPolicy.getAndIncrementRetryDelay()); + Assert.assertEquals(new Duration("PT2S"), retryPolicy.getAndIncrementRetryDelay()); + Assert.assertEquals(new Duration("PT4S"), retryPolicy.getAndIncrementRetryDelay()); + Assert.assertEquals(new Duration("PT8S"), retryPolicy.getAndIncrementRetryDelay()); + Assert.assertEquals(new Duration("PT10S"), retryPolicy.getAndIncrementRetryDelay()); + Assert.assertEquals(new Duration("PT10S"), retryPolicy.getAndIncrementRetryDelay()); + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java index f5d682ba781..11a837196cc 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -27,12 +27,8 @@ import com.amazonaws.services.ec2.model.Reservation; import com.amazonaws.services.ec2.model.RunInstancesRequest; import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; -import com.google.common.collect.Maps; -import com.metamx.druid.merger.coordinator.WorkerWrapper; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; -import com.metamx.druid.merger.worker.Worker; import org.easymock.EasyMock; -import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -40,8 +36,6 @@ import org.junit.Test; import java.util.Arrays; import java.util.Date; -import java.util.Map; -import java.util.concurrent.ConcurrentSkipListSet; /** */ @@ -56,7 +50,6 @@ public class EC2AutoScalingStrategyTest private DescribeInstancesResult describeInstancesResult; private Reservation reservation; private Instance instance; - private WorkerWrapper worker; private EC2AutoScalingStrategy strategy; @Before @@ -73,12 +66,6 @@ public class EC2AutoScalingStrategyTest .withImageId(AMI_ID) .withPrivateIpAddress(IP); - worker = new WorkerWrapper( - new Worker("dummyHost", IP, 2, "0"), - new ConcurrentSkipListSet(), - null - ); - worker.setLastCompletedTaskTime(new DateTime(0)); strategy = new EC2AutoScalingStrategy( amazonEC2Client, new EC2AutoScalingStrategyConfig() { @@ -145,23 +132,12 @@ public class EC2AutoScalingStrategyTest EasyMock.expect(reservation.getInstances()).andReturn(Arrays.asList(instance)).atLeastOnce(); EasyMock.replay(reservation); - worker.getRunningTasks().add("task1"); - - Assert.assertFalse(worker.isAtCapacity()); - - worker.getRunningTasks().add("task2"); - - Assert.assertTrue(worker.isAtCapacity()); - - AutoScalingData created = strategy.provision(); + AutoScalingData created = strategy.provision(0); Assert.assertEquals(created.getNodeIds().size(), 1); Assert.assertEquals(created.getNodes().size(), 1); Assert.assertEquals(String.format("%s:8080", IP), created.getNodeIds().get(0)); - worker.getRunningTasks().remove("task1"); - worker.getRunningTasks().remove("task2"); - AutoScalingData deleted = strategy.terminate(Arrays.asList("dummyHost")); Assert.assertEquals(deleted.getNodeIds().size(), 1); diff --git a/pom.xml b/pom.xml index cbeb399f8ed..8a4b8b3d650 100644 --- a/pom.xml +++ b/pom.xml @@ -19,7 +19,7 @@ --> + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> 4.0.0 com.metamx druid @@ -131,6 +131,11 @@ curator-x-discovery 1.2.2 + + com.netflix.curator + curator-test + 1.2.2 + it.uniroma3.mat extendedset From f39af83be4cf828df2dbe5bc41e8ec902427b273 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Thu, 8 Nov 2012 09:30:32 -0800 Subject: [PATCH 16/19] make switching between scaling strategies config based --- .../config/IndexerCoordinatorConfig.java | 4 +++ .../http/IndexerCoordinatorNode.java | 32 ++++++++++++------- 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java index 18f317525fe..6d6218c3bff 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java @@ -83,4 +83,8 @@ public abstract class IndexerCoordinatorConfig @Config("druid.merger.rowFlushBoundary") @Default("500000") public abstract long getRowFlushBoundary(); + + @Config("druid.indexer.strategy") + @Default("noop") + public abstract String getStrategyImpl(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 2b689c71706..3bcfca0deff 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -468,17 +468,27 @@ public class IndexerCoordinatorNode .build() ); - ScalingStrategy strategy = new EC2AutoScalingStrategy( - new AmazonEC2Client( - new BasicAWSCredentials( - props.getProperty("com.metamx.aws.accessKey"), - props.getProperty("com.metamx.aws.secretKey") - ) - ), - configFactory.build(EC2AutoScalingStrategyConfig.class) - ); - // TODO: use real strategy before actual deployment - strategy = new NoopScalingStrategy(); + ScalingStrategy strategy; + if (config.getStrategyImpl().equalsIgnoreCase("ec2")) { + strategy = new EC2AutoScalingStrategy( + new AmazonEC2Client( + new BasicAWSCredentials( + props.getProperty("com.metamx.aws.accessKey"), + props.getProperty("com.metamx.aws.secretKey") + ) + ), + configFactory.build(EC2AutoScalingStrategyConfig.class) + ); + } else if (config.getStorageImpl().equalsIgnoreCase("noop")) { + strategy = new NoopScalingStrategy(); + } else { + throw new IllegalStateException( + String.format( + "Invalid strategy implementation: %s", + config.getStrategyImpl() + ) + ); + } return new RemoteTaskRunner( jsonMapper, From cdd9cdb2bddb838d45b97be820bced1ea4aefdc2 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Thu, 8 Nov 2012 09:31:22 -0800 Subject: [PATCH 17/19] use PropUtils for indexer coordinator node --- .../merger/coordinator/http/IndexerCoordinatorNode.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 3bcfca0deff..c8fc5cebb78 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -384,8 +384,8 @@ public class IndexerCoordinatorNode if (taskToolbox == null) { final RestS3Service s3Client = new RestS3Service( new AWSCredentials( - props.getProperty("com.metamx.aws.accessKey"), - props.getProperty("com.metamx.aws.secretKey") + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ); final SegmentPusher segmentPusher = new S3SegmentPusher( @@ -473,8 +473,8 @@ public class IndexerCoordinatorNode strategy = new EC2AutoScalingStrategy( new AmazonEC2Client( new BasicAWSCredentials( - props.getProperty("com.metamx.aws.accessKey"), - props.getProperty("com.metamx.aws.secretKey") + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ), configFactory.build(EC2AutoScalingStrategyConfig.class) From 7083821fbff153c61c17c999d6330023f20301d2 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Thu, 8 Nov 2012 10:08:40 -0800 Subject: [PATCH 18/19] no param for scaling provision and minor fix for initialization --- .../druid/initialization/Initialization.java | 46 +++++++++++++------ .../merger/coordinator/RemoteTaskRunner.java | 18 +------- .../merger/coordinator/WorkerWrapper.java | 20 +++++++- .../scaling/EC2AutoScalingStrategy.java | 2 +- .../scaling/NoopScalingStrategy.java | 2 +- .../coordinator/scaling/ScalingStrategy.java | 2 +- 6 files changed, 54 insertions(+), 36 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index addf3ac6ebb..1d2c5c12a11 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -69,7 +69,8 @@ public class Initialization "druid.zk.paths.announcementsPath", "druid.zk.paths.servedSegmentsPath", "druid.zk.paths.loadQueuePath", - "druid.zk.paths.masterPath"}; + "druid.zk.paths.masterPath" + }; public static final String DEFAULT_ZPATH = "/druid"; public static ZkClient makeZkClient(ZkClientConfig config, Lifecycle lifecycle) @@ -119,10 +120,12 @@ public class Initialization } - /** Load properties. + /** + * Load properties. * Properties are layered, high to low precedence: cmdLine -D, runtime.properties file, stored in zookeeper. * Idempotent. Thread-safe. Properties are only loaded once. * If property druid.zk.service.host=none then do not load properties from zookeeper. + * * @return Properties ready to use. */ public synchronized static Properties loadProperties() @@ -139,7 +142,9 @@ public class Initialization final InputStream stream = ClassLoader.getSystemResourceAsStream("runtime.properties"); if (stream == null) { - log.info("runtime.properties not found as a resource in classpath, relying only on system properties, and zookeeper now."); + log.info( + "runtime.properties not found as a resource in classpath, relying only on system properties, and zookeeper now." + ); } else { log.info("Loading properties from runtime.properties"); try { @@ -202,7 +207,7 @@ public class Initialization log.warn("property druid.zk.service.host is not set, so no way to contact zookeeper for coordination."); } // validate properties now that all levels of precedence are loaded - if (! validateResolveProps(tmp_props)) { + if (!validateResolveProps(tmp_props)) { log.error("Properties failed to validate, cannot continue"); throw new RuntimeException("Properties failed to validate"); } @@ -358,12 +363,15 @@ public class Initialization return String.format("%s/%s", basePath, PROP_SUBPATH); } - /** Validate and Resolve Properties. + /** + * Validate and Resolve Properties. * Resolve zpaths with props like druid.zk.paths.*Path using druid.zk.paths.base value. * Check validity so that if druid.zk.paths.*Path props are set, all are set, * if none set, then construct defaults relative to druid.zk.paths.base and add these * to the properties chain. + * * @param props + * * @return true if valid zpath properties. */ public static boolean validateResolveProps(Properties props) @@ -379,7 +387,9 @@ public class Initialization final String propertiesZpathOverride = props.getProperty("druid.zk.paths.propertiesPath"); - if (!zpathValidateFailed) System.out.println("Effective zpath prefix=" + zpathEffective); + if (!zpathValidateFailed) { + System.out.println("Effective zpath prefix=" + zpathEffective); + } // validate druid.zk.paths.*Path properties // @@ -408,22 +418,25 @@ public class Initialization } } if (zpathOverridesNotAbs) { - System.err.println("When overriding zk zpaths, with properties like druid.zk.paths.*Path " + - "the znode path must start with '/' (slash) ; problem overrides:"); + System.err.println( + "When overriding zk zpaths, with properties like druid.zk.paths.*Path " + + "the znode path must start with '/' (slash) ; problem overrides:" + ); System.err.print(sbErrors.toString()); } if (zpathOverrideCount > 0) { - if (zpathOverrideCount < SUB_PATH_PROPS.length + 1) { + if (zpathOverrideCount < SUB_PATH_PROPS.length) { zpathValidateFailed = true; - System.err.println("When overriding zk zpaths, with properties of form druid.zk.paths.*Path " + - "all must be overridden together; missing overrides:"); + System.err.println( + "When overriding zk zpaths, with properties of form druid.zk.paths.*Path " + + "all must be overridden together; missing overrides:" + ); for (int i = 0; i < SUB_PATH_PROPS.length; i++) { String val = props.getProperty(SUB_PATH_PROPS[i]); if (val == null) { System.err.println(" " + SUB_PATH_PROPS[i]); } } - if (propertiesZpathOverride == null) System.err.println(" " + "druid.zk.paths.propertiesPath"); } else { // proper overrides // do not prefix with property druid.zk.paths.base ; // fallthru @@ -440,13 +453,16 @@ public class Initialization } props.setProperty("druid.zk.paths.propertiesPath", zpathEffective + "/properties"); } - return ! zpathValidateFailed; + return !zpathValidateFailed; } - /** Check znode zpath base for proper slash, no trailing slash. - * @param zpathBase znode base path, if null then this method does nothing. + /** + * Check znode zpath base for proper slash, no trailing slash. + * + * @param zpathBase znode base path, if null then this method does nothing. * @param errorMsgPrefix error context to use if errors are emitted, should indicate * where the zpathBase value came from. + * * @return true if validate failed. */ public static boolean zpathBaseCheck(String zpathBase, String errorMsgPrefix) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index a2bc0c3688e..fce83b8618a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -42,7 +42,6 @@ import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; import com.metamx.druid.merger.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; -import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; @@ -53,7 +52,6 @@ import org.joda.time.Duration; import org.joda.time.Period; import javax.annotation.Nullable; -import java.io.IOException; import java.util.Comparator; import java.util.List; import java.util.Set; @@ -373,19 +371,7 @@ public class RemoteTaskRunner implements TaskRunner final WorkerWrapper workerWrapper = new WorkerWrapper( worker, statusCache, - new Function() - { - @Override - public String apply(@Nullable ChildData input) - { - try { - return jsonMapper.readValue(input.getData(), TaskStatus.class).getId(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } + jsonMapper ); // Add status listener to the watcher for status changes @@ -520,7 +506,7 @@ public class RemoteTaskRunner implements TaskRunner log.info("Worker nodes do not have capacity to run any more tasks!"); if (currentlyProvisioning.isEmpty()) { - AutoScalingData provisioned = strategy.provision(currentlyProvisioning.size()); + AutoScalingData provisioned = strategy.provision(); if (provisioned != null) { currentlyProvisioning.addAll(provisioned.getNodeIds()); lastProvisionTime = new DateTime(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index 68d4f0a128c..c6353698cbd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -20,13 +20,17 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Function; +import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.worker.Worker; import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; +import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.util.Set; @@ -41,11 +45,23 @@ public class WorkerWrapper implements Closeable private volatile DateTime lastCompletedTaskTime = new DateTime(); - public WorkerWrapper(Worker worker, PathChildrenCache statusCache, Function cacheConverter) + public WorkerWrapper(Worker worker, PathChildrenCache statusCache, final ObjectMapper jsonMapper) { this.worker = worker; this.statusCache = statusCache; - this.cacheConverter = cacheConverter; + this.cacheConverter = new Function() + { + @Override + public String apply(@Nullable ChildData input) + { + try { + return jsonMapper.readValue(input.getData(), TaskStatus.class).getId(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + }; } public Worker getWorker() diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index a85c3ade8fd..cd94b70d3ce 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -56,7 +56,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy } @Override - public AutoScalingData provision(long numUnassignedTasks) + public AutoScalingData provision() { try { log.info("Creating new instance(s)..."); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java index 923de463870..67eb99293e4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java @@ -12,7 +12,7 @@ public class NoopScalingStrategy implements ScalingStrategy private static final EmittingLogger log = new EmittingLogger(NoopScalingStrategy.class); @Override - public AutoScalingData provision(long numUnassignedTasks) + public AutoScalingData provision() { log.info("If I were a real strategy I'd create something now"); return null; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java index ec71d856301..9b7da8fb3a4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java @@ -25,7 +25,7 @@ import java.util.List; */ public interface ScalingStrategy { - public AutoScalingData provision(long numUnassignedTasks); + public AutoScalingData provision(); public AutoScalingData terminate(List nodeIds); } From 88bb02963a07c1df4b3409a3652a5da12b24afc7 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Thu, 8 Nov 2012 10:32:18 -0800 Subject: [PATCH 19/19] fix compile breakage introduced by scaling strategies --- .../coordinator/http/IndexerCoordinatorNode.java | 12 ++++-------- .../merger/coordinator/RemoteTaskRunnerTest.java | 9 ++++++++- .../scaling/EC2AutoScalingStrategyTest.java | 2 +- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index c8fc5cebb78..f7e5fbe6c81 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -28,6 +28,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; +import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.config.Config; @@ -444,7 +445,7 @@ public class IndexerCoordinatorNode final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class); taskStorage = new DbTaskStorage(jsonMapper, dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI()); } else { - throw new IllegalStateException(String.format("Invalid storage implementation: %s", config.getStorageImpl())); + throw new ISE("Invalid storage implementation: %s", config.getStorageImpl()); } } } @@ -482,12 +483,7 @@ public class IndexerCoordinatorNode } else if (config.getStorageImpl().equalsIgnoreCase("noop")) { strategy = new NoopScalingStrategy(); } else { - throw new IllegalStateException( - String.format( - "Invalid strategy implementation: %s", - config.getStrategyImpl() - ) - ); + throw new ISE("Invalid strategy implementation: %s",config.getStrategyImpl()); } return new RemoteTaskRunner( @@ -513,7 +509,7 @@ public class IndexerCoordinatorNode } }; } else { - throw new IllegalStateException(String.format("Invalid runner implementation: %s", config.getRunnerImpl())); + throw new ISE("Invalid runner implementation: %s", config.getRunnerImpl()); } } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index e219c2984a0..47373b0e6c1 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -245,6 +245,13 @@ public class RemoteTaskRunnerTest { return 0; } + + + @Override + public String getStrategyImpl() + { + return null; + } }, null, null, null, jsonMapper ), Executors.newSingleThreadExecutor() @@ -319,7 +326,7 @@ public class RemoteTaskRunnerTest private static class TestScalingStrategy implements ScalingStrategy { @Override - public AutoScalingData provision(long numUnassignedTasks) + public AutoScalingData provision() { return null; } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java index 11a837196cc..95039978476 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -132,7 +132,7 @@ public class EC2AutoScalingStrategyTest EasyMock.expect(reservation.getInstances()).andReturn(Arrays.asList(instance)).atLeastOnce(); EasyMock.replay(reservation); - AutoScalingData created = strategy.provision(0); + AutoScalingData created = strategy.provision(); Assert.assertEquals(created.getNodeIds().size(), 1); Assert.assertEquals(created.getNodes().size(), 1);