From 366216f9f38d93e283577bf21cde26c475194ce7 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Fri, 28 Dec 2012 16:30:21 -0800 Subject: [PATCH 001/121] initial cost analyzer commit --- .../druid/master/BalancerCostAnalyzer.java | 229 ++++++++++++++++++ .../druid/master/BalancerSegmentHolder2.java | 70 ++++++ .../com/metamx/druid/master/DruidMaster.java | 2 +- .../druid/master/DruidMasterBalancer.java | 61 +++-- .../druid/master/DruidMasterBalancerTest.java | 27 ++- 5 files changed, 367 insertions(+), 22 deletions(-) create mode 100644 server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java create mode 100644 server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder2.java diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java new file mode 100644 index 00000000000..86b220420f9 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -0,0 +1,229 @@ +/* + * 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.master; + +import com.google.common.collect.Lists; +import com.google.common.collect.MinMaxPriorityQueue; +import com.google.common.collect.Sets; +import com.metamx.common.Pair; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.DruidDataSource; +import com.metamx.druid.client.DruidServer; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Random; +import java.util.Set; + +/** + * The BalancerCostAnalyzer + */ +public class BalancerCostAnalyzer +{ + private static final Logger log = new Logger(BalancerCostAnalyzer.class); + private static final int MAX_SEGMENTS_TO_MOVE = 5; + private static final int DAY_IN_MILLIS = 1000 * 60 * 60 * 24; + private float initialTotalCost; + private float totalCostChange; + + private List serverHolderList; + private Random rand; + + public BalancerCostAnalyzer(){ + rand = new Random(); + totalCostChange = 0f; + } + + public void init(List serverHolderList){ + this.initialTotalCost = calculateInitialTotalCost(serverHolderList); + this.serverHolderList = serverHolderList; + } + + public float getInitialTotalCost() { + return initialTotalCost; + } + + public float getTotalCostChange() { + return totalCostChange; + } + + public float calculateInitialTotalCost(List serverHolderList){ + int cost = 0; + for (ServerHolder server : serverHolderList) { + DataSegment[] segments = server.getServer().getSegments().values().toArray(new DataSegment[]{}); + for (int i = 0; i < segments.length; ++i) { + for (int j = i; j < segments.length; ++j) { + cost += computeJointSegmentCosts(segments[i], segments[j]); + } + } + } + return cost; + } + + public float calculateTotalCostChange(List serverHolderList, Set segmentHoldersToMove){ + int cost = 0; + Iterator it = segmentHoldersToMove.iterator(); + while (it.hasNext()) { + BalancerSegmentHolder2 segmentHolder = (BalancerSegmentHolder2) it.next(); + for (DataSegment fromSegment : segmentHolder.getFromServer().getSegments().values()) { + cost -= computeJointSegmentCosts(segmentHolder.getSegment(), fromSegment); + } + for (DataSegment toSegment : segmentHolder.getToServer().getSegments().values()) { + cost += computeJointSegmentCosts(segmentHolder.getSegment(), toSegment); + } + return cost; + } + + for (ServerHolder server : serverHolderList) { + DataSegment[] segments = server.getServer().getSegments().values().toArray(new DataSegment[]{}); + for (int i = 0; i < segments.length; ++i) { + for (int j = i; j < segments.length; ++j) { + cost += computeJointSegmentCosts(segments[i], segments[j]); + } + } + } + return cost; + } + + public float computeJointSegmentCosts(DataSegment segment1, DataSegment segment2){ + float cost = 0f; + Interval gap = segment1.getInterval().gap(segment2.getInterval()); + + // gap is null if the two segment intervals overlap or if they're adjacent + if (gap == null) { + cost += 1f; + } else { + long gapMillis = gap.toDurationMillis(); + if (gapMillis < DAY_IN_MILLIS) cost += 1f; + } + + if(segment1.getDataSource().equals(segment2.getDataSource())) cost += 1f; + + return cost; + } + + public Set findSegmentsToMove() + { + Set segmentHoldersToMove = Sets.newHashSet(); + Set movingSegments = Sets.newHashSet(); + + int counter = 0; + + while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 2 * MAX_SEGMENTS_TO_MOVE) { + counter++; + ServerHolder fromServerHolder = serverHolderList.get(rand.nextInt(serverHolderList.size())); + List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); + if (segments.size() == 0) continue; + DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); + if (movingSegments.contains(proposalSegment)) continue; + + //Just need a regular priority queue for the min. element. + MinMaxPriorityQueue> pQueue = MinMaxPriorityQueue.orderedBy( + new Comparator>() + { + @Override + public int compare( + Pair o, + Pair o1 + ) + { + return Float.compare(o.lhs, o1.lhs); + } + } + ).create(); + + for (ServerHolder server : serverHolderList) { + float cost = 0f; + for (DataSegment segment : server.getServer().getSegments().values()) { + cost += computeJointSegmentCosts(proposalSegment, segment); + } + + //Take into account costs of segments that will be moved. + Iterator it = segmentHoldersToMove.iterator(); + while (it.hasNext()) { + BalancerSegmentHolder2 segmentToMove = (BalancerSegmentHolder2) it.next(); + if (server.getServer().equals(segmentToMove.getToServer())) { + cost += computeJointSegmentCosts(proposalSegment, segmentToMove.getSegment()); + } + if (server.getServer().equals(segmentToMove.getFromServer())) { + cost -= computeJointSegmentCosts(proposalSegment, segmentToMove.getSegment()); + } + } + + pQueue.add(Pair.of(cost, server)); + } + + Pair minPair = pQueue.peekFirst(); + if (!minPair.rhs.equals(fromServerHolder)) { + movingSegments.add(proposalSegment); + segmentHoldersToMove.add( + new BalancerSegmentHolder2( + fromServerHolder.getServer(), + minPair.rhs.getServer(), + proposalSegment + ) + ); + totalCostChange += minPair.lhs; + } + + } + + totalCostChange = calculateTotalCostChange(serverHolderList, segmentHoldersToMove); + return segmentHoldersToMove; + + /* + double currPercentDiff = getPercentDiff(); + + if (currPercentDiff < PERCENT_THRESHOLD) { + log.info("Cluster usage is balanced."); + return segmentsToMove; + } + + List dataSources = Lists.newArrayList(server.getDataSources()); + Collections.shuffle(dataSources); + + for (DruidDataSource dataSource : dataSources) { + List segments = Lists.newArrayList(dataSource.getSegments()); + Collections.shuffle(segments); + + for (DataSegment segment : segments) { + if (segmentsToMove.size() >= MAX_SEGMENTS_TO_MOVE) { + return segmentsToMove; + } + + if (getLookaheadPercentDiff(highestSizeUsed - segment.getSize(), lowestSizeUsed + segment.getSize()) + < currPercentDiff) { + segmentsToMove.add(new BalancerSegmentHolder(server, segment)); + update(highestSizeUsed - segment.getSize(), lowestSizeUsed + segment.getSize()); + } + } + } + + return segmentsToMove; + */ + } +} + + diff --git a/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder2.java b/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder2.java new file mode 100644 index 00000000000..22277285ee0 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder2.java @@ -0,0 +1,70 @@ +/* + * 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.master; + +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.DruidServer; + +/** + */ +public class BalancerSegmentHolder2 +{ + private final DruidServer fromServer; + private final DruidServer toServer; + private final DataSegment segment; + + private volatile int lifetime = 15; + + public BalancerSegmentHolder2( + DruidServer fromServer, + DruidServer toServer, + DataSegment segment + ) + { + this.fromServer = fromServer; + this.toServer = toServer; + this.segment = segment; + } + + public DruidServer getFromServer() + { + return fromServer; + } + + public DruidServer getToServer() + { + return toServer; + } + + public DataSegment getSegment() + { + return segment; + } + + public int getLifetime() + { + return lifetime; + } + + public void reduceLifetime() + { + lifetime--; + } +} diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 80172c432fa..4dfbb6018bb 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -674,7 +674,7 @@ public class DruidMaster }, new DruidMasterRuleRunner(DruidMaster.this), new DruidMasterCleanup(DruidMaster.this), - new DruidMasterBalancer(DruidMaster.this, new BalancerAnalyzer()), + new DruidMasterBalancer(DruidMaster.this, new BalancerCostAnalyzer()), new DruidMasterLogger() ) ); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 75d3534fef6..ba417f401e4 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -24,14 +24,19 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Sets; +import com.metamx.common.Pair; import com.metamx.common.guava.Comparators; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; import com.metamx.emitter.EmittingLogger; +import org.joda.time.Interval; +import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; @@ -52,14 +57,14 @@ public class DruidMasterBalancer implements DruidMasterHelper ); private static final EmittingLogger log = new EmittingLogger(DruidMasterBalancer.class); - private final BalancerAnalyzer analyzer; + private final BalancerCostAnalyzer analyzer; private final DruidMaster master; - private final Map> currentlyMovingSegments = Maps.newHashMap(); + private final Map> currentlyMovingSegments = Maps.newHashMap(); public DruidMasterBalancer( DruidMaster master, - BalancerAnalyzer analyzer + BalancerCostAnalyzer analyzer ) { this.master = master; @@ -68,7 +73,7 @@ public class DruidMasterBalancer implements DruidMasterHelper private void reduceLifetimes(String tier) { - for (BalancerSegmentHolder holder : currentlyMovingSegments.get(tier).values()) { + for (BalancerSegmentHolder2 holder : currentlyMovingSegments.get(tier).values()) { holder.reduceLifetime(); if (holder.getLifetime() <= 0) { log.makeAlert( @@ -92,7 +97,7 @@ public class DruidMasterBalancer implements DruidMasterHelper String tier = entry.getKey(); if (currentlyMovingSegments.get(tier) == null) { - currentlyMovingSegments.put(tier, new ConcurrentHashMap()); + currentlyMovingSegments.put(tier, new ConcurrentHashMap()); } if (!currentlyMovingSegments.get(tier).isEmpty()) { @@ -108,6 +113,21 @@ public class DruidMasterBalancer implements DruidMasterHelper TreeSet serversByPercentUsed = Sets.newTreeSet(percentUsedComparator); serversByPercentUsed.addAll(entry.getValue()); + List serverHolderList = new ArrayList(entry.getValue()); + + analyzer.init(serverHolderList); + log.info( + "Initial Total Cost: [%s]", + analyzer.getInitialTotalCost() + ); + moveSegments(analyzer.findSegmentsToMove(), params); + + stats.addToTieredStat("costChange", tier, (long) analyzer.getTotalCostChange()); + log.info( + "Cost Change: [%s]", + analyzer.getTotalCostChange() + ); + if (serversByPercentUsed.size() <= 1) { log.info( "[%s]: No unique values found for highest and lowest percent used servers: nothing to balance", @@ -119,6 +139,7 @@ public class DruidMasterBalancer implements DruidMasterHelper ServerHolder highestPercentUsedServer = serversByPercentUsed.first(); ServerHolder lowestPercentUsedServer = serversByPercentUsed.last(); + /* analyzer.init(highestPercentUsedServer, lowestPercentUsedServer); log.info( @@ -149,6 +170,8 @@ public class DruidMasterBalancer implements DruidMasterHelper analyzer.findSegmentsToMove(highestPercentUsedServer.getServer()), params ); + */ + stats.addToTieredStat("movedCount", tier, currentlyMovingSegments.get(tier).size()); } @@ -159,53 +182,55 @@ public class DruidMasterBalancer implements DruidMasterHelper } private void moveSegments( - final DruidServer server, - final Set segments, + final Set segments, final DruidMasterRuntimeParams params ) { - String toServer = server.getName(); - LoadQueuePeon toPeon = params.getLoadManagementPeons().get(toServer); - for (final BalancerSegmentHolder segment : Sets.newHashSet(segments)) { - String fromServer = segment.getServer().getName(); + for (final BalancerSegmentHolder2 segment : Sets.newHashSet(segments)) { + final DruidServer toServer = segment.getToServer(); + final String toServerName = segment.getToServer().getName(); + LoadQueuePeon toPeon = params.getLoadManagementPeons().get(toServerName); + + String fromServer = segment.getFromServer().getName(); DataSegment segmentToMove = segment.getSegment(); final String segmentName = segmentToMove.getIdentifier(); if (!toPeon.getSegmentsToLoad().contains(segmentToMove) && - (server.getSegment(segmentName) == null) && - new ServerHolder(server, toPeon).getAvailableSize() > segmentToMove.getSize()) { + (toServer.getSegment(segmentName) == null) && + new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) { log.info( "Moving [%s] from [%s] to [%s]", segmentName, fromServer, - toServer + toServerName ); try { master.moveSegment( fromServer, - toServer, + toServerName, segmentToMove.getIdentifier(), new LoadPeonCallback() { @Override protected void execute() { - Map movingSegments = currentlyMovingSegments.get(server.getTier()); + Map movingSegments = currentlyMovingSegments.get(toServer.getTier()); if (movingSegments != null) { movingSegments.remove(segmentName); } } } ); - currentlyMovingSegments.get(server.getTier()).put(segmentName, segment); + currentlyMovingSegments.get(toServer.getTier()).put(segmentName, segment); } catch (Exception e) { log.makeAlert(e, String.format("[%s] : Moving exception", segmentName)).emit(); } } else { - currentlyMovingSegments.get(server.getTier()).remove(segment); + currentlyMovingSegments.get(toServer.getTier()).remove(segment); } } + } } \ No newline at end of file diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 7e1304aba7a..21fb99dc637 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -27,11 +27,14 @@ import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; import junit.framework.Assert; import org.easymock.EasyMock; +import org.joda.time.Interval; import org.junit.After; import org.junit.Before; import org.junit.Test; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; /** */ @@ -46,6 +49,8 @@ public class DruidMasterBalancerTest private DataSegment segment4; private LoadQueuePeon peon; private DruidDataSource dataSource; + private Map segments; + private static final int DAY_IN_MILLIS = 1000 * 60 * 60 * 24; @Before public void setUp() throws Exception @@ -78,17 +83,25 @@ public class DruidMasterBalancerTest @Test public void testRun() { + Map segments = new HashMap(); + segments.put("segment1", segment1); + segments.put("segment2", segment2); + segments.put("segment3", segment3); + segments.put("segment4", segment4); + // Mock some servers of different usages EasyMock.expect(druidServerHigh.getName()).andReturn("from").atLeastOnce(); EasyMock.expect(druidServerHigh.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServerHigh.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServerHigh.getDataSources()).andReturn(Arrays.asList(dataSource)).atLeastOnce(); + EasyMock.expect(druidServerHigh.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); + EasyMock.expect(druidServerHigh.getSegments()).andReturn(segments).anyTimes(); EasyMock.replay(druidServerHigh); EasyMock.expect(druidServerLow.getName()).andReturn("to").atLeastOnce(); EasyMock.expect(druidServerLow.getTier()).andReturn("normal").atLeastOnce(); EasyMock.expect(druidServerLow.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServerLow.getMaxSize()).andReturn(100L).atLeastOnce(); + EasyMock.expect(druidServerLow.getSegments()).andReturn(new HashMap()).anyTimes(); EasyMock.expect(druidServerLow.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); EasyMock.expect(druidServerLow.getSegment("segment1")).andReturn(null).anyTimes(); EasyMock.expect(druidServerLow.getSegment("segment2")).andReturn(null).anyTimes(); @@ -104,21 +117,29 @@ public class DruidMasterBalancerTest segment3, segment4 ) - ).atLeastOnce(); + ).anyTimes(); EasyMock.replay(dataSource); // Mock some segments of different sizes EasyMock.expect(segment1.getSize()).andReturn(11L).anyTimes(); EasyMock.expect(segment1.getIdentifier()).andReturn("segment1").anyTimes(); + EasyMock.expect(segment1.getDataSource()).andReturn("datasource1").anyTimes(); + EasyMock.expect(segment1.getInterval()).andReturn(new Interval(0, 1 * DAY_IN_MILLIS)).anyTimes(); EasyMock.replay(segment1); EasyMock.expect(segment2.getSize()).andReturn(7L).anyTimes(); EasyMock.expect(segment2.getIdentifier()).andReturn("segment2").anyTimes(); + EasyMock.expect(segment2.getDataSource()).andReturn("datasource1").anyTimes(); + EasyMock.expect(segment2.getInterval()).andReturn(new Interval(10 * DAY_IN_MILLIS, 11 * DAY_IN_MILLIS)).anyTimes(); EasyMock.replay(segment2); EasyMock.expect(segment3.getSize()).andReturn(4L).anyTimes(); EasyMock.expect(segment3.getIdentifier()).andReturn("segment3").anyTimes(); + EasyMock.expect(segment3.getDataSource()).andReturn("datasource1").anyTimes(); + EasyMock.expect(segment3.getInterval()).andReturn(new Interval(0, 1 * DAY_IN_MILLIS)).anyTimes(); EasyMock.replay(segment3); EasyMock.expect(segment4.getSize()).andReturn(8L).anyTimes(); EasyMock.expect(segment4.getIdentifier()).andReturn("segment4").anyTimes(); + EasyMock.expect(segment4.getDataSource()).andReturn("datasource1").anyTimes(); + EasyMock.expect(segment4.getInterval()).andReturn(new Interval(10 * DAY_IN_MILLIS, 11 * DAY_IN_MILLIS)).anyTimes(); EasyMock.replay(segment4); // Mock stuff that the master needs @@ -154,7 +175,7 @@ public class DruidMasterBalancerTest .withLoadManagementPeons(ImmutableMap.of("from", peon, "to", peon)) .build(); - params = new DruidMasterBalancer(master, new BalancerAnalyzer()).run(params); + params = new DruidMasterBalancer(master, new BalancerCostAnalyzer()).run(params); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); } } From 3eda9dc5f2fb122885477959a994b5100353294c Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 31 Dec 2012 11:08:37 -0800 Subject: [PATCH 002/121] working balancercostanalyzer --- .../druid/master/BalancerCostAnalyzer.java | 79 ++++--------------- .../druid/master/DruidMasterBalancerTest.java | 7 ++ 2 files changed, 22 insertions(+), 64 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 86b220420f9..5ca0bc2aa1c 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -52,7 +52,7 @@ public class BalancerCostAnalyzer private Random rand; public BalancerCostAnalyzer(){ - rand = new Random(); + rand = new Random(0); totalCostChange = 0f; } @@ -69,33 +69,8 @@ public class BalancerCostAnalyzer return totalCostChange; } - public float calculateInitialTotalCost(List serverHolderList){ - int cost = 0; - for (ServerHolder server : serverHolderList) { - DataSegment[] segments = server.getServer().getSegments().values().toArray(new DataSegment[]{}); - for (int i = 0; i < segments.length; ++i) { - for (int j = i; j < segments.length; ++j) { - cost += computeJointSegmentCosts(segments[i], segments[j]); - } - } - } - return cost; - } - - public float calculateTotalCostChange(List serverHolderList, Set segmentHoldersToMove){ - int cost = 0; - Iterator it = segmentHoldersToMove.iterator(); - while (it.hasNext()) { - BalancerSegmentHolder2 segmentHolder = (BalancerSegmentHolder2) it.next(); - for (DataSegment fromSegment : segmentHolder.getFromServer().getSegments().values()) { - cost -= computeJointSegmentCosts(segmentHolder.getSegment(), fromSegment); - } - for (DataSegment toSegment : segmentHolder.getToServer().getSegments().values()) { - cost += computeJointSegmentCosts(segmentHolder.getSegment(), toSegment); - } - return cost; - } - + private float calculateInitialTotalCost(List serverHolderList){ + float cost = 0; for (ServerHolder server : serverHolderList) { DataSegment[] segments = server.getServer().getSegments().values().toArray(new DataSegment[]{}); for (int i = 0; i < segments.length; ++i) { @@ -130,8 +105,9 @@ public class BalancerCostAnalyzer Set movingSegments = Sets.newHashSet(); int counter = 0; + float currCost = 0f; - while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 2 * MAX_SEGMENTS_TO_MOVE) { + while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 3 * MAX_SEGMENTS_TO_MOVE) { counter++; ServerHolder fromServerHolder = serverHolderList.get(rand.nextInt(serverHolderList.size())); List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); @@ -139,7 +115,7 @@ public class BalancerCostAnalyzer DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); if (movingSegments.contains(proposalSegment)) continue; - //Just need a regular priority queue for the min. element. + // Just need a regular priority queue for the min. element. MinMaxPriorityQueue> pQueue = MinMaxPriorityQueue.orderedBy( new Comparator>() { @@ -160,7 +136,10 @@ public class BalancerCostAnalyzer cost += computeJointSegmentCosts(proposalSegment, segment); } - //Take into account costs of segments that will be moved. + // self cost + if (!server.getServer().equals(fromServerHolder.getServer())) cost += computeJointSegmentCosts(proposalSegment, proposalSegment); + + // Take into account costs of segments that will be moved. Iterator it = segmentHoldersToMove.iterator(); while (it.hasNext()) { BalancerSegmentHolder2 segmentToMove = (BalancerSegmentHolder2) it.next(); @@ -172,6 +151,10 @@ public class BalancerCostAnalyzer } } + if (server.getServer().equals(fromServerHolder.getServer())){ + currCost = cost; + } + pQueue.add(Pair.of(cost, server)); } @@ -185,44 +168,12 @@ public class BalancerCostAnalyzer proposalSegment ) ); - totalCostChange += minPair.lhs; + totalCostChange += currCost - minPair.lhs; } } - totalCostChange = calculateTotalCostChange(serverHolderList, segmentHoldersToMove); return segmentHoldersToMove; - - /* - double currPercentDiff = getPercentDiff(); - - if (currPercentDiff < PERCENT_THRESHOLD) { - log.info("Cluster usage is balanced."); - return segmentsToMove; - } - - List dataSources = Lists.newArrayList(server.getDataSources()); - Collections.shuffle(dataSources); - - for (DruidDataSource dataSource : dataSources) { - List segments = Lists.newArrayList(dataSource.getSegments()); - Collections.shuffle(segments); - - for (DataSegment segment : segments) { - if (segmentsToMove.size() >= MAX_SEGMENTS_TO_MOVE) { - return segmentsToMove; - } - - if (getLookaheadPercentDiff(highestSizeUsed - segment.getSize(), lowestSizeUsed + segment.getSize()) - < currPercentDiff) { - segmentsToMove.add(new BalancerSegmentHolder(server, segment)); - update(highestSizeUsed - segment.getSize(), lowestSizeUsed + segment.getSize()); - } - } - } - - return segmentsToMove; - */ } } diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 21fb99dc637..f3b4e8aae1c 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -120,6 +120,13 @@ public class DruidMasterBalancerTest ).anyTimes(); EasyMock.replay(dataSource); + /* + (2, 1, 2, 1 + -, 2, 1, 2 + -, -, 2, 1 + -, -, -, 2) + */ + // Mock some segments of different sizes EasyMock.expect(segment1.getSize()).andReturn(11L).anyTimes(); EasyMock.expect(segment1.getIdentifier()).andReturn("segment1").anyTimes(); From 72758e1345eb19359b4f748cbc819a20085f1061 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 31 Dec 2012 13:55:48 -0800 Subject: [PATCH 003/121] update --- .../java/com/metamx/druid/master/BalancerCostAnalyzer.java | 6 +++--- .../com/metamx/druid/master/DruidMasterBalancerTest.java | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 5ca0bc2aa1c..5bd25ed487a 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -45,12 +45,13 @@ public class BalancerCostAnalyzer private static final Logger log = new Logger(BalancerCostAnalyzer.class); private static final int MAX_SEGMENTS_TO_MOVE = 5; private static final int DAY_IN_MILLIS = 1000 * 60 * 60 * 24; - private float initialTotalCost; - private float totalCostChange; private List serverHolderList; private Random rand; + private float initialTotalCost; + private float totalCostChange; + public BalancerCostAnalyzer(){ rand = new Random(0); totalCostChange = 0f; @@ -170,7 +171,6 @@ public class BalancerCostAnalyzer ); totalCostChange += currCost - minPair.lhs; } - } return segmentHoldersToMove; diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index f3b4e8aae1c..ba1b71c8316 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -184,5 +184,6 @@ public class DruidMasterBalancerTest params = new DruidMasterBalancer(master, new BalancerCostAnalyzer()).run(params); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); + Assert.assertTrue(params.getMasterStats().getPerTierStats().get("costChange").get("normal").get() > 0); } } From bb31fa7b68642304317a751d9e06a3c117c024a3 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Thu, 3 Jan 2013 10:48:05 -0800 Subject: [PATCH 004/121] update --- .../druid/master/BalancerCostAnalyzer.java | 34 +++++++++---------- .../druid/master/DruidMasterBalancer.java | 11 +++--- .../metamx/druid/master/rules/LoadRule.java | 3 ++ 3 files changed, 26 insertions(+), 22 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 5bd25ed487a..9b0cd4b9c08 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -49,12 +49,12 @@ public class BalancerCostAnalyzer private List serverHolderList; private Random rand; - private float initialTotalCost; - private float totalCostChange; + private double initialTotalCost; + private double totalCostChange; public BalancerCostAnalyzer(){ rand = new Random(0); - totalCostChange = 0f; + totalCostChange = 0; } public void init(List serverHolderList){ @@ -62,16 +62,16 @@ public class BalancerCostAnalyzer this.serverHolderList = serverHolderList; } - public float getInitialTotalCost() { + public double getInitialTotalCost() { return initialTotalCost; } - public float getTotalCostChange() { + public double getTotalCostChange() { return totalCostChange; } - private float calculateInitialTotalCost(List serverHolderList){ - float cost = 0; + private double calculateInitialTotalCost(List serverHolderList){ + double cost = 0; for (ServerHolder server : serverHolderList) { DataSegment[] segments = server.getServer().getSegments().values().toArray(new DataSegment[]{}); for (int i = 0; i < segments.length; ++i) { @@ -83,8 +83,8 @@ public class BalancerCostAnalyzer return cost; } - public float computeJointSegmentCosts(DataSegment segment1, DataSegment segment2){ - float cost = 0f; + public double computeJointSegmentCosts(DataSegment segment1, DataSegment segment2){ + double cost = 0; Interval gap = segment1.getInterval().gap(segment2.getInterval()); // gap is null if the two segment intervals overlap or if they're adjacent @@ -106,7 +106,7 @@ public class BalancerCostAnalyzer Set movingSegments = Sets.newHashSet(); int counter = 0; - float currCost = 0f; + double currCost = 0; while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 3 * MAX_SEGMENTS_TO_MOVE) { counter++; @@ -117,22 +117,22 @@ public class BalancerCostAnalyzer if (movingSegments.contains(proposalSegment)) continue; // Just need a regular priority queue for the min. element. - MinMaxPriorityQueue> pQueue = MinMaxPriorityQueue.orderedBy( - new Comparator>() + MinMaxPriorityQueue> pQueue = MinMaxPriorityQueue.orderedBy( + new Comparator>() { @Override public int compare( - Pair o, - Pair o1 + Pair o, + Pair o1 ) { - return Float.compare(o.lhs, o1.lhs); + return Double.compare(o.lhs, o1.lhs); } } ).create(); for (ServerHolder server : serverHolderList) { - float cost = 0f; + double cost = 0f; for (DataSegment segment : server.getServer().getSegments().values()) { cost += computeJointSegmentCosts(proposalSegment, segment); } @@ -159,7 +159,7 @@ public class BalancerCostAnalyzer pQueue.add(Pair.of(cost, server)); } - Pair minPair = pQueue.peekFirst(); + Pair minPair = pQueue.peekFirst(); if (!minPair.rhs.equals(fromServerHolder)) { movingSegments.add(proposalSegment); segmentHoldersToMove.add( diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index ba417f401e4..81c3b382bca 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -110,8 +110,8 @@ public class DruidMasterBalancer implements DruidMasterHelper continue; } - TreeSet serversByPercentUsed = Sets.newTreeSet(percentUsedComparator); - serversByPercentUsed.addAll(entry.getValue()); + //TreeSet serversByPercentUsed = Sets.newTreeSet(percentUsedComparator); + //serversByPercentUsed.addAll(entry.getValue()); List serverHolderList = new ArrayList(entry.getValue()); @@ -128,18 +128,19 @@ public class DruidMasterBalancer implements DruidMasterHelper analyzer.getTotalCostChange() ); - if (serversByPercentUsed.size() <= 1) { + if (serverHolderList.size() <= 1) { log.info( - "[%s]: No unique values found for highest and lowest percent used servers: nothing to balance", + "[%s]: One or fewer servers found. Cannot balance.", tier ); continue; } + /* ServerHolder highestPercentUsedServer = serversByPercentUsed.first(); ServerHolder lowestPercentUsedServer = serversByPercentUsed.last(); - /* + analyzer.init(highestPercentUsedServer, lowestPercentUsedServer); log.info( diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index 5bf232c9474..c0844641b5d 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -30,6 +30,7 @@ import com.metamx.druid.master.MasterStats; import com.metamx.druid.master.ServerHolder; import com.metamx.emitter.EmittingLogger; +import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -70,6 +71,8 @@ public abstract class LoadRule implements Rule { MasterStats stats = new MasterStats(); + List serverHolderList = new ArrayList(serverQueue); + List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { ServerHolder holder = serverQueue.pollFirst(); From 117cac2795b67e8bcfcac5aa3de09e3705ea132f Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Thu, 3 Jan 2013 14:12:10 -0800 Subject: [PATCH 005/121] update --- .../metamx/druid/master/BalancerAnalyzer.java | 131 ------------- .../druid/master/BalancerCostAnalyzer.java | 173 ++++++++++++++---- .../druid/master/BalancerSegmentHolder.java | 18 +- .../druid/master/BalancerSegmentHolder2.java | 70 ------- .../druid/master/DruidMasterBalancer.java | 59 +----- .../metamx/druid/master/rules/LoadRule.java | 9 +- .../druid/master/BalancerAnalyzerTest.java | 131 ------------- 7 files changed, 166 insertions(+), 425 deletions(-) delete mode 100644 server/src/main/java/com/metamx/druid/master/BalancerAnalyzer.java delete mode 100644 server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder2.java delete mode 100644 server/src/test/java/com/metamx/druid/master/BalancerAnalyzerTest.java diff --git a/server/src/main/java/com/metamx/druid/master/BalancerAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerAnalyzer.java deleted file mode 100644 index bdf475b8d1b..00000000000 --- a/server/src/main/java/com/metamx/druid/master/BalancerAnalyzer.java +++ /dev/null @@ -1,131 +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.master; - -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidDataSource; -import com.metamx.druid.client.DruidServer; - -import java.util.Collections; -import java.util.List; -import java.util.Set; - -/** - * The BalancerAnalyzer keeps the state of the highest and lowest percent used servers. It will update - * these states and perform lookaheads to make sure the updated states result in a more balanced cluster. - */ -public class BalancerAnalyzer -{ - private static final Logger log = new Logger(BalancerAnalyzer.class); - private static final int PERCENT_THRESHOLD = 3; - private static final int MAX_SEGMENTS_TO_MOVE = 5; - - private volatile Long highestSizeUsed; - private volatile double highestPercentUsed; - private volatile Long highestPercentUsedServerMaxSize; - private volatile Long lowestSizeUsed; - private volatile double lowestPercentUsed; - private volatile Long lowestPercentUsedServerMaxSize; - - public BalancerAnalyzer() - { - this.highestSizeUsed = 0L; - this.highestPercentUsed = 0; - this.highestPercentUsedServerMaxSize = 0L; - this.lowestSizeUsed = 0L; - this.lowestPercentUsed = 0; - this.lowestPercentUsedServerMaxSize = 0L; - } - - public void init(ServerHolder highestPercentUsedServer, ServerHolder lowestPercentUsedServer) - { - highestSizeUsed = highestPercentUsedServer.getSizeUsed(); - highestPercentUsed = highestPercentUsedServer.getPercentUsed(); - highestPercentUsedServerMaxSize = highestPercentUsedServer.getMaxSize(); - lowestSizeUsed = lowestPercentUsedServer.getSizeUsed(); - lowestPercentUsed = lowestPercentUsedServer.getPercentUsed(); - lowestPercentUsedServerMaxSize = lowestPercentUsedServer.getMaxSize(); - } - - public void update(long newHighestSizeUsed, long newLowestSizedUsed) - { - highestSizeUsed = newHighestSizeUsed; - highestPercentUsed = highestSizeUsed.doubleValue() / highestPercentUsedServerMaxSize; - lowestSizeUsed = newLowestSizedUsed; - lowestPercentUsed = lowestSizeUsed.doubleValue() / lowestPercentUsedServerMaxSize; - } - - public double getPercentDiff() - { - return Math.abs( - 100 * ((highestPercentUsed - lowestPercentUsed) - / ((highestPercentUsed + lowestPercentUsed) / 2)) - ); - } - - public double getLookaheadPercentDiff(Long newHighestSizeUsed, Long newLowestSizedUsed) - { - double newHighestPercentUsed = 100 * (newHighestSizeUsed.doubleValue() / highestPercentUsedServerMaxSize); - double newLowestPercentUsed = 100 * (newLowestSizedUsed.doubleValue() / lowestPercentUsedServerMaxSize); - - return Math.abs( - 100 * ((newHighestPercentUsed - newLowestPercentUsed) - / ((newHighestPercentUsed + newLowestPercentUsed) / 2)) - ); - } - - public Set findSegmentsToMove(DruidServer server) - { - Set segmentsToMove = Sets.newHashSet(); - double currPercentDiff = getPercentDiff(); - - if (currPercentDiff < PERCENT_THRESHOLD) { - log.info("Cluster usage is balanced."); - return segmentsToMove; - } - - List dataSources = Lists.newArrayList(server.getDataSources()); - Collections.shuffle(dataSources); - - for (DruidDataSource dataSource : dataSources) { - List segments = Lists.newArrayList(dataSource.getSegments()); - Collections.shuffle(segments); - - for (DataSegment segment : segments) { - if (segmentsToMove.size() >= MAX_SEGMENTS_TO_MOVE) { - return segmentsToMove; - } - - if (getLookaheadPercentDiff(highestSizeUsed - segment.getSize(), lowestSizeUsed + segment.getSize()) - < currPercentDiff) { - segmentsToMove.add(new BalancerSegmentHolder(server, segment)); - update(highestSizeUsed - segment.getSize(), lowestSizeUsed + segment.getSize()); - } - } - } - - return segmentsToMove; - } -} - - diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 9b0cd4b9c08..0c68ae8385b 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -25,20 +25,19 @@ import com.google.common.collect.Sets; import com.metamx.common.Pair; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidDataSource; -import com.metamx.druid.client.DruidServer; import org.joda.time.Interval; -import java.util.ArrayList; -import java.util.Collections; import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Random; import java.util.Set; + /** - * The BalancerCostAnalyzer + * The BalancerCostAnalyzer will compute the total initial cost of the cluster, with costs defined in + * computeJointSegmentCosts. It will then propose to move randomly chosen segments from their respective + * initial servers to other servers, chosen greedily to minimize the cost of the cluster. */ public class BalancerCostAnalyzer { @@ -52,25 +51,30 @@ public class BalancerCostAnalyzer private double initialTotalCost; private double totalCostChange; - public BalancerCostAnalyzer(){ + public BalancerCostAnalyzer() + { rand = new Random(0); totalCostChange = 0; } - public void init(List serverHolderList){ + public void init(List serverHolderList) + { this.initialTotalCost = calculateInitialTotalCost(serverHolderList); this.serverHolderList = serverHolderList; } - public double getInitialTotalCost() { + public double getInitialTotalCost() + { return initialTotalCost; } - public double getTotalCostChange() { + public double getTotalCostChange() + { return totalCostChange; } - private double calculateInitialTotalCost(List serverHolderList){ + private double calculateInitialTotalCost(List serverHolderList) + { double cost = 0; for (ServerHolder server : serverHolderList) { DataSegment[] segments = server.getServer().getSegments().values().toArray(new DataSegment[]{}); @@ -83,7 +87,8 @@ public class BalancerCostAnalyzer return cost; } - public double computeJointSegmentCosts(DataSegment segment1, DataSegment segment2){ + public double computeJointSegmentCosts(DataSegment segment1, DataSegment segment2) + { double cost = 0; Interval gap = segment1.getInterval().gap(segment2.getInterval()); @@ -92,32 +97,88 @@ public class BalancerCostAnalyzer cost += 1f; } else { long gapMillis = gap.toDurationMillis(); - if (gapMillis < DAY_IN_MILLIS) cost += 1f; + if (gapMillis < DAY_IN_MILLIS) { + cost += 1f; + } } - if(segment1.getDataSource().equals(segment2.getDataSource())) cost += 1f; + if (segment1.getDataSource().equals(segment2.getDataSource())) { + cost += 1f; + } return cost; } - public Set findSegmentsToMove() + public class NullServerHolder extends ServerHolder { - Set segmentHoldersToMove = Sets.newHashSet(); - Set movingSegments = Sets.newHashSet(); + public NullServerHolder() + { + super(null, null); + } - int counter = 0; - double currCost = 0; + @Override + public boolean equals(Object o) + { + return false; + } + } - while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 3 * MAX_SEGMENTS_TO_MOVE) { - counter++; - ServerHolder fromServerHolder = serverHolderList.get(rand.nextInt(serverHolderList.size())); - List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); - if (segments.size() == 0) continue; - DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); - if (movingSegments.contains(proposalSegment)) continue; + public class BalancerCostAnalyzerHelper + { + private MinMaxPriorityQueue> costsServerHolderPairs; + private List serverHolderList; + private DataSegment proposalSegment; + private ServerHolder fromServerHolder; + private Set segmentHoldersToMove; + private double currCost; + public MinMaxPriorityQueue> getCostsServerHolderPairs() + { + return costsServerHolderPairs; + } + + public List getServerHolderList() + { + return serverHolderList; + } + + public DataSegment getProposalSegment() + { + return proposalSegment; + } + + public ServerHolder getFromServerHolder() + { + return fromServerHolder; + } + + public Set getSegmentHoldersToMove() + { + return segmentHoldersToMove; + } + + public double getCurrCost() + { + return currCost; + } + + public BalancerCostAnalyzerHelper( + List serverHolderList, + DataSegment proposalSegment + ) + { + this(serverHolderList, proposalSegment, new NullServerHolder(), Sets.newHashSet()); + } + + public BalancerCostAnalyzerHelper( + List serverHolderList, + DataSegment proposalSegment, + ServerHolder fromServerHolder, + Set segmentHoldersToMove + ) + { // Just need a regular priority queue for the min. element. - MinMaxPriorityQueue> pQueue = MinMaxPriorityQueue.orderedBy( + this.costsServerHolderPairs = MinMaxPriorityQueue.orderedBy( new Comparator>() { @Override @@ -130,7 +191,15 @@ public class BalancerCostAnalyzer } } ).create(); + this.serverHolderList = serverHolderList; + this.proposalSegment = proposalSegment; + this.fromServerHolder = fromServerHolder; + this.segmentHoldersToMove = segmentHoldersToMove; + this.currCost = 0; + } + public void computeAllCosts() + { for (ServerHolder server : serverHolderList) { double cost = 0f; for (DataSegment segment : server.getServer().getSegments().values()) { @@ -138,12 +207,14 @@ public class BalancerCostAnalyzer } // self cost - if (!server.getServer().equals(fromServerHolder.getServer())) cost += computeJointSegmentCosts(proposalSegment, proposalSegment); + if (!server.getServer().equals(fromServerHolder.getServer())) { + cost += computeJointSegmentCosts(proposalSegment, proposalSegment); + } // Take into account costs of segments that will be moved. Iterator it = segmentHoldersToMove.iterator(); while (it.hasNext()) { - BalancerSegmentHolder2 segmentToMove = (BalancerSegmentHolder2) it.next(); + BalancerSegmentHolder segmentToMove = (BalancerSegmentHolder) it.next(); if (server.getServer().equals(segmentToMove.getToServer())) { cost += computeJointSegmentCosts(proposalSegment, segmentToMove.getSegment()); } @@ -152,24 +223,58 @@ public class BalancerCostAnalyzer } } - if (server.getServer().equals(fromServerHolder.getServer())){ + if (server.getServer().equals(fromServerHolder.getServer())) { currCost = cost; } - pQueue.add(Pair.of(cost, server)); + if (proposalSegment.getSize() < server.getAvailableSize()) { + costsServerHolderPairs.add(Pair.of(cost, server)); + } + + } + } + + } + + public Set findSegmentsToMove() + { + Set segmentHoldersToMove = Sets.newHashSet(); + Set movingSegments = Sets.newHashSet(); + + int counter = 0; + + while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 3 * MAX_SEGMENTS_TO_MOVE) { + counter++; + ServerHolder fromServerHolder = serverHolderList.get(rand.nextInt(serverHolderList.size())); + List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); + if (segments.size() == 0) { + continue; + } + DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); + if (movingSegments.contains(proposalSegment)) { + continue; } - Pair minPair = pQueue.peekFirst(); - if (!minPair.rhs.equals(fromServerHolder)) { + BalancerCostAnalyzerHelper helper = new BalancerCostAnalyzerHelper( + serverHolderList, + proposalSegment, + fromServerHolder, + segmentHoldersToMove + ); + helper.computeAllCosts(); + + Pair minPair = helper.getCostsServerHolderPairs().pollFirst(); + + if (minPair.rhs != null && !minPair.rhs.equals(fromServerHolder)) { movingSegments.add(proposalSegment); segmentHoldersToMove.add( - new BalancerSegmentHolder2( + new BalancerSegmentHolder( fromServerHolder.getServer(), minPair.rhs.getServer(), proposalSegment ) ); - totalCostChange += currCost - minPair.lhs; + totalCostChange += helper.getCurrCost() - minPair.lhs; } } diff --git a/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java b/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java index 97beb20ac59..5e7a9369fe4 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java @@ -26,23 +26,31 @@ import com.metamx.druid.client.DruidServer; */ public class BalancerSegmentHolder { - private final DruidServer server; + private final DruidServer fromServer; + private final DruidServer toServer; private final DataSegment segment; private volatile int lifetime = 15; public BalancerSegmentHolder( - DruidServer server, + DruidServer fromServer, + DruidServer toServer, DataSegment segment ) { - this.server = server; + this.fromServer = fromServer; + this.toServer = toServer; this.segment = segment; } - public DruidServer getServer() + public DruidServer getFromServer() { - return server; + return fromServer; + } + + public DruidServer getToServer() + { + return toServer; } public DataSegment getSegment() diff --git a/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder2.java b/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder2.java deleted file mode 100644 index 22277285ee0..00000000000 --- a/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder2.java +++ /dev/null @@ -1,70 +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.master; - -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; - -/** - */ -public class BalancerSegmentHolder2 -{ - private final DruidServer fromServer; - private final DruidServer toServer; - private final DataSegment segment; - - private volatile int lifetime = 15; - - public BalancerSegmentHolder2( - DruidServer fromServer, - DruidServer toServer, - DataSegment segment - ) - { - this.fromServer = fromServer; - this.toServer = toServer; - this.segment = segment; - } - - public DruidServer getFromServer() - { - return fromServer; - } - - public DruidServer getToServer() - { - return toServer; - } - - public DataSegment getSegment() - { - return segment; - } - - public int getLifetime() - { - return lifetime; - } - - public void reduceLifetime() - { - lifetime--; - } -} diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 81c3b382bca..4e0bfdb1db2 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -20,25 +20,19 @@ package com.metamx.druid.master; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Sets; -import com.metamx.common.Pair; import com.metamx.common.guava.Comparators; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; import com.metamx.emitter.EmittingLogger; -import org.joda.time.Interval; import java.util.ArrayList; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; /** @@ -60,7 +54,7 @@ public class DruidMasterBalancer implements DruidMasterHelper private final BalancerCostAnalyzer analyzer; private final DruidMaster master; - private final Map> currentlyMovingSegments = Maps.newHashMap(); + private final Map> currentlyMovingSegments = Maps.newHashMap(); public DruidMasterBalancer( DruidMaster master, @@ -73,7 +67,7 @@ public class DruidMasterBalancer implements DruidMasterHelper private void reduceLifetimes(String tier) { - for (BalancerSegmentHolder2 holder : currentlyMovingSegments.get(tier).values()) { + for (BalancerSegmentHolder holder : currentlyMovingSegments.get(tier).values()) { holder.reduceLifetime(); if (holder.getLifetime() <= 0) { log.makeAlert( @@ -97,7 +91,7 @@ public class DruidMasterBalancer implements DruidMasterHelper String tier = entry.getKey(); if (currentlyMovingSegments.get(tier) == null) { - currentlyMovingSegments.put(tier, new ConcurrentHashMap()); + currentlyMovingSegments.put(tier, new ConcurrentHashMap()); } if (!currentlyMovingSegments.get(tier).isEmpty()) { @@ -110,9 +104,6 @@ public class DruidMasterBalancer implements DruidMasterHelper continue; } - //TreeSet serversByPercentUsed = Sets.newTreeSet(percentUsedComparator); - //serversByPercentUsed.addAll(entry.getValue()); - List serverHolderList = new ArrayList(entry.getValue()); analyzer.init(serverHolderList); @@ -136,44 +127,6 @@ public class DruidMasterBalancer implements DruidMasterHelper continue; } - /* - ServerHolder highestPercentUsedServer = serversByPercentUsed.first(); - ServerHolder lowestPercentUsedServer = serversByPercentUsed.last(); - - - analyzer.init(highestPercentUsedServer, lowestPercentUsedServer); - - log.info( - "[%s]: Percent difference in percent size used between highest/lowest servers: %s%%", - tier, - analyzer.getPercentDiff() - ); - - log.info( - "[%s]: Highest percent used [%s]: size used[%s], percent used[%s%%]", - tier, - highestPercentUsedServer.getServer().getName(), - highestPercentUsedServer.getSizeUsed(), - highestPercentUsedServer.getPercentUsed() - ); - - log.info( - "[%s]: Lowest percent used [%s]: size used[%s], percent used[%s%%]", - tier, - lowestPercentUsedServer.getServer().getName(), - lowestPercentUsedServer.getSizeUsed(), - lowestPercentUsedServer.getPercentUsed() - ); - - // Use the analyzer to find segments to move and then move them - moveSegments( - lowestPercentUsedServer.getServer(), - analyzer.findSegmentsToMove(highestPercentUsedServer.getServer()), - params - ); - */ - - stats.addToTieredStat("movedCount", tier, currentlyMovingSegments.get(tier).size()); } @@ -183,12 +136,12 @@ public class DruidMasterBalancer implements DruidMasterHelper } private void moveSegments( - final Set segments, + final Set segments, final DruidMasterRuntimeParams params ) { - for (final BalancerSegmentHolder2 segment : Sets.newHashSet(segments)) { + for (final BalancerSegmentHolder segment : Sets.newHashSet(segments)) { final DruidServer toServer = segment.getToServer(); final String toServerName = segment.getToServer().getName(); LoadQueuePeon toPeon = params.getLoadManagementPeons().get(toServerName); @@ -216,7 +169,7 @@ public class DruidMasterBalancer implements DruidMasterHelper @Override protected void execute() { - Map movingSegments = currentlyMovingSegments.get(toServer.getTier()); + Map movingSegments = currentlyMovingSegments.get(toServer.getTier()); if (movingSegments != null) { movingSegments.remove(segmentName); } diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index c0844641b5d..2d26be1745b 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -22,7 +22,9 @@ package com.metamx.druid.master.rules; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; +import com.metamx.common.Pair; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.master.BalancerCostAnalyzer; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterRuntimeParams; import com.metamx.druid.master.LoadPeonCallback; @@ -75,7 +77,12 @@ public abstract class LoadRule implements Rule List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { - ServerHolder holder = serverQueue.pollFirst(); + BalancerCostAnalyzer analyzer = new BalancerCostAnalyzer(); + BalancerCostAnalyzer.BalancerCostAnalyzerHelper helper = analyzer.new BalancerCostAnalyzerHelper(serverHolderList, segment); + helper.computeAllCosts(); + Pair minPair = helper.getCostsServerHolderPairs().pollFirst(); + + ServerHolder holder = minPair.rhs; if (holder == null) { log.warn( "Not enough %s servers[%d] to assign segment[%s]! Expected Replicants[%d]", diff --git a/server/src/test/java/com/metamx/druid/master/BalancerAnalyzerTest.java b/server/src/test/java/com/metamx/druid/master/BalancerAnalyzerTest.java deleted file mode 100644 index 5c5c96b520d..00000000000 --- a/server/src/test/java/com/metamx/druid/master/BalancerAnalyzerTest.java +++ /dev/null @@ -1,131 +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.master; - -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidDataSource; -import com.metamx.druid.client.DruidServer; -import junit.framework.Assert; -import org.easymock.EasyMock; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -/** - */ -public class BalancerAnalyzerTest -{ - private ServerHolder high; - private ServerHolder low; - private DruidServer server; - private DruidDataSource dataSource; - private DataSegment segment; - - @Before - public void setUp() throws Exception - { - high = EasyMock.createMock(ServerHolder.class); - low = EasyMock.createMock(ServerHolder.class); - server = EasyMock.createMock(DruidServer.class); - dataSource = EasyMock.createMock(DruidDataSource.class); - segment = EasyMock.createMock(DataSegment.class); - } - - @After - public void tearDown() throws Exception - { - EasyMock.verify(high); - EasyMock.verify(low); - } - - @Test - public void testGetPercentDifference() - { - EasyMock.expect(high.getSizeUsed()).andReturn(6L); - EasyMock.expect(high.getPercentUsed()).andReturn(60.0); - EasyMock.expect(high.getMaxSize()).andReturn(10L); - EasyMock.replay(high); - - EasyMock.expect(low.getSizeUsed()).andReturn(2L); - EasyMock.expect(low.getPercentUsed()).andReturn(20.0); - EasyMock.expect(low.getMaxSize()).andReturn(10L); - EasyMock.replay(low); - - BalancerAnalyzer analyzer = new BalancerAnalyzer(); - - analyzer.init(high, low); - Assert.assertEquals(100.0, analyzer.getPercentDiff()); - } - - @Test - public void testGetLookaheadPercentDifference() - { - EasyMock.expect(high.getSizeUsed()).andReturn(2L); - EasyMock.expect(high.getPercentUsed()).andReturn(20.0); - EasyMock.expect(high.getMaxSize()).andReturn(10L); - EasyMock.replay(high); - - EasyMock.expect(low.getSizeUsed()).andReturn(1L); - EasyMock.expect(low.getPercentUsed()).andReturn(10.0); - EasyMock.expect(low.getMaxSize()).andReturn(10L); - EasyMock.replay(low); - - BalancerAnalyzer analyzer = new BalancerAnalyzer(); - - analyzer.init(high, low); - Assert.assertEquals(100.0, analyzer.getLookaheadPercentDiff(2L, 6L)); - } - - @Test - public void testFindSegmentsToMove() - { - EasyMock.expect(high.getSizeUsed()).andReturn(6L); - EasyMock.expect(high.getPercentUsed()).andReturn(60.0); - EasyMock.expect(high.getMaxSize()).andReturn(10L); - EasyMock.replay(high); - - EasyMock.expect(low.getSizeUsed()).andReturn(2L); - EasyMock.expect(low.getPercentUsed()).andReturn(20.0); - EasyMock.expect(low.getMaxSize()).andReturn(10L); - EasyMock.replay(low); - - EasyMock.expect(segment.getSize()).andReturn(1L).atLeastOnce(); - EasyMock.replay(segment); - - EasyMock.expect(dataSource.getSegments()).andReturn(Sets.newHashSet(segment)); - EasyMock.replay(dataSource); - - EasyMock.expect(server.getDataSources()).andReturn(Lists.newArrayList(dataSource)); - EasyMock.replay(server); - - BalancerAnalyzer analyzer = new BalancerAnalyzer(); - - analyzer.init(high, low); - - Assert.assertEquals(analyzer.findSegmentsToMove(server).iterator().next().getSegment(), segment); - - EasyMock.verify(server); - EasyMock.verify(dataSource); - EasyMock.verify(segment); - } - -} From 402ddbf26150894d8789c3d083af3f06635271c3 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Fri, 4 Jan 2013 11:20:06 -0800 Subject: [PATCH 006/121] update --- .../druid/master/BalancerCostAnalyzer.java | 113 +++++++++++++++--- .../com/metamx/druid/master/DruidMaster.java | 3 +- .../druid/master/DruidMasterBalancer.java | 13 ++ .../metamx/druid/master/rules/LoadRule.java | 3 +- .../druid/master/DruidMasterBalancerTest.java | 3 +- 5 files changed, 116 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 0c68ae8385b..94a55363bc8 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -25,6 +25,8 @@ import com.google.common.collect.Sets; import com.metamx.common.Pair; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.DruidServer; +import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.Comparator; @@ -36,23 +38,28 @@ import java.util.Set; /** * The BalancerCostAnalyzer will compute the total initial cost of the cluster, with costs defined in - * computeJointSegmentCosts. It will then propose to move randomly chosen segments from their respective - * initial servers to other servers, chosen greedily to minimize the cost of the cluster. + * computeJointSegmentCosts. It will then propose to move (pseudo-)randomly chosen segments from their + * respective initial servers to other servers, chosen greedily to minimize the cost of the cluster. */ public class BalancerCostAnalyzer { private static final Logger log = new Logger(BalancerCostAnalyzer.class); private static final int MAX_SEGMENTS_TO_MOVE = 5; private static final int DAY_IN_MILLIS = 1000 * 60 * 60 * 24; + private static final int SEVEN_DAYS_IN_MILLIS = 7 * DAY_IN_MILLIS; + private static final int THIRTY_DAYS_IN_MILLIS = 30 * DAY_IN_MILLIS; private List serverHolderList; private Random rand; + private DateTime referenceTimestamp; private double initialTotalCost; + private double normalization; private double totalCostChange; - public BalancerCostAnalyzer() + public BalancerCostAnalyzer(DateTime referenceTimestamp) { + this.referenceTimestamp = referenceTimestamp; rand = new Random(0); totalCostChange = 0; } @@ -60,6 +67,7 @@ public class BalancerCostAnalyzer public void init(List serverHolderList) { this.initialTotalCost = calculateInitialTotalCost(serverHolderList); + this.normalization = calculateNormalization(serverHolderList); this.serverHolderList = serverHolderList; } @@ -68,11 +76,38 @@ public class BalancerCostAnalyzer return initialTotalCost; } + public double getNormalization() + { + return normalization; + } + + public double getNormalizedInitialCost() + { + return initialTotalCost / normalization; + } + public double getTotalCostChange() { return totalCostChange; } + /* + * Calculates the cost normalization. This is such that the normalized cost is lower bounded + * by 1 (e.g. when each segment gets its own compute node). + */ + private double calculateNormalization(List serverHolderList) + { + double cost = 0; + for (ServerHolder server : serverHolderList) { + DataSegment[] segments = server.getServer().getSegments().values().toArray(new DataSegment[]{}); + for (int i = 0; i < segments.length; ++i) { + cost += computeJointSegmentCosts(segments[i], segments[i]); + } + } + return cost; + } + + // Calculates the initial cost of the Druid segment configuration. private double calculateInitialTotalCost(List serverHolderList) { double cost = 0; @@ -87,28 +122,56 @@ public class BalancerCostAnalyzer return cost; } + /* + * This defines the unnormalized cost function between two segments. There is a base cost given by + * the minimum size of the two segments and additional penalties. + * recencyPenalty: it is more likely that recent segments will be queried together + * dataSourcePenalty: if two segments belong to the same data source, they are more likely to be involved + * in the same queries + * gapPenalty: it is more likely that segments close together in time will be queried together + */ public double computeJointSegmentCosts(DataSegment segment1, DataSegment segment2) { double cost = 0; Interval gap = segment1.getInterval().gap(segment2.getInterval()); + double baseCost = Math.min(segment1.getSize(), segment2.getSize()); + double recencyPenalty = 1; + double dataSourcePenalty = 1; + double gapPenalty = 1; + + if (segment1.getDataSource().equals(segment2.getDataSource())) { + dataSourcePenalty = 2; + } + + double maxDiff = Math.max( + referenceTimestamp.getMillis() - segment1.getInterval().getEndMillis(), + referenceTimestamp.getMillis() - segment2.getInterval().getEndMillis() + ); + if (maxDiff < SEVEN_DAYS_IN_MILLIS) { + recencyPenalty = 2 - maxDiff / SEVEN_DAYS_IN_MILLIS; + } + // gap is null if the two segment intervals overlap or if they're adjacent if (gap == null) { - cost += 1f; + gapPenalty = 2; } else { long gapMillis = gap.toDurationMillis(); - if (gapMillis < DAY_IN_MILLIS) { - cost += 1f; + if (gapMillis < THIRTY_DAYS_IN_MILLIS) { + gapPenalty = 2 - gapMillis / THIRTY_DAYS_IN_MILLIS; } } - if (segment1.getDataSource().equals(segment2.getDataSource())) { - cost += 1f; - } + cost = baseCost * recencyPenalty * dataSourcePenalty * gapPenalty; return cost; } + /* + * These could be anonymous in BalancerCostAnalyzerHelper + * Their purpose is to unify the balance/assignment code since a segment that has not yet been assigned + * does not have a source server. + */ public class NullServerHolder extends ServerHolder { public NullServerHolder() @@ -116,10 +179,24 @@ public class BalancerCostAnalyzer super(null, null); } - @Override - public boolean equals(Object o) + public class NullDruidServer extends DruidServer { - return false; + public NullDruidServer() + { + super(null, null, 0, null, null); + } + + @Override + public boolean equals(Object o) + { + return false; + } + } + + @Override + public DruidServer getServer() + { + return new NullDruidServer(); } } @@ -200,18 +277,20 @@ public class BalancerCostAnalyzer public void computeAllCosts() { + // The contribution to the total cost of a given server by proposing to move the segment to that server is... for (ServerHolder server : serverHolderList) { double cost = 0f; + // the sum of the costs of other (inclusive) segments on the server for (DataSegment segment : server.getServer().getSegments().values()) { cost += computeJointSegmentCosts(proposalSegment, segment); } - // self cost - if (!server.getServer().equals(fromServerHolder.getServer())) { + // plus the self cost if the proposed new server is different + if (!fromServerHolder.getServer().equals(server.getServer())) { cost += computeJointSegmentCosts(proposalSegment, proposalSegment); } - // Take into account costs of segments that will be moved. + // plus the costs of segments that will be moved. Iterator it = segmentHoldersToMove.iterator(); while (it.hasNext()) { BalancerSegmentHolder segmentToMove = (BalancerSegmentHolder) it.next(); @@ -223,10 +302,12 @@ public class BalancerCostAnalyzer } } - if (server.getServer().equals(fromServerHolder.getServer())) { + // currCost keeps track of the current cost for that server (so we can compute the cost change). + if (fromServerHolder.getServer().equals(server.getServer())) { currCost = cost; } + // Only enter the queue if the server has enough size. if (proposalSegment.getSize() < server.getAvailableSize()) { costsServerHolderPairs.add(Pair.of(cost, server)); } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 4dfbb6018bb..ff01957d102 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -52,6 +52,7 @@ import com.metamx.phonebook.PhoneBookPeon; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.exception.ZkNodeExistsException; import org.codehaus.jackson.map.ObjectMapper; +import org.joda.time.DateTime; import org.joda.time.Duration; import javax.annotation.Nullable; @@ -674,7 +675,7 @@ public class DruidMaster }, new DruidMasterRuleRunner(DruidMaster.this), new DruidMasterCleanup(DruidMaster.this), - new DruidMasterBalancer(DruidMaster.this, new BalancerCostAnalyzer()), + new DruidMasterBalancer(DruidMaster.this, new BalancerCostAnalyzer(DateTime.now())), new DruidMasterLogger() ) ); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 4e0bfdb1db2..9a7445eb03a 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -111,6 +111,19 @@ public class DruidMasterBalancer implements DruidMasterHelper "Initial Total Cost: [%s]", analyzer.getInitialTotalCost() ); + + analyzer.init(serverHolderList); + log.info( + "Normalization: [%s]", + analyzer.getNormalization() + ); + + analyzer.init(serverHolderList); + log.info( + "Normalized Inital Cost: [%s]", + analyzer.getNormalizedInitialCost() + ); + moveSegments(analyzer.findSegmentsToMove(), params); stats.addToTieredStat("costChange", tier, (long) analyzer.getTotalCostChange()); diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index 2d26be1745b..748f66f135d 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -31,6 +31,7 @@ import com.metamx.druid.master.LoadPeonCallback; import com.metamx.druid.master.MasterStats; import com.metamx.druid.master.ServerHolder; import com.metamx.emitter.EmittingLogger; +import org.joda.time.DateTime; import java.util.ArrayList; import java.util.List; @@ -77,7 +78,7 @@ public abstract class LoadRule implements Rule List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { - BalancerCostAnalyzer analyzer = new BalancerCostAnalyzer(); + BalancerCostAnalyzer analyzer = new BalancerCostAnalyzer(DateTime.now()); BalancerCostAnalyzer.BalancerCostAnalyzerHelper helper = analyzer.new BalancerCostAnalyzerHelper(serverHolderList, segment); helper.computeAllCosts(); Pair minPair = helper.getCostsServerHolderPairs().pollFirst(); diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index ba1b71c8316..777b4063a10 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -27,6 +27,7 @@ import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; import junit.framework.Assert; import org.easymock.EasyMock; +import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.After; import org.junit.Before; @@ -182,7 +183,7 @@ public class DruidMasterBalancerTest .withLoadManagementPeons(ImmutableMap.of("from", peon, "to", peon)) .build(); - params = new DruidMasterBalancer(master, new BalancerCostAnalyzer()).run(params); + params = new DruidMasterBalancer(master, new BalancerCostAnalyzer(new DateTime("2013-01-01"))).run(params); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("costChange").get("normal").get() > 0); } From 2ea164acd98e4cee1330f3da17297631f02bc031 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 7 Jan 2013 09:09:56 -0800 Subject: [PATCH 007/121] get rid of extraneous analyzer.inits --- .../java/com/metamx/druid/master/DruidMasterBalancer.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 9a7445eb03a..b82a3ba5b89 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -106,19 +106,18 @@ public class DruidMasterBalancer implements DruidMasterHelper List serverHolderList = new ArrayList(entry.getValue()); - analyzer.init(serverHolderList); + analyzer.init(serverHolderList, params); + log.info( "Initial Total Cost: [%s]", analyzer.getInitialTotalCost() ); - analyzer.init(serverHolderList); log.info( "Normalization: [%s]", analyzer.getNormalization() ); - analyzer.init(serverHolderList); log.info( "Normalized Inital Cost: [%s]", analyzer.getNormalizedInitialCost() From a7553776fac60722f1230c17deffc569194c344d Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 7 Jan 2013 12:51:51 -0800 Subject: [PATCH 008/121] make sampling of segments uniformly at random and add unit test --- .../druid/master/BalancerCostAnalyzer.java | 37 +- .../druid/master/DruidMasterBalancerTest.java | 323 ++++++++++++++---- 2 files changed, 288 insertions(+), 72 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 94a55363bc8..3375e4b4c91 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -56,6 +56,7 @@ public class BalancerCostAnalyzer private double initialTotalCost; private double normalization; private double totalCostChange; + private int totalSegments; public BalancerCostAnalyzer(DateTime referenceTimestamp) { @@ -64,11 +65,12 @@ public class BalancerCostAnalyzer totalCostChange = 0; } - public void init(List serverHolderList) + public void init(List serverHolderList, DruidMasterRuntimeParams params) { this.initialTotalCost = calculateInitialTotalCost(serverHolderList); this.normalization = calculateNormalization(serverHolderList); this.serverHolderList = serverHolderList; + this.totalSegments = params.getAvailableSegments().size(); } public double getInitialTotalCost() @@ -317,6 +319,23 @@ public class BalancerCostAnalyzer } + /* + * Sample from each server with probability proportional to the number of segments on that server. + */ + private ServerHolder sampleServer() + { + int num = rand.nextInt(totalSegments); + int cumulativeSegments = 0; + int numToStopAt = 0; + + while (cumulativeSegments <= num) { + cumulativeSegments += serverHolderList.get(numToStopAt).getServer().getSegments().size(); + numToStopAt++; + } + + return serverHolderList.get(numToStopAt - 1); + } + public Set findSegmentsToMove() { Set segmentHoldersToMove = Sets.newHashSet(); @@ -326,11 +345,19 @@ public class BalancerCostAnalyzer while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 3 * MAX_SEGMENTS_TO_MOVE) { counter++; - ServerHolder fromServerHolder = serverHolderList.get(rand.nextInt(serverHolderList.size())); + + int numServers = serverHolderList.size(); + if (numServers == 0) break; + + // We want to sample from each server w.p. numSegmentsOnServer / totalSegments + ServerHolder fromServerHolder = sampleServer(); + + // and actually pick that segment uniformly at random w.p. 1 / numSegmentsOnServer + // so that the probability of picking a segment is 1 / totalSegments. List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); - if (segments.size() == 0) { - continue; - } + + if (segments.size() == 0) continue; + DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); if (movingSegments.contains(proposalSegment)) { continue; diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 777b4063a10..551bd424bb4 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -20,11 +20,14 @@ package com.metamx.druid.master; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Sets; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; +import com.metamx.druid.shard.NoneShardSpec; import junit.framework.Assert; import org.easymock.EasyMock; import org.joda.time.DateTime; @@ -42,73 +45,139 @@ import java.util.Map; public class DruidMasterBalancerTest { private DruidMaster master; - private DruidServer druidServerHigh; - private DruidServer druidServerLow; + private DruidServer druidServer1; + private DruidServer druidServer2; + private DruidServer druidServer3; + private DruidServer druidServer4; private DataSegment segment1; private DataSegment segment2; private DataSegment segment3; private DataSegment segment4; + Map segments; private LoadQueuePeon peon; private DruidDataSource dataSource; - private Map segments; - private static final int DAY_IN_MILLIS = 1000 * 60 * 60 * 24; @Before public void setUp() throws Exception { master = EasyMock.createMock(DruidMaster.class); - druidServerHigh = EasyMock.createMock(DruidServer.class); - druidServerLow = EasyMock.createMock(DruidServer.class); + druidServer1 = EasyMock.createMock(DruidServer.class); + druidServer2 = EasyMock.createMock(DruidServer.class); + druidServer3 = EasyMock.createMock(DruidServer.class); + druidServer4 = EasyMock.createMock(DruidServer.class); segment1 = EasyMock.createMock(DataSegment.class); segment2 = EasyMock.createMock(DataSegment.class); segment3 = EasyMock.createMock(DataSegment.class); segment4 = EasyMock.createMock(DataSegment.class); peon = EasyMock.createMock(LoadQueuePeon.class); dataSource = EasyMock.createMock(DruidDataSource.class); + + DateTime start1 = new DateTime("2012-01-01"); + DateTime start2 = new DateTime("2012-02-01"); + DateTime version = new DateTime("2012-03-01"); + segment1 = new DataSegment( + "datasource1", + new Interval(start1, start1.plusHours(1)), + version.toString(), + Maps.newHashMap(), + Lists.newArrayList(), + Lists.newArrayList(), + new NoneShardSpec(), + 11L + ); + segment2 = new DataSegment( + "datasource1", + new Interval(start2, start2.plusHours(1)), + version.toString(), + Maps.newHashMap(), + Lists.newArrayList(), + Lists.newArrayList(), + new NoneShardSpec(), + 7L + ); + segment3 = new DataSegment( + "datasource2", + new Interval(start1, start1.plusHours(1)), + version.toString(), + Maps.newHashMap(), + Lists.newArrayList(), + Lists.newArrayList(), + new NoneShardSpec(), + 4L + ); + segment4 = new DataSegment( + "datasource2", + new Interval(start2, start2.plusHours(1)), + version.toString(), + Maps.newHashMap(), + Lists.newArrayList(), + Lists.newArrayList(), + new NoneShardSpec(), + 8L + ); + + segments = new HashMap(); + segments.put("segment1", segment1); + segments.put("segment2", segment2); + segments.put("segment3", segment3); + segments.put("segment4", segment4); } @After public void tearDown() throws Exception { EasyMock.verify(master); - EasyMock.verify(druidServerHigh); - EasyMock.verify(druidServerLow); - EasyMock.verify(segment1); - EasyMock.verify(segment2); - EasyMock.verify(segment3); - EasyMock.verify(segment4); + EasyMock.verify(druidServer1); + EasyMock.verify(druidServer2); + EasyMock.verify(druidServer3); + EasyMock.verify(druidServer4); EasyMock.verify(peon); EasyMock.verify(dataSource); } @Test - public void testRun() + public void testRun1() { - Map segments = new HashMap(); - segments.put("segment1", segment1); - segments.put("segment2", segment2); - segments.put("segment3", segment3); - segments.put("segment4", segment4); - // Mock some servers of different usages - EasyMock.expect(druidServerHigh.getName()).andReturn("from").atLeastOnce(); - EasyMock.expect(druidServerHigh.getCurrSize()).andReturn(30L).atLeastOnce(); - EasyMock.expect(druidServerHigh.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServerHigh.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); - EasyMock.expect(druidServerHigh.getSegments()).andReturn(segments).anyTimes(); - EasyMock.replay(druidServerHigh); - EasyMock.expect(druidServerLow.getName()).andReturn("to").atLeastOnce(); - EasyMock.expect(druidServerLow.getTier()).andReturn("normal").atLeastOnce(); - EasyMock.expect(druidServerLow.getCurrSize()).andReturn(0L).atLeastOnce(); - EasyMock.expect(druidServerLow.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServerLow.getSegments()).andReturn(new HashMap()).anyTimes(); - EasyMock.expect(druidServerLow.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); - EasyMock.expect(druidServerLow.getSegment("segment1")).andReturn(null).anyTimes(); - EasyMock.expect(druidServerLow.getSegment("segment2")).andReturn(null).anyTimes(); - EasyMock.expect(druidServerLow.getSegment("segment3")).andReturn(null).anyTimes(); - EasyMock.expect(druidServerLow.getSegment("segment4")).andReturn(null).anyTimes(); - EasyMock.replay(druidServerLow); + EasyMock.expect(druidServer1.getName()).andReturn("from").atLeastOnce(); + EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); + EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); + EasyMock.expect(druidServer1.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); + EasyMock.expect(druidServer1.getSegments()).andReturn(segments).anyTimes(); + EasyMock.replay(druidServer1); + + EasyMock.expect(druidServer2.getName()).andReturn("to").atLeastOnce(); + EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); + EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce(); + EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); + EasyMock.expect(druidServer2.getSegments()).andReturn(new HashMap()).anyTimes(); + EasyMock.expect(druidServer2.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); + EasyMock.expect( + druidServer2.getSegment( + "datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer2.getSegment( + "datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer2.getSegment( + "datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer2.getSegment( + "datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect(druidServer2.getSegment("segment3")).andReturn(null).anyTimes(); + EasyMock.expect(druidServer2.getSegment("segment4")).andReturn(null).anyTimes(); + EasyMock.replay(druidServer2); + EasyMock.replay(druidServer3); + EasyMock.replay(druidServer4); // Mock a datasource EasyMock.expect(dataSource.getSegments()).andReturn( @@ -121,35 +190,6 @@ public class DruidMasterBalancerTest ).anyTimes(); EasyMock.replay(dataSource); - /* - (2, 1, 2, 1 - -, 2, 1, 2 - -, -, 2, 1 - -, -, -, 2) - */ - - // Mock some segments of different sizes - EasyMock.expect(segment1.getSize()).andReturn(11L).anyTimes(); - EasyMock.expect(segment1.getIdentifier()).andReturn("segment1").anyTimes(); - EasyMock.expect(segment1.getDataSource()).andReturn("datasource1").anyTimes(); - EasyMock.expect(segment1.getInterval()).andReturn(new Interval(0, 1 * DAY_IN_MILLIS)).anyTimes(); - EasyMock.replay(segment1); - EasyMock.expect(segment2.getSize()).andReturn(7L).anyTimes(); - EasyMock.expect(segment2.getIdentifier()).andReturn("segment2").anyTimes(); - EasyMock.expect(segment2.getDataSource()).andReturn("datasource1").anyTimes(); - EasyMock.expect(segment2.getInterval()).andReturn(new Interval(10 * DAY_IN_MILLIS, 11 * DAY_IN_MILLIS)).anyTimes(); - EasyMock.replay(segment2); - EasyMock.expect(segment3.getSize()).andReturn(4L).anyTimes(); - EasyMock.expect(segment3.getIdentifier()).andReturn("segment3").anyTimes(); - EasyMock.expect(segment3.getDataSource()).andReturn("datasource1").anyTimes(); - EasyMock.expect(segment3.getInterval()).andReturn(new Interval(0, 1 * DAY_IN_MILLIS)).anyTimes(); - EasyMock.replay(segment3); - EasyMock.expect(segment4.getSize()).andReturn(8L).anyTimes(); - EasyMock.expect(segment4.getIdentifier()).andReturn("segment4").anyTimes(); - EasyMock.expect(segment4.getDataSource()).andReturn("datasource1").anyTimes(); - EasyMock.expect(segment4.getInterval()).andReturn(new Interval(10 * DAY_IN_MILLIS, 11 * DAY_IN_MILLIS)).anyTimes(); - EasyMock.replay(segment4); - // Mock stuff that the master needs master.moveSegment( EasyMock.anyObject(), @@ -173,14 +213,163 @@ public class DruidMasterBalancerTest MinMaxPriorityQueue.orderedBy(DruidMasterBalancer.percentUsedComparator) .create( Arrays.asList( - new ServerHolder(druidServerHigh, peon), - new ServerHolder(druidServerLow, peon) + new ServerHolder(druidServer1, peon), + new ServerHolder(druidServer2, peon) ) ) ) ) ) .withLoadManagementPeons(ImmutableMap.of("from", peon, "to", peon)) + .withAvailableSegments(segments.values()) + .build(); + + params = new DruidMasterBalancer(master, new BalancerCostAnalyzer(new DateTime("2013-01-01"))).run(params); + Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); + Assert.assertTrue(params.getMasterStats().getPerTierStats().get("costChange").get("normal").get() > 0); + } + + @Test + public void testRun2() + { + // Mock some servers of different usages + EasyMock.expect(druidServer1.getName()).andReturn("1").atLeastOnce(); + EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); + EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); + EasyMock.expect(druidServer1.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); + EasyMock.expect(druidServer1.getSegments()).andReturn(segments).anyTimes(); + EasyMock.replay(druidServer1); + + EasyMock.expect(druidServer2.getName()).andReturn("2").atLeastOnce(); + EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); + EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce(); + EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); + EasyMock.expect(druidServer2.getSegments()).andReturn(new HashMap()).anyTimes(); + EasyMock.expect(druidServer2.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); + EasyMock.expect( + druidServer2.getSegment( + "datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer2.getSegment( + "datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer2.getSegment( + "datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer2.getSegment( + "datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.replay(druidServer2); + + EasyMock.expect(druidServer3.getName()).andReturn("3").atLeastOnce(); + EasyMock.expect(druidServer3.getTier()).andReturn("normal").anyTimes(); + EasyMock.expect(druidServer3.getCurrSize()).andReturn(0L).atLeastOnce(); + EasyMock.expect(druidServer3.getMaxSize()).andReturn(100L).atLeastOnce(); + EasyMock.expect(druidServer3.getSegments()).andReturn(new HashMap()).anyTimes(); + EasyMock.expect(druidServer3.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); + EasyMock.expect( + druidServer3.getSegment( + "datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer3.getSegment( + "datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer3.getSegment( + "datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer3.getSegment( + "datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.replay(druidServer3); + + EasyMock.expect(druidServer4.getName()).andReturn("4").atLeastOnce(); + EasyMock.expect(druidServer4.getTier()).andReturn("normal").anyTimes(); + EasyMock.expect(druidServer4.getCurrSize()).andReturn(0L).atLeastOnce(); + EasyMock.expect(druidServer4.getMaxSize()).andReturn(100L).atLeastOnce(); + EasyMock.expect(druidServer4.getSegments()).andReturn(new HashMap()).anyTimes(); + EasyMock.expect(druidServer4.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); + EasyMock.expect( + druidServer4.getSegment( + "datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer4.getSegment( + "datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer4.getSegment( + "datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer4.getSegment( + "datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.replay(druidServer4); + + + // Mock a datasource + EasyMock.expect(dataSource.getSegments()).andReturn( + Sets.newHashSet( + segment1, + segment2, + segment3, + segment4 + ) + ).anyTimes(); + EasyMock.replay(dataSource); + + // Mock stuff that the master needs + master.moveSegment( + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + ); + EasyMock.expectLastCall().atLeastOnce(); + EasyMock.replay(master); + + EasyMock.expect(peon.getLoadQueueSize()).andReturn(0L).atLeastOnce(); + EasyMock.expect(peon.getSegmentsToLoad()).andReturn(Sets.newHashSet()).atLeastOnce(); + EasyMock.replay(peon); + + DruidMasterRuntimeParams params = + DruidMasterRuntimeParams.newBuilder() + .withDruidCluster( + new DruidCluster( + ImmutableMap.>of( + "normal", + MinMaxPriorityQueue.orderedBy(DruidMasterBalancer.percentUsedComparator) + .create( + Arrays.asList( + new ServerHolder(druidServer1, peon), + new ServerHolder(druidServer2, peon), + new ServerHolder(druidServer3, peon), + new ServerHolder(druidServer4, peon) + ) + ) + ) + ) + ) + .withLoadManagementPeons(ImmutableMap.of("1", peon, "2", peon, "3", peon, "4", peon)) + .withAvailableSegments(segments.values()) .build(); params = new DruidMasterBalancer(master, new BalancerCostAnalyzer(new DateTime("2013-01-01"))).run(params); From d2ff88e923cf52f2f9f84bc759639e4650c56ff5 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 7 Jan 2013 13:09:18 -0800 Subject: [PATCH 009/121] add more stats to be emitted --- .../druid/master/DruidMasterBalancer.java | 34 ++++++++----------- 1 file changed, 15 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index b82a3ba5b89..ab0aba86943 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -107,28 +107,24 @@ public class DruidMasterBalancer implements DruidMasterHelper List serverHolderList = new ArrayList(entry.getValue()); analyzer.init(serverHolderList, params); - - log.info( - "Initial Total Cost: [%s]", - analyzer.getInitialTotalCost() - ); - - log.info( - "Normalization: [%s]", - analyzer.getNormalization() - ); - - log.info( - "Normalized Inital Cost: [%s]", - analyzer.getNormalizedInitialCost() - ); - moveSegments(analyzer.findSegmentsToMove(), params); - stats.addToTieredStat("costChange", tier, (long) analyzer.getTotalCostChange()); + double initialTotalCost = analyzer.getInitialTotalCost(); + double normalization = analyzer.getNormalization(); + double normalizedInitialCost = analyzer.getNormalizedInitialCost(); + double costChange = analyzer.getTotalCostChange(); + + stats.addToTieredStat("initialCost", tier, (long) initialTotalCost); + stats.addToTieredStat("normalization", tier, (long) normalization); + stats.addToTieredStat("costChange", tier, (long) costChange); + log.info( - "Cost Change: [%s]", - analyzer.getTotalCostChange() + "Initial Total Cost: [%f], Initial Normalized Cost: [%f], Cost Change: [%f], Normalized Cost Change: [%f], New Normalized Cost: [%f]", + initialTotalCost, + normalizedInitialCost, + costChange, + costChange / normalization, + (initialTotalCost - costChange) / normalization ); if (serverHolderList.size() <= 1) { From ba82fe746a0044ffcc435a2b3bbf57071d0fff96 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 7 Jan 2013 16:28:27 -0800 Subject: [PATCH 010/121] put MAX_SEGMENTS_TO_MOVE in the config --- .../metamx/druid/master/BalancerCostAnalyzer.java | 5 +++-- .../java/com/metamx/druid/master/DruidMaster.java | 7 ++++++- .../metamx/druid/master/DruidMasterConfig.java | 4 ++++ .../com/metamx/druid/master/rules/LoadRule.java | 15 ++++++++++++--- 4 files changed, 25 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 3375e4b4c91..4a0f2b90bcd 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -44,7 +44,7 @@ import java.util.Set; public class BalancerCostAnalyzer { private static final Logger log = new Logger(BalancerCostAnalyzer.class); - private static final int MAX_SEGMENTS_TO_MOVE = 5; + private final int MAX_SEGMENTS_TO_MOVE; private static final int DAY_IN_MILLIS = 1000 * 60 * 60 * 24; private static final int SEVEN_DAYS_IN_MILLIS = 7 * DAY_IN_MILLIS; private static final int THIRTY_DAYS_IN_MILLIS = 30 * DAY_IN_MILLIS; @@ -58,9 +58,10 @@ public class BalancerCostAnalyzer private double totalCostChange; private int totalSegments; - public BalancerCostAnalyzer(DateTime referenceTimestamp) + public BalancerCostAnalyzer(DateTime referenceTimestamp, int MAX_SEGMENTS_TO_MOVE) { this.referenceTimestamp = referenceTimestamp; + this.MAX_SEGMENTS_TO_MOVE = MAX_SEGMENTS_TO_MOVE; rand = new Random(0); totalCostChange = 0; } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index ff01957d102..d5833b203fe 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -132,6 +132,11 @@ public class DruidMaster return master; } + public DruidMasterConfig getConfig() + { + return config; + } + public Map getLoadStatus() { Map availableSegmentMap = Maps.newHashMap(); @@ -675,7 +680,7 @@ public class DruidMaster }, new DruidMasterRuleRunner(DruidMaster.this), new DruidMasterCleanup(DruidMaster.this), - new DruidMasterBalancer(DruidMaster.this, new BalancerCostAnalyzer(DateTime.now())), + new DruidMasterBalancer(DruidMaster.this, new BalancerCostAnalyzer(DateTime.now(), config.getMaxSegmentsToMove())), new DruidMasterLogger() ) ); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java b/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java index 87df78d4f5e..274b3765164 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java @@ -80,4 +80,8 @@ public abstract class DruidMasterConfig { return Integer.MAX_VALUE; } + + @Config("druid.master.balancer.maxSegmentsToMove") + @Default("5") + public abstract int getMaxSegmentsToMove(); } diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index 748f66f135d..cd63fa69a7e 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -59,7 +59,15 @@ public abstract class LoadRule implements Rule return stats; } - stats.accumulate(assign(expectedReplicants, totalReplicants, serverQueue, segment)); + stats.accumulate( + assign( + expectedReplicants, + totalReplicants, + serverQueue, + segment, + master.getConfig().getMaxSegmentsToMove() + ) + ); stats.accumulate(drop(expectedReplicants, clusterReplicants, segment, params)); return stats; @@ -69,7 +77,8 @@ public abstract class LoadRule implements Rule int expectedReplicants, int totalReplicants, MinMaxPriorityQueue serverQueue, - DataSegment segment + DataSegment segment, + int MAX_SEGMENTS_TO_MOVE ) { MasterStats stats = new MasterStats(); @@ -78,7 +87,7 @@ public abstract class LoadRule implements Rule List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { - BalancerCostAnalyzer analyzer = new BalancerCostAnalyzer(DateTime.now()); + BalancerCostAnalyzer analyzer = new BalancerCostAnalyzer(DateTime.now(), MAX_SEGMENTS_TO_MOVE); BalancerCostAnalyzer.BalancerCostAnalyzerHelper helper = analyzer.new BalancerCostAnalyzerHelper(serverHolderList, segment); helper.computeAllCosts(); Pair minPair = helper.getCostsServerHolderPairs().pollFirst(); From 6fda5330fda43c75d6fff74045e0fe444aadda21 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 7 Jan 2013 16:56:55 -0800 Subject: [PATCH 011/121] finish making MAX_SEGMENTS_TO_MOVE a config item --- .../druid/master/BalancerCostAnalyzer.java | 9 +++--- .../com/metamx/druid/master/DruidMaster.java | 3 +- .../master/DruidMasterRuntimeParams.java | 28 ++++++++++++++++--- .../metamx/druid/master/rules/LoadRule.java | 4 +-- .../druid/master/DruidMasterBalancerTest.java | 3 ++ .../master/DruidMasterRuleRunnerTest.java | 1 + .../metamx/druid/master/DruidMasterTest.java | 6 ++++ 7 files changed, 43 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 4a0f2b90bcd..7a387460a9b 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -44,7 +44,7 @@ import java.util.Set; public class BalancerCostAnalyzer { private static final Logger log = new Logger(BalancerCostAnalyzer.class); - private final int MAX_SEGMENTS_TO_MOVE; + private int MAX_SEGMENTS_TO_MOVE; private static final int DAY_IN_MILLIS = 1000 * 60 * 60 * 24; private static final int SEVEN_DAYS_IN_MILLIS = 7 * DAY_IN_MILLIS; private static final int THIRTY_DAYS_IN_MILLIS = 30 * DAY_IN_MILLIS; @@ -58,20 +58,21 @@ public class BalancerCostAnalyzer private double totalCostChange; private int totalSegments; - public BalancerCostAnalyzer(DateTime referenceTimestamp, int MAX_SEGMENTS_TO_MOVE) + public BalancerCostAnalyzer(DateTime referenceTimestamp) { this.referenceTimestamp = referenceTimestamp; - this.MAX_SEGMENTS_TO_MOVE = MAX_SEGMENTS_TO_MOVE; + rand = new Random(0); totalCostChange = 0; } public void init(List serverHolderList, DruidMasterRuntimeParams params) { + this.serverHolderList = serverHolderList; this.initialTotalCost = calculateInitialTotalCost(serverHolderList); this.normalization = calculateNormalization(serverHolderList); - this.serverHolderList = serverHolderList; this.totalSegments = params.getAvailableSegments().size(); + this.MAX_SEGMENTS_TO_MOVE = params.getMaxSegmentsToMove(); } public double getInitialTotalCost() diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index d5833b203fe..386b85cfbb0 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -575,6 +575,7 @@ public class DruidMaster .withEmitter(emitter) .withMergeBytesLimit(config.getMergeBytesLimit()) .withMergeSegmentsLimit(config.getMergeSegmentsLimit()) + .withMaxSegmentsToMove(config.getMaxSegmentsToMove()) .build(); for (DruidMasterHelper helper : helpers) { @@ -680,7 +681,7 @@ public class DruidMaster }, new DruidMasterRuleRunner(DruidMaster.this), new DruidMasterCleanup(DruidMaster.this), - new DruidMasterBalancer(DruidMaster.this, new BalancerCostAnalyzer(DateTime.now(), config.getMaxSegmentsToMove())), + new DruidMasterBalancer(DruidMaster.this, new BalancerCostAnalyzer(DateTime.now())), new DruidMasterLogger() ) ); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java index a8658937e68..0357baa4d64 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java @@ -49,6 +49,7 @@ public class DruidMasterRuntimeParams private final MasterStats stats; private final long mergeBytesLimit; private final int mergeSegmentsLimit; + private final int maxSegmentsToMove; public DruidMasterRuntimeParams( long startTime, @@ -62,7 +63,8 @@ public class DruidMasterRuntimeParams long millisToWaitBeforeDeleting, MasterStats stats, long mergeBytesLimit, - int mergeSegmentsLimit + int mergeSegmentsLimit, + int maxSegmentsToMove ) { this.startTime = startTime; @@ -77,6 +79,12 @@ public class DruidMasterRuntimeParams this.stats = stats; this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; + this.maxSegmentsToMove = maxSegmentsToMove; + } + + public int getMaxSegmentsToMove() + { + return maxSegmentsToMove; } public long getStartTime() @@ -163,7 +171,8 @@ public class DruidMasterRuntimeParams millisToWaitBeforeDeleting, stats, mergeBytesLimit, - mergeSegmentsLimit + mergeSegmentsLimit, + maxSegmentsToMove ); } @@ -181,6 +190,7 @@ public class DruidMasterRuntimeParams private MasterStats stats; private long mergeBytesLimit; private int mergeSegmentsLimit; + private int maxSegmentsToMove; Builder() { @@ -196,6 +206,7 @@ public class DruidMasterRuntimeParams this.stats = new MasterStats(); this.mergeBytesLimit = 0; this.mergeSegmentsLimit = 0; + this.maxSegmentsToMove = 0; } Builder( @@ -210,7 +221,8 @@ public class DruidMasterRuntimeParams long millisToWaitBeforeDeleting, MasterStats stats, long mergeBytesLimit, - int mergeSegmentsLimit + int mergeSegmentsLimit, + int maxSegmentsToMove ) { this.startTime = startTime; @@ -225,6 +237,7 @@ public class DruidMasterRuntimeParams this.stats = stats; this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; + this.maxSegmentsToMove = maxSegmentsToMove; } public DruidMasterRuntimeParams build() @@ -241,7 +254,8 @@ public class DruidMasterRuntimeParams millisToWaitBeforeDeleting, stats, mergeBytesLimit, - mergeSegmentsLimit + mergeSegmentsLimit, + maxSegmentsToMove ); } @@ -316,5 +330,11 @@ public class DruidMasterRuntimeParams this.mergeSegmentsLimit = mergeSegmentsLimit; return this; } + + public Builder withMaxSegmentsToMove(int maxSegmentsToMove) + { + this.maxSegmentsToMove = maxSegmentsToMove; + return this; + } } } diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index cd63fa69a7e..75cc76d7c7a 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -65,7 +65,7 @@ public abstract class LoadRule implements Rule totalReplicants, serverQueue, segment, - master.getConfig().getMaxSegmentsToMove() + params.getMaxSegmentsToMove() ) ); stats.accumulate(drop(expectedReplicants, clusterReplicants, segment, params)); @@ -87,7 +87,7 @@ public abstract class LoadRule implements Rule List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { - BalancerCostAnalyzer analyzer = new BalancerCostAnalyzer(DateTime.now(), MAX_SEGMENTS_TO_MOVE); + BalancerCostAnalyzer analyzer = new BalancerCostAnalyzer(DateTime.now()); BalancerCostAnalyzer.BalancerCostAnalyzerHelper helper = analyzer.new BalancerCostAnalyzerHelper(serverHolderList, segment); helper.computeAllCosts(); Pair minPair = helper.getCostsServerHolderPairs().pollFirst(); diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 551bd424bb4..082c4a78077 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -44,6 +44,7 @@ import java.util.Map; */ public class DruidMasterBalancerTest { + private static final int MAX_SEGMENTS_TO_MOVE = 5; private DruidMaster master; private DruidServer druidServer1; private DruidServer druidServer2; @@ -222,6 +223,7 @@ public class DruidMasterBalancerTest ) .withLoadManagementPeons(ImmutableMap.of("from", peon, "to", peon)) .withAvailableSegments(segments.values()) + .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) .build(); params = new DruidMasterBalancer(master, new BalancerCostAnalyzer(new DateTime("2013-01-01"))).run(params); @@ -370,6 +372,7 @@ public class DruidMasterBalancerTest ) .withLoadManagementPeons(ImmutableMap.of("1", peon, "2", peon, "3", peon, "4", peon)) .withAvailableSegments(segments.values()) + .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) .build(); params = new DruidMasterBalancer(master, new BalancerCostAnalyzer(new DateTime("2013-01-01"))).run(params); diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java index 2ac32578cbf..074d61225c1 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java @@ -175,6 +175,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withMaxSegmentsToMove(5) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java index c54a9a66564..f3f5e0a6d30 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -124,6 +124,12 @@ public class DruidMasterTest { return ""; } + + @Override + public int getMaxSegmentsToMove() + { + return 0; + } }, null, null, From 2b4dbdde0389b1bdefcca676fa5188f4fb7e2801 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 7 Jan 2013 17:28:53 -0800 Subject: [PATCH 012/121] lots of changes --- .../druid/master/BalancerCostAnalyzer.java | 90 ++++++++----------- .../metamx/druid/master/rules/LoadRule.java | 4 +- 2 files changed, 38 insertions(+), 56 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 7a387460a9b..ab319fae72e 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -32,6 +32,7 @@ import org.joda.time.Interval; import java.util.Comparator; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.Set; @@ -103,9 +104,8 @@ public class BalancerCostAnalyzer { double cost = 0; for (ServerHolder server : serverHolderList) { - DataSegment[] segments = server.getServer().getSegments().values().toArray(new DataSegment[]{}); - for (int i = 0; i < segments.length; ++i) { - cost += computeJointSegmentCosts(segments[i], segments[i]); + for (DataSegment segment : server.getServer().getSegments().values()) { + cost += computeJointSegmentCosts(segment, segment); } } return cost; @@ -176,14 +176,14 @@ public class BalancerCostAnalyzer * Their purpose is to unify the balance/assignment code since a segment that has not yet been assigned * does not have a source server. */ - public class NullServerHolder extends ServerHolder + public static class NullServerHolder extends ServerHolder { public NullServerHolder() { super(null, null); } - public class NullDruidServer extends DruidServer + public static class NullDruidServer extends DruidServer { public NullDruidServer() { @@ -206,36 +206,17 @@ public class BalancerCostAnalyzer public class BalancerCostAnalyzerHelper { - private MinMaxPriorityQueue> costsServerHolderPairs; - private List serverHolderList; - private DataSegment proposalSegment; - private ServerHolder fromServerHolder; - private Set segmentHoldersToMove; + private final List serverHolderList; + private final DataSegment proposalSegment; + private final ServerHolder fromServerHolder; + private final Set segmentHoldersToMove; + + private Pair minPair; private double currCost; - public MinMaxPriorityQueue> getCostsServerHolderPairs() + public Pair getMinPair() { - return costsServerHolderPairs; - } - - public List getServerHolderList() - { - return serverHolderList; - } - - public DataSegment getProposalSegment() - { - return proposalSegment; - } - - public ServerHolder getFromServerHolder() - { - return fromServerHolder; - } - - public Set getSegmentHoldersToMove() - { - return segmentHoldersToMove; + return minPair; } public double getCurrCost() @@ -257,9 +238,20 @@ public class BalancerCostAnalyzer ServerHolder fromServerHolder, Set segmentHoldersToMove ) + { + this.serverHolderList = serverHolderList; + this.proposalSegment = proposalSegment; + this.fromServerHolder = fromServerHolder; + this.segmentHoldersToMove = segmentHoldersToMove; + this.currCost = 0; + + computeAllCosts(); + } + + public void computeAllCosts() { // Just need a regular priority queue for the min. element. - this.costsServerHolderPairs = MinMaxPriorityQueue.orderedBy( + MinMaxPriorityQueue> costsServerHolderPairs = MinMaxPriorityQueue.orderedBy( new Comparator>() { @Override @@ -272,17 +264,12 @@ public class BalancerCostAnalyzer } } ).create(); - this.serverHolderList = serverHolderList; - this.proposalSegment = proposalSegment; - this.fromServerHolder = fromServerHolder; - this.segmentHoldersToMove = segmentHoldersToMove; - this.currCost = 0; - } - public void computeAllCosts() - { - // The contribution to the total cost of a given server by proposing to move the segment to that server is... for (ServerHolder server : serverHolderList) { + // Only calculate costs if the server has enough space. + if (proposalSegment.getSize() > server.getAvailableSize()) break; + + // The contribution to the total cost of a given server by proposing to move the segment to that server is... double cost = 0f; // the sum of the costs of other (inclusive) segments on the server for (DataSegment segment : server.getServer().getSegments().values()) { @@ -295,9 +282,7 @@ public class BalancerCostAnalyzer } // plus the costs of segments that will be moved. - Iterator it = segmentHoldersToMove.iterator(); - while (it.hasNext()) { - BalancerSegmentHolder segmentToMove = (BalancerSegmentHolder) it.next(); + for (BalancerSegmentHolder segmentToMove : segmentHoldersToMove) { if (server.getServer().equals(segmentToMove.getToServer())) { cost += computeJointSegmentCosts(proposalSegment, segmentToMove.getSegment()); } @@ -311,12 +296,10 @@ public class BalancerCostAnalyzer currCost = cost; } - // Only enter the queue if the server has enough size. - if (proposalSegment.getSize() < server.getAvailableSize()) { - costsServerHolderPairs.add(Pair.of(cost, server)); - } - + costsServerHolderPairs.add(Pair.of(cost, server)); } + + minPair = costsServerHolderPairs.pollFirst(); } } @@ -342,13 +325,13 @@ public class BalancerCostAnalyzer { Set segmentHoldersToMove = Sets.newHashSet(); Set movingSegments = Sets.newHashSet(); + int numServers = serverHolderList.size(); int counter = 0; while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 3 * MAX_SEGMENTS_TO_MOVE) { counter++; - int numServers = serverHolderList.size(); if (numServers == 0) break; // We want to sample from each server w.p. numSegmentsOnServer / totalSegments @@ -358,7 +341,7 @@ public class BalancerCostAnalyzer // so that the probability of picking a segment is 1 / totalSegments. List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); - if (segments.size() == 0) continue; + if (segments.isEmpty()) continue; DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); if (movingSegments.contains(proposalSegment)) { @@ -371,9 +354,8 @@ public class BalancerCostAnalyzer fromServerHolder, segmentHoldersToMove ); - helper.computeAllCosts(); - Pair minPair = helper.getCostsServerHolderPairs().pollFirst(); + Pair minPair = helper.getMinPair(); if (minPair.rhs != null && !minPair.rhs.equals(fromServerHolder)) { movingSegments.add(proposalSegment); diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index 75cc76d7c7a..6801c4a3945 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -89,8 +89,8 @@ public abstract class LoadRule implements Rule while (totalReplicants < expectedReplicants) { BalancerCostAnalyzer analyzer = new BalancerCostAnalyzer(DateTime.now()); BalancerCostAnalyzer.BalancerCostAnalyzerHelper helper = analyzer.new BalancerCostAnalyzerHelper(serverHolderList, segment); - helper.computeAllCosts(); - Pair minPair = helper.getCostsServerHolderPairs().pollFirst(); + + Pair minPair = helper.getMinPair(); ServerHolder holder = minPair.rhs; if (holder == null) { From 4549c3a8c3bcbb76e87ec27049aca632d45a8961 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Tue, 8 Jan 2013 09:07:43 -0800 Subject: [PATCH 013/121] add some finals, reorganize --- .../druid/master/BalancerCostAnalyzer.java | 205 +++++++++--------- .../metamx/druid/master/rules/LoadRule.java | 2 +- 2 files changed, 104 insertions(+), 103 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index ab319fae72e..a7dae0925b8 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -30,9 +30,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.Comparator; -import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Random; import java.util.Set; @@ -45,15 +43,13 @@ import java.util.Set; public class BalancerCostAnalyzer { private static final Logger log = new Logger(BalancerCostAnalyzer.class); - private int MAX_SEGMENTS_TO_MOVE; private static final int DAY_IN_MILLIS = 1000 * 60 * 60 * 24; private static final int SEVEN_DAYS_IN_MILLIS = 7 * DAY_IN_MILLIS; private static final int THIRTY_DAYS_IN_MILLIS = 30 * DAY_IN_MILLIS; - + private final Random rand; + private final DateTime referenceTimestamp; + private int MAX_SEGMENTS_TO_MOVE; private List serverHolderList; - private Random rand; - private DateTime referenceTimestamp; - private double initialTotalCost; private double normalization; private double totalCostChange; @@ -67,6 +63,7 @@ public class BalancerCostAnalyzer totalCostChange = 0; } + // The assignment usage doesn't require initialization. public void init(List serverHolderList, DruidMasterRuntimeParams params) { this.serverHolderList = serverHolderList; @@ -136,10 +133,9 @@ public class BalancerCostAnalyzer */ public double computeJointSegmentCosts(DataSegment segment1, DataSegment segment2) { - double cost = 0; - Interval gap = segment1.getInterval().gap(segment2.getInterval()); + final Interval gap = segment1.getInterval().gap(segment2.getInterval()); - double baseCost = Math.min(segment1.getSize(), segment2.getSize()); + final double baseCost = Math.min(segment1.getSize(), segment2.getSize()); double recencyPenalty = 1; double dataSourcePenalty = 1; double gapPenalty = 1; @@ -166,13 +162,86 @@ public class BalancerCostAnalyzer } } - cost = baseCost * recencyPenalty * dataSourcePenalty * gapPenalty; + final double cost = baseCost * recencyPenalty * dataSourcePenalty * gapPenalty; return cost; } /* - * These could be anonymous in BalancerCostAnalyzerHelper + * Sample from each server with probability proportional to the number of segments on that server. + */ + private ServerHolder sampleServer() + { + final int num = rand.nextInt(totalSegments); + int cumulativeSegments = 0; + int numToStopAt = 0; + + while (cumulativeSegments <= num) { + cumulativeSegments += serverHolderList.get(numToStopAt).getServer().getSegments().size(); + numToStopAt++; + } + + return serverHolderList.get(numToStopAt - 1); + } + + public Set findSegmentsToMove() + { + final Set segmentHoldersToMove = Sets.newHashSet(); + final Set movingSegments = Sets.newHashSet(); + int numServers = serverHolderList.size(); + + int counter = 0; + + while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 3 * MAX_SEGMENTS_TO_MOVE) { + counter++; + + if (numServers == 0) { + break; + } + + // We want to sample from each server w.p. numSegmentsOnServer / totalSegments + ServerHolder fromServerHolder = sampleServer(); + + // and actually pick that segment uniformly at random w.p. 1 / numSegmentsOnServer + // so that the probability of picking a segment is 1 / totalSegments. + List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); + + if (segments.isEmpty()) { + continue; + } + + DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); + if (movingSegments.contains(proposalSegment)) { + continue; + } + + BalancerCostComputer helper = new BalancerCostComputer( + serverHolderList, + proposalSegment, + fromServerHolder, + segmentHoldersToMove + ); + + Pair minPair = helper.getMinPair(); + + if (minPair.rhs != null && !minPair.rhs.equals(fromServerHolder)) { + movingSegments.add(proposalSegment); + segmentHoldersToMove.add( + new BalancerSegmentHolder( + fromServerHolder.getServer(), + minPair.rhs.getServer(), + proposalSegment + ) + ); + totalCostChange += helper.getCurrCost() - minPair.lhs; + } + } + + return segmentHoldersToMove; + } + + /* + * These could be anonymous in BalancerCostComputer * Their purpose is to unify the balance/assignment code since a segment that has not yet been assigned * does not have a source server. */ @@ -183,6 +252,12 @@ public class BalancerCostAnalyzer super(null, null); } + @Override + public DruidServer getServer() + { + return new NullDruidServer(); + } + public static class NullDruidServer extends DruidServer { public NullDruidServer() @@ -196,35 +271,18 @@ public class BalancerCostAnalyzer return false; } } - - @Override - public DruidServer getServer() - { - return new NullDruidServer(); - } } - public class BalancerCostAnalyzerHelper + public class BalancerCostComputer { private final List serverHolderList; private final DataSegment proposalSegment; private final ServerHolder fromServerHolder; private final Set segmentHoldersToMove; - private Pair minPair; private double currCost; - public Pair getMinPair() - { - return minPair; - } - - public double getCurrCost() - { - return currCost; - } - - public BalancerCostAnalyzerHelper( + public BalancerCostComputer( List serverHolderList, DataSegment proposalSegment ) @@ -232,7 +290,7 @@ public class BalancerCostAnalyzer this(serverHolderList, proposalSegment, new NullServerHolder(), Sets.newHashSet()); } - public BalancerCostAnalyzerHelper( + public BalancerCostComputer( List serverHolderList, DataSegment proposalSegment, ServerHolder fromServerHolder, @@ -248,6 +306,16 @@ public class BalancerCostAnalyzer computeAllCosts(); } + public Pair getMinPair() + { + return minPair; + } + + public double getCurrCost() + { + return currCost; + } + public void computeAllCosts() { // Just need a regular priority queue for the min. element. @@ -267,7 +335,9 @@ public class BalancerCostAnalyzer for (ServerHolder server : serverHolderList) { // Only calculate costs if the server has enough space. - if (proposalSegment.getSize() > server.getAvailableSize()) break; + if (proposalSegment.getSize() > server.getAvailableSize()) { + break; + } // The contribution to the total cost of a given server by proposing to move the segment to that server is... double cost = 0f; @@ -303,75 +373,6 @@ public class BalancerCostAnalyzer } } - - /* - * Sample from each server with probability proportional to the number of segments on that server. - */ - private ServerHolder sampleServer() - { - int num = rand.nextInt(totalSegments); - int cumulativeSegments = 0; - int numToStopAt = 0; - - while (cumulativeSegments <= num) { - cumulativeSegments += serverHolderList.get(numToStopAt).getServer().getSegments().size(); - numToStopAt++; - } - - return serverHolderList.get(numToStopAt - 1); - } - - public Set findSegmentsToMove() - { - Set segmentHoldersToMove = Sets.newHashSet(); - Set movingSegments = Sets.newHashSet(); - int numServers = serverHolderList.size(); - - int counter = 0; - - while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 3 * MAX_SEGMENTS_TO_MOVE) { - counter++; - - if (numServers == 0) break; - - // We want to sample from each server w.p. numSegmentsOnServer / totalSegments - ServerHolder fromServerHolder = sampleServer(); - - // and actually pick that segment uniformly at random w.p. 1 / numSegmentsOnServer - // so that the probability of picking a segment is 1 / totalSegments. - List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); - - if (segments.isEmpty()) continue; - - DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); - if (movingSegments.contains(proposalSegment)) { - continue; - } - - BalancerCostAnalyzerHelper helper = new BalancerCostAnalyzerHelper( - serverHolderList, - proposalSegment, - fromServerHolder, - segmentHoldersToMove - ); - - Pair minPair = helper.getMinPair(); - - if (minPair.rhs != null && !minPair.rhs.equals(fromServerHolder)) { - movingSegments.add(proposalSegment); - segmentHoldersToMove.add( - new BalancerSegmentHolder( - fromServerHolder.getServer(), - minPair.rhs.getServer(), - proposalSegment - ) - ); - totalCostChange += helper.getCurrCost() - minPair.lhs; - } - } - - return segmentHoldersToMove; - } } diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index 6801c4a3945..ba4b065bacb 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -88,7 +88,7 @@ public abstract class LoadRule implements Rule List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { BalancerCostAnalyzer analyzer = new BalancerCostAnalyzer(DateTime.now()); - BalancerCostAnalyzer.BalancerCostAnalyzerHelper helper = analyzer.new BalancerCostAnalyzerHelper(serverHolderList, segment); + BalancerCostAnalyzer.BalancerCostComputer helper = analyzer.new BalancerCostComputer(serverHolderList, segment); Pair minPair = helper.getMinPair(); From 617907d85bf659a97c59883be2bc1d37eed0e911 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Tue, 8 Jan 2013 16:12:16 -0800 Subject: [PATCH 014/121] move empty server holder list check outside of loop --- .../com/metamx/druid/master/BalancerCostAnalyzer.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index a7dae0925b8..717e3a333a1 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -188,17 +188,15 @@ public class BalancerCostAnalyzer { final Set segmentHoldersToMove = Sets.newHashSet(); final Set movingSegments = Sets.newHashSet(); - int numServers = serverHolderList.size(); + if (serverHolderList.isEmpty()) { + return segmentHoldersToMove; + } int counter = 0; while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 3 * MAX_SEGMENTS_TO_MOVE) { counter++; - if (numServers == 0) { - break; - } - // We want to sample from each server w.p. numSegmentsOnServer / totalSegments ServerHolder fromServerHolder = sampleServer(); From 5f40dc6d9c66011785afc83514e7e4691e854e8b Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Thu, 10 Jan 2013 10:27:43 -0800 Subject: [PATCH 015/121] put BalancerCostAnalyzer into params --- .../com/metamx/druid/master/DruidMaster.java | 3 +- .../druid/master/DruidMasterBalancer.java | 6 +-- .../master/DruidMasterRuntimeParams.java | 38 ++++++++++++++----- .../metamx/druid/master/rules/LoadRule.java | 7 ++-- .../druid/master/DruidMasterBalancerTest.java | 6 ++- .../master/DruidMasterRuleRunnerTest.java | 6 +++ 6 files changed, 46 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 386b85cfbb0..b9748ac7f1c 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -676,12 +676,13 @@ public class DruidMaster .withDruidCluster(cluster) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) + .withBalancerCostAnalyzer(new BalancerCostAnalyzer(DateTime.now())) .build(); } }, new DruidMasterRuleRunner(DruidMaster.this), new DruidMasterCleanup(DruidMaster.this), - new DruidMasterBalancer(DruidMaster.this, new BalancerCostAnalyzer(DateTime.now())), + new DruidMasterBalancer(DruidMaster.this), new DruidMasterLogger() ) ); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index ab0aba86943..ac1ae7283c4 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -51,18 +51,15 @@ public class DruidMasterBalancer implements DruidMasterHelper ); private static final EmittingLogger log = new EmittingLogger(DruidMasterBalancer.class); - private final BalancerCostAnalyzer analyzer; private final DruidMaster master; private final Map> currentlyMovingSegments = Maps.newHashMap(); public DruidMasterBalancer( - DruidMaster master, - BalancerCostAnalyzer analyzer + DruidMaster master ) { this.master = master; - this.analyzer = analyzer; } private void reduceLifetimes(String tier) @@ -106,6 +103,7 @@ public class DruidMasterBalancer implements DruidMasterHelper List serverHolderList = new ArrayList(entry.getValue()); + BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(); analyzer.init(serverHolderList, params); moveSegments(analyzer.findSegmentsToMove(), params); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java index 0357baa4d64..ed2843d9faa 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java @@ -50,6 +50,7 @@ public class DruidMasterRuntimeParams private final long mergeBytesLimit; private final int mergeSegmentsLimit; private final int maxSegmentsToMove; + private final BalancerCostAnalyzer balancerCostAnalyzer; public DruidMasterRuntimeParams( long startTime, @@ -64,7 +65,8 @@ public class DruidMasterRuntimeParams MasterStats stats, long mergeBytesLimit, int mergeSegmentsLimit, - int maxSegmentsToMove + int maxSegmentsToMove, + BalancerCostAnalyzer balancerCostAnalyzer ) { this.startTime = startTime; @@ -80,11 +82,7 @@ public class DruidMasterRuntimeParams this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; - } - - public int getMaxSegmentsToMove() - { - return maxSegmentsToMove; + this.balancerCostAnalyzer = balancerCostAnalyzer; } public long getStartTime() @@ -147,6 +145,16 @@ public class DruidMasterRuntimeParams return mergeSegmentsLimit; } + public int getMaxSegmentsToMove() + { + return maxSegmentsToMove; + } + + public BalancerCostAnalyzer getBalancerCostAnalyzer() + { + return balancerCostAnalyzer; + } + public boolean hasDeletionWaitTimeElapsed() { return (System.currentTimeMillis() - getStartTime() > getMillisToWaitBeforeDeleting()); @@ -172,7 +180,8 @@ public class DruidMasterRuntimeParams stats, mergeBytesLimit, mergeSegmentsLimit, - maxSegmentsToMove + maxSegmentsToMove, + balancerCostAnalyzer ); } @@ -191,6 +200,7 @@ public class DruidMasterRuntimeParams private long mergeBytesLimit; private int mergeSegmentsLimit; private int maxSegmentsToMove; + private BalancerCostAnalyzer balancerCostAnalyzer; Builder() { @@ -207,6 +217,7 @@ public class DruidMasterRuntimeParams this.mergeBytesLimit = 0; this.mergeSegmentsLimit = 0; this.maxSegmentsToMove = 0; + this.balancerCostAnalyzer = null; } Builder( @@ -222,7 +233,8 @@ public class DruidMasterRuntimeParams MasterStats stats, long mergeBytesLimit, int mergeSegmentsLimit, - int maxSegmentsToMove + int maxSegmentsToMove, + BalancerCostAnalyzer balancerCostAnalyzer ) { this.startTime = startTime; @@ -238,6 +250,7 @@ public class DruidMasterRuntimeParams this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; + this.balancerCostAnalyzer = balancerCostAnalyzer; } public DruidMasterRuntimeParams build() @@ -255,7 +268,8 @@ public class DruidMasterRuntimeParams stats, mergeBytesLimit, mergeSegmentsLimit, - maxSegmentsToMove + maxSegmentsToMove, + balancerCostAnalyzer ); } @@ -336,5 +350,11 @@ public class DruidMasterRuntimeParams this.maxSegmentsToMove = maxSegmentsToMove; return this; } + + public Builder withBalancerCostAnalyzer(BalancerCostAnalyzer balancerCostAnalyzer) + { + this.balancerCostAnalyzer = balancerCostAnalyzer; + return this; + } } } diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index ba4b065bacb..c18d7d8db87 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -31,7 +31,6 @@ import com.metamx.druid.master.LoadPeonCallback; import com.metamx.druid.master.MasterStats; import com.metamx.druid.master.ServerHolder; import com.metamx.emitter.EmittingLogger; -import org.joda.time.DateTime; import java.util.ArrayList; import java.util.List; @@ -65,7 +64,7 @@ public abstract class LoadRule implements Rule totalReplicants, serverQueue, segment, - params.getMaxSegmentsToMove() + params ) ); stats.accumulate(drop(expectedReplicants, clusterReplicants, segment, params)); @@ -78,7 +77,7 @@ public abstract class LoadRule implements Rule int totalReplicants, MinMaxPriorityQueue serverQueue, DataSegment segment, - int MAX_SEGMENTS_TO_MOVE + DruidMasterRuntimeParams params ) { MasterStats stats = new MasterStats(); @@ -87,7 +86,7 @@ public abstract class LoadRule implements Rule List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { - BalancerCostAnalyzer analyzer = new BalancerCostAnalyzer(DateTime.now()); + BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(); BalancerCostAnalyzer.BalancerCostComputer helper = analyzer.new BalancerCostComputer(serverHolderList, segment); Pair minPair = helper.getMinPair(); diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 082c4a78077..4e677acbae1 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -224,9 +224,10 @@ public class DruidMasterBalancerTest .withLoadManagementPeons(ImmutableMap.of("from", peon, "to", peon)) .withAvailableSegments(segments.values()) .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) + .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) .build(); - params = new DruidMasterBalancer(master, new BalancerCostAnalyzer(new DateTime("2013-01-01"))).run(params); + params = new DruidMasterBalancer(master).run(params); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("costChange").get("normal").get() > 0); } @@ -373,9 +374,10 @@ public class DruidMasterBalancerTest .withLoadManagementPeons(ImmutableMap.of("1", peon, "2", peon, "3", peon, "4", peon)) .withAvailableSegments(segments.values()) .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) + .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) .build(); - params = new DruidMasterBalancer(master, new BalancerCostAnalyzer(new DateTime("2013-01-01"))).run(params); + params = new DruidMasterBalancer(master).run(params); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("costChange").get("normal").get() > 0); } diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java index 074d61225c1..17e4ef3bd0b 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java @@ -176,6 +176,7 @@ public class DruidMasterRuleRunnerTest .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withMaxSegmentsToMove(5) + .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); @@ -265,6 +266,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); @@ -350,6 +352,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) + .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); @@ -411,6 +414,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) .build(); @@ -675,6 +679,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) + .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); @@ -751,6 +756,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) + .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); From ef9e077d9a1b858c72aab11778983b3bad2fd23d Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 14 Jan 2013 16:01:29 -0800 Subject: [PATCH 016/121] remove getConfig method from DruidMaster --- .../src/main/java/com/metamx/druid/master/DruidMaster.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index b9748ac7f1c..27049da1d27 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -132,11 +132,6 @@ public class DruidMaster return master; } - public DruidMasterConfig getConfig() - { - return config; - } - public Map getLoadStatus() { Map availableSegmentMap = Maps.newHashMap(); From e847faf02fdc95854b8d51655564e3cd28b10cbd Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Tue, 15 Jan 2013 11:29:46 -0800 Subject: [PATCH 017/121] change name of MAX_SEGMENTS_TO_MOVE since it's not a final --- .../java/com/metamx/druid/master/BalancerCostAnalyzer.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 717e3a333a1..8a6f2c17e28 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -48,7 +48,7 @@ public class BalancerCostAnalyzer private static final int THIRTY_DAYS_IN_MILLIS = 30 * DAY_IN_MILLIS; private final Random rand; private final DateTime referenceTimestamp; - private int MAX_SEGMENTS_TO_MOVE; + private int maxSegmentsToMove; private List serverHolderList; private double initialTotalCost; private double normalization; @@ -70,7 +70,7 @@ public class BalancerCostAnalyzer this.initialTotalCost = calculateInitialTotalCost(serverHolderList); this.normalization = calculateNormalization(serverHolderList); this.totalSegments = params.getAvailableSegments().size(); - this.MAX_SEGMENTS_TO_MOVE = params.getMaxSegmentsToMove(); + this.maxSegmentsToMove = params.getMaxSegmentsToMove(); } public double getInitialTotalCost() @@ -194,7 +194,7 @@ public class BalancerCostAnalyzer int counter = 0; - while (segmentHoldersToMove.size() < MAX_SEGMENTS_TO_MOVE && counter < 3 * MAX_SEGMENTS_TO_MOVE) { + while (segmentHoldersToMove.size() < maxSegmentsToMove && counter < 3 * maxSegmentsToMove) { counter++; // We want to sample from each server w.p. numSegmentsOnServer / totalSegments From 2d7113b263322c68ca5adca19ca17e01afcc7be6 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 21 Jan 2013 14:28:25 -0800 Subject: [PATCH 018/121] huge simplification of balancing code --- .../druid/master/BalancerCostAnalyzer.java | 256 +++--------------- .../druid/master/DruidMasterBalancer.java | 122 +++++---- .../metamx/druid/master/rules/LoadRule.java | 28 +- .../druid/master/DruidMasterBalancerTest.java | 2 - 4 files changed, 107 insertions(+), 301 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 8a6f2c17e28..56563cb1521 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -20,19 +20,13 @@ package com.metamx.druid.master; import com.google.common.collect.Lists; -import com.google.common.collect.MinMaxPriorityQueue; -import com.google.common.collect.Sets; -import com.metamx.common.Pair; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; import org.joda.time.DateTime; import org.joda.time.Interval; -import java.util.Comparator; import java.util.List; import java.util.Random; -import java.util.Set; /** @@ -48,56 +42,18 @@ public class BalancerCostAnalyzer private static final int THIRTY_DAYS_IN_MILLIS = 30 * DAY_IN_MILLIS; private final Random rand; private final DateTime referenceTimestamp; - private int maxSegmentsToMove; - private List serverHolderList; - private double initialTotalCost; - private double normalization; - private double totalCostChange; - private int totalSegments; public BalancerCostAnalyzer(DateTime referenceTimestamp) { this.referenceTimestamp = referenceTimestamp; - rand = new Random(0); - totalCostChange = 0; - } - - // The assignment usage doesn't require initialization. - public void init(List serverHolderList, DruidMasterRuntimeParams params) - { - this.serverHolderList = serverHolderList; - this.initialTotalCost = calculateInitialTotalCost(serverHolderList); - this.normalization = calculateNormalization(serverHolderList); - this.totalSegments = params.getAvailableSegments().size(); - this.maxSegmentsToMove = params.getMaxSegmentsToMove(); - } - - public double getInitialTotalCost() - { - return initialTotalCost; - } - - public double getNormalization() - { - return normalization; - } - - public double getNormalizedInitialCost() - { - return initialTotalCost / normalization; - } - - public double getTotalCostChange() - { - return totalCostChange; } /* * Calculates the cost normalization. This is such that the normalized cost is lower bounded * by 1 (e.g. when each segment gets its own compute node). */ - private double calculateNormalization(List serverHolderList) + public double calculateNormalization(List serverHolderList) { double cost = 0; for (ServerHolder server : serverHolderList) { @@ -109,7 +65,7 @@ public class BalancerCostAnalyzer } // Calculates the initial cost of the Druid segment configuration. - private double calculateInitialTotalCost(List serverHolderList) + public double calculateInitialTotalCost(List serverHolderList) { double cost = 0; for (ServerHolder server : serverHolderList) { @@ -170,9 +126,9 @@ public class BalancerCostAnalyzer /* * Sample from each server with probability proportional to the number of segments on that server. */ - private ServerHolder sampleServer() + private ServerHolder sampleServer(List serverHolderList, int numSegments) { - final int num = rand.nextInt(totalSegments); + final int num = rand.nextInt(numSegments); int cumulativeSegments = 0; int numToStopAt = 0; @@ -184,193 +140,57 @@ public class BalancerCostAnalyzer return serverHolderList.get(numToStopAt - 1); } - public Set findSegmentsToMove() + // The balancing application requires us to pick a proposal segment. + public BalancerSegmentHolder findNewSegmentHome(List serverHolders, int numSegments) { - final Set segmentHoldersToMove = Sets.newHashSet(); - final Set movingSegments = Sets.newHashSet(); - if (serverHolderList.isEmpty()) { - return segmentHoldersToMove; - } + // We want to sample from each server w.p. numSegmentsOnServer / totalSegments + ServerHolder fromServerHolder = sampleServer(serverHolders, numSegments); - int counter = 0; + // and actually pick that segment uniformly at random w.p. 1 / numSegmentsOnServer + // so that the probability of picking a segment is 1 / totalSegments. + List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); - while (segmentHoldersToMove.size() < maxSegmentsToMove && counter < 3 * maxSegmentsToMove) { - counter++; + DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); + ServerHolder toServer = findNewSegmentHome(proposalSegment, serverHolders); - // We want to sample from each server w.p. numSegmentsOnServer / totalSegments - ServerHolder fromServerHolder = sampleServer(); - - // and actually pick that segment uniformly at random w.p. 1 / numSegmentsOnServer - // so that the probability of picking a segment is 1 / totalSegments. - List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); - - if (segments.isEmpty()) { - continue; - } - - DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); - if (movingSegments.contains(proposalSegment)) { - continue; - } - - BalancerCostComputer helper = new BalancerCostComputer( - serverHolderList, - proposalSegment, - fromServerHolder, - segmentHoldersToMove - ); - - Pair minPair = helper.getMinPair(); - - if (minPair.rhs != null && !minPair.rhs.equals(fromServerHolder)) { - movingSegments.add(proposalSegment); - segmentHoldersToMove.add( - new BalancerSegmentHolder( - fromServerHolder.getServer(), - minPair.rhs.getServer(), - proposalSegment - ) - ); - totalCostChange += helper.getCurrCost() - minPair.lhs; - } - } - - return segmentHoldersToMove; + return new BalancerSegmentHolder(fromServerHolder.getServer(), toServer.getServer(), proposalSegment); } - /* - * These could be anonymous in BalancerCostComputer - * Their purpose is to unify the balance/assignment code since a segment that has not yet been assigned - * does not have a source server. - */ - public static class NullServerHolder extends ServerHolder + // The assignment application requires us to supply a proposal segment. + public ServerHolder findNewSegmentHome(DataSegment proposalSegment, Iterable serverHolders) { - public NullServerHolder() - { - super(null, null); - } + final long proposalSegmentSize = proposalSegment.getSize(); + double minCost = Double.MAX_VALUE; + ServerHolder toServer = null; - @Override - public DruidServer getServer() - { - return new NullDruidServer(); - } - - public static class NullDruidServer extends DruidServer - { - public NullDruidServer() - { - super(null, null, 0, null, null); + for (ServerHolder server : serverHolders) { + // Only calculate costs if the server has enough space. + if (proposalSegmentSize > server.getAvailableSize()) { + break; } - @Override - public boolean equals(Object o) - { - return false; - } - } - } - - public class BalancerCostComputer - { - private final List serverHolderList; - private final DataSegment proposalSegment; - private final ServerHolder fromServerHolder; - private final Set segmentHoldersToMove; - private Pair minPair; - private double currCost; - - public BalancerCostComputer( - List serverHolderList, - DataSegment proposalSegment - ) - { - this(serverHolderList, proposalSegment, new NullServerHolder(), Sets.newHashSet()); - } - - public BalancerCostComputer( - List serverHolderList, - DataSegment proposalSegment, - ServerHolder fromServerHolder, - Set segmentHoldersToMove - ) - { - this.serverHolderList = serverHolderList; - this.proposalSegment = proposalSegment; - this.fromServerHolder = fromServerHolder; - this.segmentHoldersToMove = segmentHoldersToMove; - this.currCost = 0; - - computeAllCosts(); - } - - public Pair getMinPair() - { - return minPair; - } - - public double getCurrCost() - { - return currCost; - } - - public void computeAllCosts() - { - // Just need a regular priority queue for the min. element. - MinMaxPriorityQueue> costsServerHolderPairs = MinMaxPriorityQueue.orderedBy( - new Comparator>() - { - @Override - public int compare( - Pair o, - Pair o1 - ) - { - return Double.compare(o.lhs, o1.lhs); - } - } - ).create(); - - for (ServerHolder server : serverHolderList) { - // Only calculate costs if the server has enough space. - if (proposalSegment.getSize() > server.getAvailableSize()) { - break; - } - - // The contribution to the total cost of a given server by proposing to move the segment to that server is... - double cost = 0f; - // the sum of the costs of other (inclusive) segments on the server - for (DataSegment segment : server.getServer().getSegments().values()) { + // The contribution to the total cost of a given server by proposing to move the segment to that server is... + double cost = 0f; + // the sum of the costs of other (exclusive of the proposalSegment) segments on the server + for (DataSegment segment : server.getServer().getSegments().values()) { + if (!proposalSegment.equals(segment)) { cost += computeJointSegmentCosts(proposalSegment, segment); } - - // plus the self cost if the proposed new server is different - if (!fromServerHolder.getServer().equals(server.getServer())) { - cost += computeJointSegmentCosts(proposalSegment, proposalSegment); - } - - // plus the costs of segments that will be moved. - for (BalancerSegmentHolder segmentToMove : segmentHoldersToMove) { - if (server.getServer().equals(segmentToMove.getToServer())) { - cost += computeJointSegmentCosts(proposalSegment, segmentToMove.getSegment()); - } - if (server.getServer().equals(segmentToMove.getFromServer())) { - cost -= computeJointSegmentCosts(proposalSegment, segmentToMove.getSegment()); - } - } - - // currCost keeps track of the current cost for that server (so we can compute the cost change). - if (fromServerHolder.getServer().equals(server.getServer())) { - currCost = cost; - } - - costsServerHolderPairs.add(Pair.of(cost, server)); + } + // plus the costs of segments that will be loaded + for (DataSegment segment : server.getPeon().getSegmentsToLoad()) { + cost += computeJointSegmentCosts(proposalSegment, segment); } - minPair = costsServerHolderPairs.pollFirst(); + if (cost < minCost) { + minCost = cost; + toServer = server; + } } + return toServer; } + } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index ac1ae7283c4..d63b1226212 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -22,7 +22,6 @@ package com.metamx.druid.master; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; -import com.google.common.collect.Sets; import com.metamx.common.guava.Comparators; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; @@ -30,6 +29,7 @@ import com.metamx.emitter.EmittingLogger; import java.util.ArrayList; import java.util.Comparator; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -81,7 +81,9 @@ public class DruidMasterBalancer implements DruidMasterHelper @Override public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) { - MasterStats stats = new MasterStats(); + final MasterStats stats = new MasterStats(); + final BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(); + final int maxSegmentsToMove = params.getMaxSegmentsToMove(); for (Map.Entry> entry : params.getDruidCluster().getCluster().entrySet()) { @@ -101,28 +103,42 @@ public class DruidMasterBalancer implements DruidMasterHelper continue; } - List serverHolderList = new ArrayList(entry.getValue()); + final List serverHolderList = new ArrayList(entry.getValue()); + int numSegments = 0; + for (ServerHolder server : serverHolderList) { + numSegments += server.getServer().getSegments().size(); + } - BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(); - analyzer.init(serverHolderList, params); - moveSegments(analyzer.findSegmentsToMove(), params); + if (numSegments == 0) { + log.info("No segments found. Cannot balance."); + continue; + } - double initialTotalCost = analyzer.getInitialTotalCost(); - double normalization = analyzer.getNormalization(); - double normalizedInitialCost = analyzer.getNormalizedInitialCost(); - double costChange = analyzer.getTotalCostChange(); + final Set segmentsBeingMoved = new HashSet(); + int iter = 0; + + while (iter < maxSegmentsToMove) { + BalancerSegmentHolder holder = analyzer.findNewSegmentHome(serverHolderList, numSegments); + if (!segmentsBeingMoved.contains(holder.getSegment())) { + moveSegment(holder, params); + segmentsBeingMoved.add(holder.getSegment()); + } + iter++; + } + + final double initialTotalCost = analyzer.calculateInitialTotalCost(serverHolderList); + final double normalization = analyzer.calculateNormalization(serverHolderList); + final double normalizedInitialCost = initialTotalCost / normalization; stats.addToTieredStat("initialCost", tier, (long) initialTotalCost); stats.addToTieredStat("normalization", tier, (long) normalization); - stats.addToTieredStat("costChange", tier, (long) costChange); + stats.addToTieredStat("movedCount", tier, segmentsBeingMoved.size()); log.info( - "Initial Total Cost: [%f], Initial Normalized Cost: [%f], Cost Change: [%f], Normalized Cost Change: [%f], New Normalized Cost: [%f]", + "Initial Total Cost: [%f], Initial Normalized Cost: [%f], Segments Moved: [%d]", initialTotalCost, normalizedInitialCost, - costChange, - costChange / normalization, - (initialTotalCost - costChange) / normalization + segmentsBeingMoved.size() ); if (serverHolderList.size() <= 1) { @@ -133,7 +149,6 @@ public class DruidMasterBalancer implements DruidMasterHelper continue; } - stats.addToTieredStat("movedCount", tier, currentlyMovingSegments.get(tier).size()); } return params.buildFromExisting() @@ -141,55 +156,52 @@ public class DruidMasterBalancer implements DruidMasterHelper .build(); } - private void moveSegments( - final Set segments, + private void moveSegment( + final BalancerSegmentHolder segment, final DruidMasterRuntimeParams params ) { + final DruidServer toServer = segment.getToServer(); + final String toServerName = segment.getToServer().getName(); + final LoadQueuePeon toPeon = params.getLoadManagementPeons().get(toServerName); - for (final BalancerSegmentHolder segment : Sets.newHashSet(segments)) { - final DruidServer toServer = segment.getToServer(); - final String toServerName = segment.getToServer().getName(); - LoadQueuePeon toPeon = params.getLoadManagementPeons().get(toServerName); + final String fromServer = segment.getFromServer().getName(); + final DataSegment segmentToMove = segment.getSegment(); + final String segmentName = segmentToMove.getIdentifier(); - String fromServer = segment.getFromServer().getName(); - DataSegment segmentToMove = segment.getSegment(); - final String segmentName = segmentToMove.getIdentifier(); - - if (!toPeon.getSegmentsToLoad().contains(segmentToMove) && - (toServer.getSegment(segmentName) == null) && - new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) { - log.info( - "Moving [%s] from [%s] to [%s]", - segmentName, + if (!toPeon.getSegmentsToLoad().contains(segmentToMove) && + (toServer.getSegment(segmentName) == null) && + new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) { + log.info( + "Moving [%s] from [%s] to [%s]", + segmentName, + fromServer, + toServerName + ); + try { + master.moveSegment( fromServer, - toServerName - ); - try { - master.moveSegment( - fromServer, - toServerName, - segmentToMove.getIdentifier(), - new LoadPeonCallback() + toServerName, + segmentToMove.getIdentifier(), + new LoadPeonCallback() + { + @Override + protected void execute() { - @Override - protected void execute() - { - Map movingSegments = currentlyMovingSegments.get(toServer.getTier()); - if (movingSegments != null) { - movingSegments.remove(segmentName); - } + Map movingSegments = currentlyMovingSegments.get(toServer.getTier()); + if (movingSegments != null) { + movingSegments.remove(segmentName); } } - ); - currentlyMovingSegments.get(toServer.getTier()).put(segmentName, segment); - } - catch (Exception e) { - log.makeAlert(e, String.format("[%s] : Moving exception", segmentName)).emit(); - } - } else { - currentlyMovingSegments.get(toServer.getTier()).remove(segment); + } + ); + currentlyMovingSegments.get(toServer.getTier()).put(segmentName, segment); } + catch (Exception e) { + log.makeAlert(e, String.format("[%s] : Moving exception", segmentName)).emit(); + } + } else { + currentlyMovingSegments.get(toServer.getTier()).remove(segment); } } diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index c18d7d8db87..c77621b4f14 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -87,14 +87,11 @@ public abstract class LoadRule implements Rule List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(); - BalancerCostAnalyzer.BalancerCostComputer helper = analyzer.new BalancerCostComputer(serverHolderList, segment); + ServerHolder holder = analyzer.findNewSegmentHome(segment, serverHolderList); - Pair minPair = helper.getMinPair(); - - ServerHolder holder = minPair.rhs; if (holder == null) { log.warn( - "Not enough %s servers[%d] to assign segment[%s]! Expected Replicants[%d]", + "Not enough %s servers[%d] or node capacity to assign segment[%s]! Expected Replicants[%d]", getTier(), assignedServers.size() + serverQueue.size() + 1, segment.getIdentifier(), @@ -107,27 +104,6 @@ public abstract class LoadRule implements Rule continue; } - if (holder.getAvailableSize() < segment.getSize()) { - log.warn( - "Not enough node capacity, closest is [%s] with %,d available, skipping segment[%s].", - holder.getServer(), - holder.getAvailableSize(), - segment - ); - log.makeAlert( - "Not enough node capacity", - ImmutableMap.builder() - .put("segmentSkipped", segment.toString()) - .put("closestNode", holder.getServer().toString()) - .put("availableSize", holder.getAvailableSize()) - .build() - ).emit(); - serverQueue.add(holder); - stats.addToTieredStat("unassignedCount", getTier(), 1); - stats.addToTieredStat("unassignedSize", getTier(), segment.getSize()); - break; - } - holder.getPeon().loadSegment( segment, new LoadPeonCallback() diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 4e677acbae1..ad29249770a 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -229,7 +229,6 @@ public class DruidMasterBalancerTest params = new DruidMasterBalancer(master).run(params); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); - Assert.assertTrue(params.getMasterStats().getPerTierStats().get("costChange").get("normal").get() > 0); } @Test @@ -379,6 +378,5 @@ public class DruidMasterBalancerTest params = new DruidMasterBalancer(master).run(params); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); - Assert.assertTrue(params.getMasterStats().getPerTierStats().get("costChange").get("normal").get() > 0); } } From 94b72e887822653bb2118de21a1c405f76288d61 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 21 Jan 2013 15:32:29 -0800 Subject: [PATCH 019/121] replace param BalancerCostAnalyzer getter with a factory --- .../com/metamx/druid/master/DruidMaster.java | 2 +- .../druid/master/DruidMasterBalancer.java | 4 ++- .../master/DruidMasterRuntimeParams.java | 32 +++++++++++-------- .../metamx/druid/master/rules/LoadRule.java | 4 ++- .../druid/master/DruidMasterBalancerTest.java | 4 +-- .../master/DruidMasterRuleRunnerTest.java | 12 +++---- 6 files changed, 34 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 27049da1d27..fcb17123066 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -671,7 +671,7 @@ public class DruidMaster .withDruidCluster(cluster) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerCostAnalyzer(new BalancerCostAnalyzer(DateTime.now())) + .withBalancerReferenceTimestamp(DateTime.now()) .build(); } }, diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index d63b1226212..19cfd89274f 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -26,6 +26,7 @@ import com.metamx.common.guava.Comparators; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; import com.metamx.emitter.EmittingLogger; +import org.joda.time.DateTime; import java.util.ArrayList; import java.util.Comparator; @@ -82,7 +83,8 @@ public class DruidMasterBalancer implements DruidMasterHelper public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) { final MasterStats stats = new MasterStats(); - final BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(); + final DateTime referenceTimestamp = params.getBalancerReferenceTimestamp(); + final BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(referenceTimestamp); final int maxSegmentsToMove = params.getMaxSegmentsToMove(); for (Map.Entry> entry : diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java index ed2843d9faa..fb56033c70f 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java @@ -27,6 +27,7 @@ import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.master.rules.RuleMap; import com.metamx.emitter.service.ServiceEmitter; +import org.joda.time.DateTime; import java.util.Collection; import java.util.Collections; @@ -50,7 +51,7 @@ public class DruidMasterRuntimeParams private final long mergeBytesLimit; private final int mergeSegmentsLimit; private final int maxSegmentsToMove; - private final BalancerCostAnalyzer balancerCostAnalyzer; + private final DateTime balancerReferenceTimestamp; public DruidMasterRuntimeParams( long startTime, @@ -66,7 +67,7 @@ public class DruidMasterRuntimeParams long mergeBytesLimit, int mergeSegmentsLimit, int maxSegmentsToMove, - BalancerCostAnalyzer balancerCostAnalyzer + DateTime balancerReferenceTimestamp ) { this.startTime = startTime; @@ -82,7 +83,7 @@ public class DruidMasterRuntimeParams this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; - this.balancerCostAnalyzer = balancerCostAnalyzer; + this.balancerReferenceTimestamp = balancerReferenceTimestamp; } public long getStartTime() @@ -150,9 +151,14 @@ public class DruidMasterRuntimeParams return maxSegmentsToMove; } - public BalancerCostAnalyzer getBalancerCostAnalyzer() + public DateTime getBalancerReferenceTimestamp() { - return balancerCostAnalyzer; + return balancerReferenceTimestamp; + } + + public BalancerCostAnalyzer getBalancerCostAnalyzer(DateTime referenceTimestamp) + { + return new BalancerCostAnalyzer(referenceTimestamp); } public boolean hasDeletionWaitTimeElapsed() @@ -181,7 +187,7 @@ public class DruidMasterRuntimeParams mergeBytesLimit, mergeSegmentsLimit, maxSegmentsToMove, - balancerCostAnalyzer + balancerReferenceTimestamp ); } @@ -200,7 +206,7 @@ public class DruidMasterRuntimeParams private long mergeBytesLimit; private int mergeSegmentsLimit; private int maxSegmentsToMove; - private BalancerCostAnalyzer balancerCostAnalyzer; + private DateTime balancerReferenceTimestamp; Builder() { @@ -217,7 +223,7 @@ public class DruidMasterRuntimeParams this.mergeBytesLimit = 0; this.mergeSegmentsLimit = 0; this.maxSegmentsToMove = 0; - this.balancerCostAnalyzer = null; + this.balancerReferenceTimestamp = null; } Builder( @@ -234,7 +240,7 @@ public class DruidMasterRuntimeParams long mergeBytesLimit, int mergeSegmentsLimit, int maxSegmentsToMove, - BalancerCostAnalyzer balancerCostAnalyzer + DateTime balancerReferenceTimestamp ) { this.startTime = startTime; @@ -250,7 +256,7 @@ public class DruidMasterRuntimeParams this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; - this.balancerCostAnalyzer = balancerCostAnalyzer; + this.balancerReferenceTimestamp = balancerReferenceTimestamp; } public DruidMasterRuntimeParams build() @@ -269,7 +275,7 @@ public class DruidMasterRuntimeParams mergeBytesLimit, mergeSegmentsLimit, maxSegmentsToMove, - balancerCostAnalyzer + balancerReferenceTimestamp ); } @@ -351,9 +357,9 @@ public class DruidMasterRuntimeParams return this; } - public Builder withBalancerCostAnalyzer(BalancerCostAnalyzer balancerCostAnalyzer) + public Builder withBalancerReferenceTimestamp(DateTime balancerReferenceTimestamp) { - this.balancerCostAnalyzer = balancerCostAnalyzer; + this.balancerReferenceTimestamp = balancerReferenceTimestamp; return this; } } diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index c77621b4f14..05d9adf35ca 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -31,6 +31,7 @@ import com.metamx.druid.master.LoadPeonCallback; import com.metamx.druid.master.MasterStats; import com.metamx.druid.master.ServerHolder; import com.metamx.emitter.EmittingLogger; +import org.joda.time.DateTime; import java.util.ArrayList; import java.util.List; @@ -86,7 +87,8 @@ public abstract class LoadRule implements Rule List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { - BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(); + final DateTime referenceTimestamp = params.getBalancerReferenceTimestamp(); + final BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(referenceTimestamp); ServerHolder holder = analyzer.findNewSegmentHome(segment, serverHolderList); if (holder == null) { diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index ad29249770a..3470f53633e 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -224,7 +224,7 @@ public class DruidMasterBalancerTest .withLoadManagementPeons(ImmutableMap.of("from", peon, "to", peon)) .withAvailableSegments(segments.values()) .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) - .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); params = new DruidMasterBalancer(master).run(params); @@ -373,7 +373,7 @@ public class DruidMasterBalancerTest .withLoadManagementPeons(ImmutableMap.of("1", peon, "2", peon, "3", peon, "4", peon)) .withAvailableSegments(segments.values()) .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) - .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); params = new DruidMasterBalancer(master).run(params); diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java index 17e4ef3bd0b..c08a3cb404d 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java @@ -176,7 +176,7 @@ public class DruidMasterRuleRunnerTest .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withMaxSegmentsToMove(5) - .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); @@ -266,7 +266,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); @@ -352,7 +352,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); @@ -414,7 +414,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); @@ -679,7 +679,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); @@ -756,7 +756,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerCostAnalyzer(new BalancerCostAnalyzer(new DateTime("2013-01-01"))) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); From fc07bc315eacf26ad0d8dca0ae1cdf18b9bf5040 Mon Sep 17 00:00:00 2001 From: Deep Ganguli Date: Wed, 23 Jan 2013 18:59:51 -0800 Subject: [PATCH 020/121] Added umbrellaInterval method, which takes an Iterable of intervals and returns a single interval spanning the entire range of input intervals. --- .../com/metamx/druid/utils/JodaUtils.java | 30 ++++++++++++++++ .../com/metamx/druid/utils/JodaUtilsTest.java | 34 +++++++++++++++++++ 2 files changed, 64 insertions(+) diff --git a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java index 53b14618001..3932ef08153 100644 --- a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java +++ b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java @@ -19,6 +19,7 @@ package com.metamx.druid.utils; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.guava.Comparators; @@ -63,6 +64,35 @@ public class JodaUtils return retVal; } + public static Interval umbrellaInterval(Iterable intervals) + { + DateTime minStart = null; + DateTime maxEnd = null; + + for (Interval interval : intervals) { + DateTime curStart = interval.getStart(); + DateTime curEnd = interval.getEnd(); + + if (minStart == null || maxEnd == null) { + minStart = curStart; + maxEnd = curEnd; + } + + if (curStart.isBefore(minStart)) { + minStart = curStart; + } + + if (curEnd.isAfter(maxEnd)) { + maxEnd = curEnd; + } + } + + if (minStart == null || maxEnd == null) { + throw new IllegalArgumentException("Empty list of intervals"); + } + return new Interval(minStart, maxEnd); + } + public static DateTime minDateTime(DateTime... times) { if (times == null) { diff --git a/common/src/test/java/com/metamx/druid/utils/JodaUtilsTest.java b/common/src/test/java/com/metamx/druid/utils/JodaUtilsTest.java index 6abaf485b68..f0248295dfd 100644 --- a/common/src/test/java/com/metamx/druid/utils/JodaUtilsTest.java +++ b/common/src/test/java/com/metamx/druid/utils/JodaUtilsTest.java @@ -31,6 +31,40 @@ import java.util.List; */ public class JodaUtilsTest { + @Test + public void testUmbrellaIntervalsSimple() throws Exception + { + List intervals = Arrays.asList( + new Interval("2011-03-03/2011-03-04"), + new Interval("2011-01-01/2011-01-02"), + new Interval("2011-02-01/2011-02-05"), + new Interval("2011-02-03/2011-02-08"), + new Interval("2011-01-01/2011-01-03"), + new Interval("2011-03-01/2011-03-02"), + new Interval("2011-03-05/2011-03-06"), + new Interval("2011-02-01/2011-02-02") + ); + + Assert.assertEquals( + new Interval("2011-01-01/2011-03-06"), + JodaUtils.umbrellaInterval(intervals) + ); + } + + @Test + public void testUmbrellaIntervalsNull() throws Exception + { + List intervals = Arrays.asList(); + Throwable thrown = null; + try { + Interval res = JodaUtils.umbrellaInterval(intervals); + } + catch (IllegalArgumentException e) { + thrown = e; + } + Assert.assertNotNull("Empty list of intervals", thrown); + } + @Test public void testCondenseIntervalsSimple() throws Exception { From 017d4779d684e12917f73cb9fd323b5cc7cbc490 Mon Sep 17 00:00:00 2001 From: Deep Ganguli Date: Wed, 23 Jan 2013 19:27:14 -0800 Subject: [PATCH 021/121] Implemented Hadoop Index Task which takes as input a HadoopDruidIndexConfig and generates index segments. The HadoopIndexTask run method wraps a HadoopDruidIndexerJob run method. The key modifications to the HadoopDruidIndexerJob are as follows: - The UpDaterJobSpec field of the config that is used to set up the indexer job is set to null. This ensures that the job does not push a list of published segments to the database, in order to allow the indexing service to handle this later. - Set the version field of the config file based on the TaskContext. Also changed config.setVersion method to take a string (as opposed to a Date) as input, and propogated this change where necessary. - Set the SegmentOutputDir field of the config file based on the TaskToolbox, to allow the indexing service to handle where to write the segments too. - Added a method to IndexGeneratorJob called getPublishedSegments, that simply returns a list of published segments without publishing this list to the database. --- .../metamx/druid/indexer/DbUpdaterJob.java | 100 +++++------------- .../indexer/HadoopDruidIndexerAzkWrapper.java | 2 +- .../indexer/HadoopDruidIndexerConfig.java | 10 +- .../druid/indexer/HadoopDruidIndexerJob.java | 9 +- .../druid/indexer/IndexGeneratorJob.java | 39 ++++++- .../merger/common/task/HadoopIndexTask.java | 91 ++++++++++++++++ .../metamx/druid/merger/common/task/Task.java | 9 +- .../metamx/druid/loading/S3SegmentPusher.java | 5 + 8 files changed, 183 insertions(+), 82 deletions(-) create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java index 720242466e4..530042d289c 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java @@ -19,24 +19,18 @@ package com.metamx.druid.indexer; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; + import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.db.DbConnector; import com.metamx.druid.indexer.updater.DbUpdaterJobSpec; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.tweak.HandleCallback; -import java.io.IOException; import java.util.List; /** @@ -51,9 +45,6 @@ public class DbUpdaterJob implements Jobby private final DbUpdaterJobSpec spec; private final DBI dbi; - // Keep track of published segment identifiers, in case a client is interested. - private volatile ImmutableList publishedSegments = null; - public DbUpdaterJob( HadoopDruidIndexerConfig config ) @@ -66,75 +57,42 @@ public class DbUpdaterJob implements Jobby @Override public boolean run() { - final Configuration conf = new Configuration(); + final List segments = IndexGeneratorJob.getPublishedSegments(config); - ImmutableList.Builder publishedSegmentsBuilder = ImmutableList.builder(); + for (final DataSegment segment : segments) { - for (String propName : System.getProperties().stringPropertyNames()) { - if (propName.startsWith("hadoop.")) { - conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); - } - } - - final Path descriptorInfoDir = config.makeDescriptorInfoDir(); - - try { - FileSystem fs = descriptorInfoDir.getFileSystem(conf); - - for (FileStatus status : fs.listStatus(descriptorInfoDir)) { - final DataSegment segment = jsonMapper.readValue(fs.open(status.getPath()), DataSegment.class); - - dbi.withHandle( - new HandleCallback() + dbi.withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception { - @Override - public Void withHandle(Handle handle) throws Exception - { - handle.createStatement(String.format( - "INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - spec.getSegmentTable() - )) - .bind("id", segment.getIdentifier()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", new DateTime().toString()) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", segment.getShardSpec().getPartitionNum()) - .bind("version", segment.getVersion()) - .bind("used", true) - .bind("payload", jsonMapper.writeValueAsString(segment)) - .execute(); + handle.createStatement( + String.format( + "INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + spec.getSegmentTable() + ) + ) + .bind("id", segment.getIdentifier()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", new DateTime().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", segment.getShardSpec().getPartitionNum()) + .bind("version", segment.getVersion()) + .bind("used", true) + .bind("payload", jsonMapper.writeValueAsString(segment)) + .execute(); - return null; - } + return null; } - ); + } + ); - publishedSegmentsBuilder.add(segment); - log.info("Published %s", segment.getIdentifier()); - } + log.info("Published %s", segment.getIdentifier()); } - catch (IOException e) { - throw Throwables.propagate(e); - } - - publishedSegments = publishedSegmentsBuilder.build(); - return true; } - /** - * Returns a list of segment identifiers published by the most recent call to run(). - * Throws an IllegalStateException if run() has never been called. - */ - public List getPublishedSegments() - { - if (publishedSegments == null) { - log.error("getPublishedSegments called before run!"); - throw new IllegalStateException("DbUpdaterJob has not run yet"); - } else { - return publishedSegments; - } - } } diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.java index 8d073dd7297..5318d5cb600 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.java @@ -83,7 +83,7 @@ public class HadoopDruidIndexerAzkWrapper final HadoopDruidIndexerConfig config = jsonMapper.convertValue(theMap, HadoopDruidIndexerConfig.class); config.setIntervals(dataInterval); - config.setVersion(new DateTime()); + config.setVersion(new DateTime().toString()); new HadoopDruidIndexerJob(config).run(); } diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index 403484b9c61..7e91cba9de5 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -174,7 +174,7 @@ public class HadoopDruidIndexerConfig private volatile PathSpec pathSpec; private volatile String jobOutputDir; private volatile String segmentOutputDir; - private volatile DateTime version = new DateTime(); + private volatile String version = new DateTime().toString(); private volatile String partitionDimension; private volatile Long targetPartitionSize; private volatile boolean leaveIntermediate = false; @@ -312,12 +312,12 @@ public class HadoopDruidIndexerConfig } @JsonProperty - public DateTime getVersion() + public String getVersion() { return version; } - public void setVersion(DateTime version) + public void setVersion(String version) { this.version = version; } @@ -544,7 +544,7 @@ public class HadoopDruidIndexerConfig */ public Path makeIntermediatePath() { - return new Path(String.format("%s/%s/%s", getJobOutputDir(), dataSource, getVersion().toString().replace(":", ""))); + return new Path(String.format("%s/%s/%s", getJobOutputDir(), dataSource, getVersion().replace(":", ""))); } public Path makeSegmentPartitionInfoPath(Bucket bucket) @@ -581,7 +581,7 @@ public class HadoopDruidIndexerConfig getSegmentOutputDir(), bucketInterval.getStart().toString(), bucketInterval.getEnd().toString(), - getVersion().toString(), + getVersion(), bucket.partitionNum ) ); diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerJob.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerJob.java index eae1bfa885d..aff1265782a 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerJob.java @@ -47,6 +47,7 @@ public class HadoopDruidIndexerJob implements Jobby private final HadoopDruidIndexerConfig config; private final DbUpdaterJob dbUpdaterJob; private IndexGeneratorJob indexJob; + private volatile List publishedSegments = null; public HadoopDruidIndexerJob( HadoopDruidIndexerConfig config @@ -102,6 +103,8 @@ public class HadoopDruidIndexerJob implements Jobby } } + publishedSegments = IndexGeneratorJob.getPublishedSegments(config); + if (!config.isLeaveIntermediate()) { if (failedMessage == null || config.isCleanupOnFailure()) { Path workingPath = config.makeIntermediatePath(); @@ -147,8 +150,10 @@ public class HadoopDruidIndexerJob implements Jobby } public List getPublishedSegments() { - Preconditions.checkState(dbUpdaterJob != null, "No updaterJobSpec set, cannot get published segments"); - return dbUpdaterJob.getPublishedSegments(); + if(publishedSegments == null) { + throw new IllegalStateException("Job hasn't run yet. No segments have been published yet."); + } + return publishedSegments; } public IndexGeneratorJob.IndexGeneratorStats getIndexJobStats() diff --git a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index 28dacd1ca9a..d9188586041 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Predicate; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -44,6 +45,7 @@ import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.indexer.rollup.DataRollupSpec; import com.metamx.druid.input.MapBasedInputRow; +import com.metamx.druid.jackson.DefaultObjectMapper; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -65,8 +67,11 @@ import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; import org.joda.time.Interval; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.tweak.HandleCallback; import javax.annotation.Nullable; import java.io.BufferedOutputStream; @@ -159,6 +164,38 @@ public class IndexGeneratorJob implements Jobby } } + public static List getPublishedSegments(HadoopDruidIndexerConfig config) { + + final Configuration conf = new Configuration(); + final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.jsonMapper; + + ImmutableList.Builder publishedSegmentsBuilder = ImmutableList.builder(); + + for (String propName : System.getProperties().stringPropertyNames()) { + if (propName.startsWith("hadoop.")) { + conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); + } + } + + final Path descriptorInfoDir = config.makeDescriptorInfoDir(); + + try { + FileSystem fs = descriptorInfoDir.getFileSystem(conf); + + for (FileStatus status : fs.listStatus(descriptorInfoDir)) { + final DataSegment segment = jsonMapper.readValue(fs.open(status.getPath()), DataSegment.class); + publishedSegmentsBuilder.add(segment); + log.info("Published %s", segment.getIdentifier()); + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + List publishedSegments = publishedSegmentsBuilder.build(); + + return publishedSegments; +} + public static class IndexGeneratorMapper extends Mapper { private HadoopDruidIndexerConfig config; @@ -467,7 +504,7 @@ public class IndexGeneratorJob implements Jobby DataSegment segment = new DataSegment( config.getDataSource(), interval, - config.getVersion().toString(), + config.getVersion(), loadSpec, dimensionNames, metricNames, diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java new file mode 100644 index 00000000000..9dda89d4371 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java @@ -0,0 +1,91 @@ +package com.metamx.druid.merger.common.task; + +import com.google.common.collect.ImmutableList; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.indexer.HadoopDruidIndexerConfig; +import com.metamx.druid.indexer.HadoopDruidIndexerJob; +import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.coordinator.TaskContext; +import com.metamx.druid.utils.JodaUtils; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.joda.time.DateTime; + +import java.util.List; + +public class HadoopIndexTask extends AbstractTask +{ + @JsonProperty + private static final Logger log = new Logger(HadoopIndexTask.class); + + @JsonProperty + private final HadoopDruidIndexerConfig config; + + /** + * @param config is used by the HadoopDruidIndexerJob to set up the appropriate parameters + * for creating Druid index segments. It may be modified. Here, we will ensure that the + * UpDaterJobSpec field of the config is set to null, such that the job does not push a + * list of published segments the database. Instead, we will use the method + * IndexGeneratorJob.getPublishedSegments() to simply return a list of the published + * segments, and let the indexing service report these segments to the database. + */ + + @JsonCreator + public HadoopIndexTask(@JsonProperty("config") HadoopDruidIndexerConfig config) + { + super( + String.format("index_hadoop_%s_interval_%s", config.getDataSource(), new DateTime().now()), + config.getDataSource(), + JodaUtils.umbrellaInterval(config.getIntervals()) + ); + + if (config.isUpdaterJobSpecSet()) { + throw new IllegalArgumentException("UpDaterJobSpec is defined"); + } + this.config = config; + } + + @Override + public Type getType() + { + return Type.INDEX; + } + + @Override + public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + { + log.info("Setting version to: %s", context.getVersion()); + config.setVersion(context.getVersion()); + + if(toolbox.getSegmentPusher() instanceof S3SegmentPusher) { + // Hack alert! Bypassing SegmentPusher... + S3SegmentPusher segmentPusher = (S3SegmentPusher) toolbox.getSegmentPusher(); + String s3Path = String.format( + "s3://%s/%s/%s", + segmentPusher.getConfig().getBucket(), + segmentPusher.getConfig().getBaseKey(), + getDataSource() + ); + + log.info("Setting segment output path to: %s", s3Path); + config.setSegmentOutputDir(s3Path); + } else { + throw new IllegalStateException("Sorry, we only work with S3SegmentPushers! Bummer!"); + } + + HadoopDruidIndexerJob job = new HadoopDruidIndexerJob(config); + log.debug("Starting a hadoop index generator job..."); + + if (job.run()) { + List publishedSegments = job.getPublishedSegments(); + return TaskStatus.success(getId(), ImmutableList.copyOf(publishedSegments)); + + } else { + return TaskStatus.failure(getId()); + } + + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index b2059210b58..4fbdd9de2a0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -37,6 +37,7 @@ import org.joda.time.Interval; @JsonSubTypes(value = { @JsonSubTypes.Type(name = "append", value = AppendTask.class), @JsonSubTypes.Type(name = "delete", value = DeleteTask.class), + @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), @JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class), @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class) @@ -64,12 +65,14 @@ public interface Task /** * Execute preflight checks for a task. This typically runs on the coordinator, and will be run while - * holding a lock on our dataSouce and interval. If this method throws an exception, the task should be + * holding a lock on our dataSource and interval. If this method throws an exception, the task should be * considered a failure. * * @param context Context for this task, gathered under indexer lock + * * @return Some kind of status (runnable means continue on to a worker, non-runnable means we completed without - * using a worker). + * using a worker). + * * @throws Exception */ public TaskStatus preflight(TaskContext context) throws Exception; @@ -81,7 +84,9 @@ public interface Task * * @param context Context for this task, gathered under indexer lock * @param toolbox Toolbox for this task + * * @return Some kind of finished status (isRunnable must be false). + * * @throws Exception */ public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception; diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java index f4099154d2d..1475e4dfe68 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java @@ -62,6 +62,11 @@ public class S3SegmentPusher implements SegmentPusher this.jsonMapper = jsonMapper; } + public S3SegmentPusherConfig getConfig() + { + return config; + } + @Override public DataSegment push(File file, DataSegment segment) throws IOException { From cb845e6f0903f7b8ff2ead9249eb4ec6a2f243d8 Mon Sep 17 00:00:00 2001 From: Deep Ganguli Date: Thu, 24 Jan 2013 17:54:06 -0800 Subject: [PATCH 022/121] Addresed Gian's code review --- .../metamx/druid/indexer/IndexGeneratorJob.java | 5 +---- .../merger/common/task/HadoopIndexTask.java | 17 ++++++++++------- .../metamx/druid/merger/common/task/Task.java | 1 + 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index d9188586041..3b064ccee4e 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -45,7 +45,6 @@ import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.indexer.rollup.DataRollupSpec; import com.metamx.druid.input.MapBasedInputRow; -import com.metamx.druid.jackson.DefaultObjectMapper; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -70,8 +69,6 @@ import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; import org.joda.time.Interval; -import org.skife.jdbi.v2.Handle; -import org.skife.jdbi.v2.tweak.HandleCallback; import javax.annotation.Nullable; import java.io.BufferedOutputStream; @@ -185,7 +182,7 @@ public class IndexGeneratorJob implements Jobby for (FileStatus status : fs.listStatus(descriptorInfoDir)) { final DataSegment segment = jsonMapper.readValue(fs.open(status.getPath()), DataSegment.class); publishedSegmentsBuilder.add(segment); - log.info("Published %s", segment.getIdentifier()); + log.info("Adding segment %s to the list of published segments", segment.getIdentifier()); } } catch (IOException e) { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java index 9dda89d4371..ec80157600c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java @@ -26,9 +26,10 @@ public class HadoopIndexTask extends AbstractTask /** * @param config is used by the HadoopDruidIndexerJob to set up the appropriate parameters - * for creating Druid index segments. It may be modified. Here, we will ensure that the - * UpDaterJobSpec field of the config is set to null, such that the job does not push a - * list of published segments the database. Instead, we will use the method + * for creating Druid index segments. It may be modified. + *

+ * Here, we will ensure that the UpDaterJobSpec field of the config is set to null, such that the + * job does not push a list of published segments the database. Instead, we will use the method * IndexGeneratorJob.getPublishedSegments() to simply return a list of the published * segments, and let the indexing service report these segments to the database. */ @@ -37,13 +38,15 @@ public class HadoopIndexTask extends AbstractTask public HadoopIndexTask(@JsonProperty("config") HadoopDruidIndexerConfig config) { super( - String.format("index_hadoop_%s_interval_%s", config.getDataSource(), new DateTime().now()), + String.format("index_hadoop_%s_interval_%s", config.getDataSource(), new DateTime()), config.getDataSource(), JodaUtils.umbrellaInterval(config.getIntervals()) ); if (config.isUpdaterJobSpecSet()) { - throw new IllegalArgumentException("UpDaterJobSpec is defined"); + throw new IllegalArgumentException( + "The UpDaterJobSpec field of the Hadoop Druid indexer config must be set to null " + ); } this.config = config; } @@ -51,7 +54,7 @@ public class HadoopIndexTask extends AbstractTask @Override public Type getType() { - return Type.INDEX; + return Type.HADOOPINDEX; } @Override @@ -60,7 +63,7 @@ public class HadoopIndexTask extends AbstractTask log.info("Setting version to: %s", context.getVersion()); config.setVersion(context.getVersion()); - if(toolbox.getSegmentPusher() instanceof S3SegmentPusher) { + if (toolbox.getSegmentPusher() instanceof S3SegmentPusher) { // Hack alert! Bypassing SegmentPusher... S3SegmentPusher segmentPusher = (S3SegmentPusher) toolbox.getSegmentPusher(); String s3Path = String.format( diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 4fbdd9de2a0..99b30f66dac 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -47,6 +47,7 @@ public interface Task enum Type { INDEX, + HADOOPINDEX, MERGE, APPEND, DELETE, From 6438401a32093fc4ca365a1f45e8cab24ccb3dfc Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 28 Jan 2013 15:06:09 -0800 Subject: [PATCH 023/121] split pick segment logic into its own method --- .../druid/master/BalancerCostAnalyzer.java | 6 ++---- .../druid/master/BalancerSegmentHolder.java | 8 -------- .../druid/master/DruidMasterBalancer.java | 19 ++++++++++--------- 3 files changed, 12 insertions(+), 21 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 56563cb1521..50c53e6fde2 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -141,7 +141,7 @@ public class BalancerCostAnalyzer } // The balancing application requires us to pick a proposal segment. - public BalancerSegmentHolder findNewSegmentHome(List serverHolders, int numSegments) + public BalancerSegmentHolder pickSegmentToMove(List serverHolders, int numSegments) { // We want to sample from each server w.p. numSegmentsOnServer / totalSegments ServerHolder fromServerHolder = sampleServer(serverHolders, numSegments); @@ -151,9 +151,7 @@ public class BalancerCostAnalyzer List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); - ServerHolder toServer = findNewSegmentHome(proposalSegment, serverHolders); - - return new BalancerSegmentHolder(fromServerHolder.getServer(), toServer.getServer(), proposalSegment); + return new BalancerSegmentHolder(fromServerHolder.getServer(), proposalSegment); } // The assignment application requires us to supply a proposal segment. diff --git a/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java b/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java index 5e7a9369fe4..7d62968e4b9 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java @@ -27,19 +27,16 @@ import com.metamx.druid.client.DruidServer; public class BalancerSegmentHolder { private final DruidServer fromServer; - private final DruidServer toServer; private final DataSegment segment; private volatile int lifetime = 15; public BalancerSegmentHolder( DruidServer fromServer, - DruidServer toServer, DataSegment segment ) { this.fromServer = fromServer; - this.toServer = toServer; this.segment = segment; } @@ -48,11 +45,6 @@ public class BalancerSegmentHolder return fromServer; } - public DruidServer getToServer() - { - return toServer; - } - public DataSegment getSegment() { return segment; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 19cfd89274f..450da27165e 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -120,10 +120,11 @@ public class DruidMasterBalancer implements DruidMasterHelper int iter = 0; while (iter < maxSegmentsToMove) { - BalancerSegmentHolder holder = analyzer.findNewSegmentHome(serverHolderList, numSegments); - if (!segmentsBeingMoved.contains(holder.getSegment())) { - moveSegment(holder, params); - segmentsBeingMoved.add(holder.getSegment()); + BalancerSegmentHolder segmentToMove = analyzer.pickSegmentToMove(serverHolderList, numSegments); + DruidServer toServer = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList).getServer(); + if (!segmentsBeingMoved.contains(segmentToMove.getSegment())) { + moveSegment(segmentToMove, toServer, params); + segmentsBeingMoved.add(segmentToMove.getSegment()); } iter++; } @@ -160,14 +161,14 @@ public class DruidMasterBalancer implements DruidMasterHelper private void moveSegment( final BalancerSegmentHolder segment, + final DruidServer toServer, final DruidMasterRuntimeParams params ) { - final DruidServer toServer = segment.getToServer(); - final String toServerName = segment.getToServer().getName(); + final String toServerName = toServer.getName(); final LoadQueuePeon toPeon = params.getLoadManagementPeons().get(toServerName); - final String fromServer = segment.getFromServer().getName(); + final String fromServerName = segment.getFromServer().getName(); final DataSegment segmentToMove = segment.getSegment(); final String segmentName = segmentToMove.getIdentifier(); @@ -177,12 +178,12 @@ public class DruidMasterBalancer implements DruidMasterHelper log.info( "Moving [%s] from [%s] to [%s]", segmentName, - fromServer, + fromServerName, toServerName ); try { master.moveSegment( - fromServer, + fromServerName, toServerName, segmentToMove.getIdentifier(), new LoadPeonCallback() From 034d6525694f00a7243b504411630bccee5227ec Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Tue, 29 Jan 2013 17:13:01 -0800 Subject: [PATCH 024/121] make balancer tests take into account loadqueue status --- .../druid/master/DruidMasterBalancer.java | 10 +- .../druid/master/DruidMasterBalancerTest.java | 98 +++++++++---------- .../master/DruidMasterBalancerTester.java | 57 +++++++++++ .../druid/master/LoadQueuePeonTester.java | 37 +++++++ 4 files changed, 147 insertions(+), 55 deletions(-) create mode 100644 server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTester.java create mode 100644 server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 450da27165e..f2a4ffeff44 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -50,11 +50,11 @@ public class DruidMasterBalancer implements DruidMasterHelper } } ); - private static final EmittingLogger log = new EmittingLogger(DruidMasterBalancer.class); + protected static final EmittingLogger log = new EmittingLogger(DruidMasterBalancer.class); - private final DruidMaster master; + protected final DruidMaster master; - private final Map> currentlyMovingSegments = Maps.newHashMap(); + protected final Map> currentlyMovingSegments = Maps.newHashMap(); public DruidMasterBalancer( DruidMaster master @@ -63,7 +63,7 @@ public class DruidMasterBalancer implements DruidMasterHelper this.master = master; } - private void reduceLifetimes(String tier) + protected void reduceLifetimes(String tier) { for (BalancerSegmentHolder holder : currentlyMovingSegments.get(tier).values()) { holder.reduceLifetime(); @@ -159,7 +159,7 @@ public class DruidMasterBalancer implements DruidMasterHelper .build(); } - private void moveSegment( + protected void moveSegment( final BalancerSegmentHolder segment, final DruidServer toServer, final DruidMasterRuntimeParams params diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 3470f53633e..6aa2814a97f 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -28,6 +28,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; import com.metamx.druid.shard.NoneShardSpec; +import com.metamx.phonebook.PhoneBook; import junit.framework.Assert; import org.easymock.EasyMock; import org.joda.time.DateTime; @@ -39,6 +40,7 @@ import org.junit.Test; import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; /** */ @@ -55,7 +57,6 @@ public class DruidMasterBalancerTest private DataSegment segment3; private DataSegment segment4; Map segments; - private LoadQueuePeon peon; private DruidDataSource dataSource; @Before @@ -70,7 +71,6 @@ public class DruidMasterBalancerTest segment2 = EasyMock.createMock(DataSegment.class); segment3 = EasyMock.createMock(DataSegment.class); segment4 = EasyMock.createMock(DataSegment.class); - peon = EasyMock.createMock(LoadQueuePeon.class); dataSource = EasyMock.createMock(DruidDataSource.class); DateTime start1 = new DateTime("2012-01-01"); @@ -118,10 +118,10 @@ public class DruidMasterBalancerTest ); segments = new HashMap(); - segments.put("segment1", segment1); - segments.put("segment2", segment2); - segments.put("segment3", segment3); - segments.put("segment4", segment4); + segments.put("datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment1); + segments.put("datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment2); + segments.put("datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment3); + segments.put("datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment4); } @After @@ -132,7 +132,6 @@ public class DruidMasterBalancerTest EasyMock.verify(druidServer2); EasyMock.verify(druidServer3); EasyMock.verify(druidServer4); - EasyMock.verify(peon); EasyMock.verify(dataSource); } @@ -146,6 +145,26 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer1.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); EasyMock.expect(druidServer1.getSegments()).andReturn(segments).anyTimes(); + EasyMock.expect( + druidServer1.getSegment( + "datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer1.getSegment( + "datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer1.getSegment( + "datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); + EasyMock.expect( + druidServer1.getSegment( + "datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" + ) + ).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); EasyMock.expect(druidServer2.getName()).andReturn("to").atLeastOnce(); @@ -154,29 +173,8 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer2.getSegments()).andReturn(new HashMap()).anyTimes(); EasyMock.expect(druidServer2.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); - EasyMock.expect( - druidServer2.getSegment( - "datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer2.getSegment( - "datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer2.getSegment( - "datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer2.getSegment( - "datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect(druidServer2.getSegment("segment3")).andReturn(null).anyTimes(); - EasyMock.expect(druidServer2.getSegment("segment4")).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); + EasyMock.replay(druidServer3); EasyMock.replay(druidServer4); @@ -198,12 +196,11 @@ public class DruidMasterBalancerTest EasyMock.anyObject(), EasyMock.anyObject() ); - EasyMock.expectLastCall().atLeastOnce(); + EasyMock.expectLastCall().anyTimes(); EasyMock.replay(master); - EasyMock.expect(peon.getLoadQueueSize()).andReturn(0L).atLeastOnce(); - EasyMock.expect(peon.getSegmentsToLoad()).andReturn(Sets.newHashSet()).atLeastOnce(); - EasyMock.replay(peon); + LoadQueuePeonTester fromPeon = new LoadQueuePeonTester(EasyMock.createMock(PhoneBook.class), "from", EasyMock.createMock(ScheduledExecutorService.class)); + LoadQueuePeonTester toPeon = new LoadQueuePeonTester(EasyMock.createMock(PhoneBook.class), "to", EasyMock.createMock(ScheduledExecutorService.class)); DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder() @@ -211,27 +208,27 @@ public class DruidMasterBalancerTest new DruidCluster( ImmutableMap.>of( "normal", - MinMaxPriorityQueue.orderedBy(DruidMasterBalancer.percentUsedComparator) + MinMaxPriorityQueue.orderedBy(DruidMasterBalancerTester.percentUsedComparator) .create( Arrays.asList( - new ServerHolder(druidServer1, peon), - new ServerHolder(druidServer2, peon) + new ServerHolder(druidServer1, fromPeon), + new ServerHolder(druidServer2, toPeon) ) ) ) ) ) - .withLoadManagementPeons(ImmutableMap.of("from", peon, "to", peon)) + .withLoadManagementPeons(ImmutableMap.of("from", fromPeon, "to", toPeon)) .withAvailableSegments(segments.values()) .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - params = new DruidMasterBalancer(master).run(params); + params = new DruidMasterBalancerTester(master).run(params); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); } - @Test + @Test public void testRun2() { // Mock some servers of different usages @@ -345,12 +342,13 @@ public class DruidMasterBalancerTest EasyMock.anyObject(), EasyMock.anyObject() ); - EasyMock.expectLastCall().atLeastOnce(); + EasyMock.expectLastCall().anyTimes(); EasyMock.replay(master); - EasyMock.expect(peon.getLoadQueueSize()).andReturn(0L).atLeastOnce(); - EasyMock.expect(peon.getSegmentsToLoad()).andReturn(Sets.newHashSet()).atLeastOnce(); - EasyMock.replay(peon); + LoadQueuePeonTester peon1 = new LoadQueuePeonTester(EasyMock.createMock(PhoneBook.class), "1", EasyMock.createMock(ScheduledExecutorService.class)); + LoadQueuePeonTester peon2 = new LoadQueuePeonTester(EasyMock.createMock(PhoneBook.class), "2", EasyMock.createMock(ScheduledExecutorService.class)); + LoadQueuePeonTester peon3 = new LoadQueuePeonTester(EasyMock.createMock(PhoneBook.class), "3", EasyMock.createMock(ScheduledExecutorService.class)); + LoadQueuePeonTester peon4 = new LoadQueuePeonTester(EasyMock.createMock(PhoneBook.class), "4", EasyMock.createMock(ScheduledExecutorService.class)); DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder() @@ -358,25 +356,25 @@ public class DruidMasterBalancerTest new DruidCluster( ImmutableMap.>of( "normal", - MinMaxPriorityQueue.orderedBy(DruidMasterBalancer.percentUsedComparator) + MinMaxPriorityQueue.orderedBy(DruidMasterBalancerTester.percentUsedComparator) .create( Arrays.asList( - new ServerHolder(druidServer1, peon), - new ServerHolder(druidServer2, peon), - new ServerHolder(druidServer3, peon), - new ServerHolder(druidServer4, peon) + new ServerHolder(druidServer1, peon1), + new ServerHolder(druidServer2, peon2), + new ServerHolder(druidServer3, peon3), + new ServerHolder(druidServer4, peon4) ) ) ) ) ) - .withLoadManagementPeons(ImmutableMap.of("1", peon, "2", peon, "3", peon, "4", peon)) + .withLoadManagementPeons(ImmutableMap.of("1", peon1, "2", peon2, "3", peon3, "4", peon4)) .withAvailableSegments(segments.values()) .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - params = new DruidMasterBalancer(master).run(params); + params = new DruidMasterBalancerTester(master).run(params); Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); } } diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTester.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTester.java new file mode 100644 index 00000000000..3b5865fb361 --- /dev/null +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTester.java @@ -0,0 +1,57 @@ +package com.metamx.druid.master; + +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.DruidServer; + +public class DruidMasterBalancerTester extends DruidMasterBalancer +{ + public DruidMasterBalancerTester(DruidMaster master) + { + super(master); + } + + @Override + protected void moveSegment( + final BalancerSegmentHolder segment, + final DruidServer toServer, + final DruidMasterRuntimeParams params + ) + { + final String toServerName = toServer.getName(); + final LoadQueuePeon toPeon = params.getLoadManagementPeons().get(toServerName); + + final String fromServerName = segment.getFromServer().getName(); + final DataSegment segmentToMove = segment.getSegment(); + final String segmentName = segmentToMove.getIdentifier(); + + if (!toPeon.getSegmentsToLoad().contains(segmentToMove) && + !currentlyMovingSegments.get("normal").containsKey(segmentName) && + !toServer.getSegments().containsKey(segmentName) && + new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) { + log.info( + "Moving [%s] from [%s] to [%s]", + segmentName, + fromServerName, + toServerName + ); + try { + final LoadQueuePeon loadPeon = params.getLoadManagementPeons().get(toServerName); + + loadPeon.loadSegment(segment.getSegment(), new LoadPeonCallback() + { + @Override + protected void execute() + { + } + }); + + currentlyMovingSegments.get("normal").put(segmentName, segment); + } + catch (Exception e) { + log.info(e, String.format("[%s] : Moving exception", segmentName)); + } + } else { + currentlyMovingSegments.get("normal").remove(segment); + } + } +} diff --git a/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java b/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java new file mode 100644 index 00000000000..9363bd2e95c --- /dev/null +++ b/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java @@ -0,0 +1,37 @@ +package com.metamx.druid.master; + +import com.metamx.druid.client.DataSegment; +import com.metamx.phonebook.PhoneBook; + +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.ScheduledExecutorService; + +public class LoadQueuePeonTester extends LoadQueuePeon +{ + private ConcurrentSkipListSet segmentsToLoad; + + public LoadQueuePeonTester( + PhoneBook yp, + String basePath, + ScheduledExecutorService zkWritingExecutor + ) + { + super(yp, basePath, zkWritingExecutor); + } + + @Override + public void loadSegment( + DataSegment segment, + LoadPeonCallback callback + ) + { + if(segmentsToLoad == null) segmentsToLoad = new ConcurrentSkipListSet(); + segmentsToLoad.add(segment); + } + + public ConcurrentSkipListSet getSegmentsToLoad() + { + if(segmentsToLoad == null) segmentsToLoad = new ConcurrentSkipListSet(); + return segmentsToLoad; + } +} From b205f7b3c46190075162a7a7a4133e65c49b75dc Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Tue, 29 Jan 2013 18:16:34 -0800 Subject: [PATCH 025/121] use existing currentlyMovingSegments instead of creating a reundant segmentsBeingMoved --- .../java/com/metamx/druid/master/DruidMasterBalancer.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index f2a4ffeff44..a48073e121f 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -116,15 +116,13 @@ public class DruidMasterBalancer implements DruidMasterHelper continue; } - final Set segmentsBeingMoved = new HashSet(); int iter = 0; while (iter < maxSegmentsToMove) { BalancerSegmentHolder segmentToMove = analyzer.pickSegmentToMove(serverHolderList, numSegments); DruidServer toServer = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList).getServer(); - if (!segmentsBeingMoved.contains(segmentToMove.getSegment())) { + if (!currentlyMovingSegments.get(tier).containsKey(segmentToMove.getSegment())) { moveSegment(segmentToMove, toServer, params); - segmentsBeingMoved.add(segmentToMove.getSegment()); } iter++; } @@ -135,13 +133,13 @@ public class DruidMasterBalancer implements DruidMasterHelper stats.addToTieredStat("initialCost", tier, (long) initialTotalCost); stats.addToTieredStat("normalization", tier, (long) normalization); - stats.addToTieredStat("movedCount", tier, segmentsBeingMoved.size()); + stats.addToTieredStat("movedCount", tier, currentlyMovingSegments.get(tier).size()); log.info( "Initial Total Cost: [%f], Initial Normalized Cost: [%f], Segments Moved: [%d]", initialTotalCost, normalizedInitialCost, - segmentsBeingMoved.size() + currentlyMovingSegments.get(tier).size() ); if (serverHolderList.size() <= 1) { From a1ce4b294eec47388160d437fcbc71910f6de21a Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Tue, 29 Jan 2013 18:16:49 -0800 Subject: [PATCH 026/121] simplify balancer test --- .../druid/master/DruidMasterBalancerTest.java | 116 +----------------- 1 file changed, 1 insertion(+), 115 deletions(-) diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 6aa2814a97f..589b472d6dd 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -23,9 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; -import com.google.common.collect.Sets; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.phonebook.PhoneBook; @@ -57,7 +55,6 @@ public class DruidMasterBalancerTest private DataSegment segment3; private DataSegment segment4; Map segments; - private DruidDataSource dataSource; @Before public void setUp() throws Exception @@ -71,7 +68,6 @@ public class DruidMasterBalancerTest segment2 = EasyMock.createMock(DataSegment.class); segment3 = EasyMock.createMock(DataSegment.class); segment4 = EasyMock.createMock(DataSegment.class); - dataSource = EasyMock.createMock(DruidDataSource.class); DateTime start1 = new DateTime("2012-01-01"); DateTime start2 = new DateTime("2012-02-01"); @@ -132,7 +128,6 @@ public class DruidMasterBalancerTest EasyMock.verify(druidServer2); EasyMock.verify(druidServer3); EasyMock.verify(druidServer4); - EasyMock.verify(dataSource); } @Test @@ -143,28 +138,7 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer1.getName()).andReturn("from").atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer1.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); EasyMock.expect(druidServer1.getSegments()).andReturn(segments).anyTimes(); - EasyMock.expect( - druidServer1.getSegment( - "datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer1.getSegment( - "datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer1.getSegment( - "datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer1.getSegment( - "datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); EasyMock.expect(druidServer2.getName()).andReturn("to").atLeastOnce(); @@ -172,23 +146,11 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer2.getSegments()).andReturn(new HashMap()).anyTimes(); - EasyMock.expect(druidServer2.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); EasyMock.replay(druidServer2); EasyMock.replay(druidServer3); EasyMock.replay(druidServer4); - // Mock a datasource - EasyMock.expect(dataSource.getSegments()).andReturn( - Sets.newHashSet( - segment1, - segment2, - segment3, - segment4 - ) - ).anyTimes(); - EasyMock.replay(dataSource); - // Mock stuff that the master needs master.moveSegment( EasyMock.anyObject(), @@ -235,7 +197,6 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer1.getName()).andReturn("1").atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer1.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); EasyMock.expect(druidServer1.getSegments()).andReturn(segments).anyTimes(); EasyMock.replay(druidServer1); @@ -244,27 +205,6 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer2.getSegments()).andReturn(new HashMap()).anyTimes(); - EasyMock.expect(druidServer2.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); - EasyMock.expect( - druidServer2.getSegment( - "datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer2.getSegment( - "datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer2.getSegment( - "datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer2.getSegment( - "datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); EasyMock.expect(druidServer3.getName()).andReturn("3").atLeastOnce(); @@ -272,27 +212,6 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer3.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer3.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer3.getSegments()).andReturn(new HashMap()).anyTimes(); - EasyMock.expect(druidServer3.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); - EasyMock.expect( - druidServer3.getSegment( - "datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer3.getSegment( - "datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer3.getSegment( - "datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer3.getSegment( - "datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); EasyMock.replay(druidServer3); EasyMock.expect(druidServer4.getName()).andReturn("4").atLeastOnce(); @@ -300,41 +219,8 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer4.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer4.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer4.getSegments()).andReturn(new HashMap()).anyTimes(); - EasyMock.expect(druidServer4.getDataSources()).andReturn(Arrays.asList(dataSource)).anyTimes(); - EasyMock.expect( - druidServer4.getSegment( - "datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer4.getSegment( - "datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer4.getSegment( - "datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); - EasyMock.expect( - druidServer4.getSegment( - "datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z" - ) - ).andReturn(null).anyTimes(); EasyMock.replay(druidServer4); - - // Mock a datasource - EasyMock.expect(dataSource.getSegments()).andReturn( - Sets.newHashSet( - segment1, - segment2, - segment3, - segment4 - ) - ).anyTimes(); - EasyMock.replay(dataSource); - // Mock stuff that the master needs master.moveSegment( EasyMock.anyObject(), @@ -375,6 +261,6 @@ public class DruidMasterBalancerTest .build(); params = new DruidMasterBalancerTester(master).run(params); - Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); + Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() == 4); } } From 73f774e768423eee15bc89205e13bc52363b1ff0 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Tue, 29 Jan 2013 18:17:38 -0800 Subject: [PATCH 027/121] make masterbalancertest more stringent --- .../java/com/metamx/druid/master/DruidMasterBalancerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 589b472d6dd..8e5a3474e7d 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -187,7 +187,7 @@ public class DruidMasterBalancerTest .build(); params = new DruidMasterBalancerTester(master).run(params); - Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); + Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() == 3); } @Test From dc49cccf56a89941b2e47a1cd564accb82efea29 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Tue, 29 Jan 2013 18:32:52 -0800 Subject: [PATCH 028/121] javadoc format --- .../druid/master/BalancerCostAnalyzer.java | 34 +++++++++++++++---- 1 file changed, 28 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 50c53e6fde2..5954b51c845 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -49,9 +49,11 @@ public class BalancerCostAnalyzer rand = new Random(0); } - /* + /** * Calculates the cost normalization. This is such that the normalized cost is lower bounded * by 1 (e.g. when each segment gets its own compute node). + * @param serverHolderList + * @return */ public double calculateNormalization(List serverHolderList) { @@ -64,7 +66,11 @@ public class BalancerCostAnalyzer return cost; } - // Calculates the initial cost of the Druid segment configuration. + /** + * Calculates the initial cost of the Druid segment configuration. + * @param serverHolderList + * @return + */ public double calculateInitialTotalCost(List serverHolderList) { double cost = 0; @@ -79,13 +85,16 @@ public class BalancerCostAnalyzer return cost; } - /* + /** * This defines the unnormalized cost function between two segments. There is a base cost given by * the minimum size of the two segments and additional penalties. * recencyPenalty: it is more likely that recent segments will be queried together * dataSourcePenalty: if two segments belong to the same data source, they are more likely to be involved * in the same queries * gapPenalty: it is more likely that segments close together in time will be queried together + * @param segment1 + * @param segment2 + * @return */ public double computeJointSegmentCosts(DataSegment segment1, DataSegment segment2) { @@ -123,8 +132,11 @@ public class BalancerCostAnalyzer return cost; } - /* + /** * Sample from each server with probability proportional to the number of segments on that server. + * @param serverHolderList + * @param numSegments + * @return */ private ServerHolder sampleServer(List serverHolderList, int numSegments) { @@ -140,7 +152,12 @@ public class BalancerCostAnalyzer return serverHolderList.get(numToStopAt - 1); } - // The balancing application requires us to pick a proposal segment. + /** + * The balancing application requires us to pick a proposal segment. + * @param serverHolders + * @param numSegments + * @return + */ public BalancerSegmentHolder pickSegmentToMove(List serverHolders, int numSegments) { // We want to sample from each server w.p. numSegmentsOnServer / totalSegments @@ -154,7 +171,12 @@ public class BalancerCostAnalyzer return new BalancerSegmentHolder(fromServerHolder.getServer(), proposalSegment); } - // The assignment application requires us to supply a proposal segment. + /** + * The assignment application requires us to supply a proposal segment. + * @param proposalSegment + * @param serverHolders + * @return + */ public ServerHolder findNewSegmentHome(DataSegment proposalSegment, Iterable serverHolders) { final long proposalSegmentSize = proposalSegment.getSize(); From 0b61f9696514931a035217b9314aacda4bf50cc4 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Fri, 1 Feb 2013 11:25:05 -0800 Subject: [PATCH 029/121] extra documentation --- .../druid/master/BalancerCostAnalyzer.java | 73 +++++++++++-------- 1 file changed, 43 insertions(+), 30 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 5954b51c845..6b68f426d8a 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -52,13 +52,16 @@ public class BalancerCostAnalyzer /** * Calculates the cost normalization. This is such that the normalized cost is lower bounded * by 1 (e.g. when each segment gets its own compute node). - * @param serverHolderList - * @return + * @param serverHolders + * A list of ServerHolders for a particular tier. + * @return The normalization value (the sum of the diagonal entries in the + * pairwise cost matrix). This is the cost of a cluster if each + * segment were to get its own compute node. */ - public double calculateNormalization(List serverHolderList) + public double calculateNormalization(List serverHolders) { double cost = 0; - for (ServerHolder server : serverHolderList) { + for (ServerHolder server : serverHolders) { for (DataSegment segment : server.getServer().getSegments().values()) { cost += computeJointSegmentCosts(segment, segment); } @@ -68,13 +71,14 @@ public class BalancerCostAnalyzer /** * Calculates the initial cost of the Druid segment configuration. - * @param serverHolderList - * @return + * @param serverHolders + * A list of ServerHolders for a particular tier. + * @return The initial cost of the Druid tier. */ - public double calculateInitialTotalCost(List serverHolderList) + public double calculateInitialTotalCost(List serverHolders) { double cost = 0; - for (ServerHolder server : serverHolderList) { + for (ServerHolder server : serverHolders) { DataSegment[] segments = server.getServer().getSegments().values().toArray(new DataSegment[]{}); for (int i = 0; i < segments.length; ++i) { for (int j = i; j < segments.length; ++j) { @@ -92,9 +96,11 @@ public class BalancerCostAnalyzer * dataSourcePenalty: if two segments belong to the same data source, they are more likely to be involved * in the same queries * gapPenalty: it is more likely that segments close together in time will be queried together - * @param segment1 - * @param segment2 - * @return + * @param segment1 + * The first DataSegment. + * @param segment2 + * The second DataSegment. + * @return The joint cost of placing the two DataSegments together on one node. */ public double computeJointSegmentCosts(DataSegment segment1, DataSegment segment2) { @@ -134,37 +140,42 @@ public class BalancerCostAnalyzer /** * Sample from each server with probability proportional to the number of segments on that server. - * @param serverHolderList - * @param numSegments - * @return + * @param serverHolders + * A list of ServerHolders for a particular tier. + * @param numSegments + + * @return A ServerHolder sampled with probability proportional to the + * number of segments on that server */ - private ServerHolder sampleServer(List serverHolderList, int numSegments) + private ServerHolder sampleServer(List serverHolders, int numSegments) { final int num = rand.nextInt(numSegments); int cumulativeSegments = 0; int numToStopAt = 0; while (cumulativeSegments <= num) { - cumulativeSegments += serverHolderList.get(numToStopAt).getServer().getSegments().size(); + cumulativeSegments += serverHolders.get(numToStopAt).getServer().getSegments().size(); numToStopAt++; } - return serverHolderList.get(numToStopAt - 1); + return serverHolders.get(numToStopAt - 1); } /** * The balancing application requires us to pick a proposal segment. - * @param serverHolders - * @param numSegments - * @return + * @param serverHolders + * A list of ServerHolders for a particular tier. + * @param numSegments + * The total number of segments on a particular tier. + * @return A BalancerSegmentHolder sampled uniformly at random. */ public BalancerSegmentHolder pickSegmentToMove(List serverHolders, int numSegments) { - // We want to sample from each server w.p. numSegmentsOnServer / totalSegments + /** We want to sample from each server w.p. numSegmentsOnServer / totalSegments */ ServerHolder fromServerHolder = sampleServer(serverHolders, numSegments); - // and actually pick that segment uniformly at random w.p. 1 / numSegmentsOnServer - // so that the probability of picking a segment is 1 / totalSegments. + /** and actually pick that segment uniformly at random w.p. 1 / numSegmentsOnServer + so that the probability of picking a segment is 1 / totalSegments. */ List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); @@ -173,9 +184,11 @@ public class BalancerCostAnalyzer /** * The assignment application requires us to supply a proposal segment. - * @param proposalSegment - * @param serverHolders - * @return + * @param proposalSegment + * A DataSegment that we are proposing to move. + * @param serverHolders + * An iterable of ServerHolders for a particular tier. + * @return A ServerHolder with the new home for a segment. */ public ServerHolder findNewSegmentHome(DataSegment proposalSegment, Iterable serverHolders) { @@ -184,20 +197,20 @@ public class BalancerCostAnalyzer ServerHolder toServer = null; for (ServerHolder server : serverHolders) { - // Only calculate costs if the server has enough space. + /** Only calculate costs if the server has enough space. */ if (proposalSegmentSize > server.getAvailableSize()) { break; } - // The contribution to the total cost of a given server by proposing to move the segment to that server is... + /** The contribution to the total cost of a given server by proposing to move the segment to that server is... */ double cost = 0f; - // the sum of the costs of other (exclusive of the proposalSegment) segments on the server + /** the sum of the costs of other (exclusive of the proposalSegment) segments on the server */ for (DataSegment segment : server.getServer().getSegments().values()) { if (!proposalSegment.equals(segment)) { cost += computeJointSegmentCosts(proposalSegment, segment); } } - // plus the costs of segments that will be loaded + /** plus the costs of segments that will be loaded */ for (DataSegment segment : server.getPeon().getSegmentsToLoad()) { cost += computeJointSegmentCosts(proposalSegment, segment); } From 1c9270add31549db4c7378919618e57f0000ee1e Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Fri, 1 Feb 2013 11:25:41 -0800 Subject: [PATCH 030/121] change format of line comment --- .../main/java/com/metamx/druid/master/BalancerCostAnalyzer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 6b68f426d8a..d57a3e2d841 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -123,7 +123,7 @@ public class BalancerCostAnalyzer recencyPenalty = 2 - maxDiff / SEVEN_DAYS_IN_MILLIS; } - // gap is null if the two segment intervals overlap or if they're adjacent + /** gap is null if the two segment intervals overlap or if they're adjacent */ if (gap == null) { gapPenalty = 2; } else { From 1b9764fffb5033106e2d0d241dfc5a4ba504be24 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 11 Feb 2013 11:36:45 -0800 Subject: [PATCH 031/121] add binary version to datasegments for balancer test --- .../java/com/metamx/druid/master/DruidMasterBalancerTest.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 8e5a3474e7d..504c4f4cfef 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -80,6 +80,7 @@ public class DruidMasterBalancerTest Lists.newArrayList(), Lists.newArrayList(), new NoneShardSpec(), + 0, 11L ); segment2 = new DataSegment( @@ -90,6 +91,7 @@ public class DruidMasterBalancerTest Lists.newArrayList(), Lists.newArrayList(), new NoneShardSpec(), + 0, 7L ); segment3 = new DataSegment( @@ -100,6 +102,7 @@ public class DruidMasterBalancerTest Lists.newArrayList(), Lists.newArrayList(), new NoneShardSpec(), + 0, 4L ); segment4 = new DataSegment( @@ -110,6 +113,7 @@ public class DruidMasterBalancerTest Lists.newArrayList(), Lists.newArrayList(), new NoneShardSpec(), + 0, 8L ); From e10025c84172ba448b6ac834e5fe784eb6fe5c84 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 8 Feb 2013 16:50:23 -0800 Subject: [PATCH 032/121] filter out unneeded segments for TimeBoundary query --- .../druid/client/CachingClusteredClient.java | 31 ++++++++++++ .../metamx/druid/query/QueryToolChest.java | 26 ++++++---- .../group/GroupByQueryQueryToolChest.java | 2 +- .../SegmentMetadataQueryQueryToolChest.java | 2 +- .../search/SearchQueryQueryToolChest.java | 2 +- .../TimeBoundaryQueryQueryToolChest.java | 49 ++++++++++++++++++- .../TimeseriesQueryQueryToolChest.java | 2 +- .../com/metamx/druid/utils/JodaUtils.java | 14 ++++++ .../druid/coordination/ServerManagerTest.java | 2 +- 9 files changed, 115 insertions(+), 15 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 163f1986a53..a07478ab751 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -20,6 +20,7 @@ package com.metamx.druid.client; import com.google.common.base.Function; +import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -58,6 +59,7 @@ import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -152,6 +154,35 @@ public class CachingClusteredClient implements QueryRunner } } + // Let tool chest filter out unneeded segments + final Set filteredSegmentDescriptors = Sets.newLinkedHashSet(toolChest.filterSegments( + query, + Iterables.transform( + segments, new Function, SegmentDescriptor>() + { + @Override + public SegmentDescriptor apply( + @Nullable Pair input + ) + { + return input.rhs; + } + } + ) + )); + + // remove unneeded segments from list of segments to query + segments = Sets.newLinkedHashSet(Iterables.filter(segments, new Predicate>() + { + @Override + public boolean apply( + @Nullable Pair input + ) + { + return filteredSegmentDescriptors.contains(input.rhs); + } + })); + final byte[] queryCacheKey; if(strategy != null) { queryCacheKey = strategy.computeCacheKey(query); diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index bec2170ec92..36847f7f77f 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -20,19 +20,23 @@ package com.metamx.druid.query; import com.google.common.base.Function; +import com.metamx.common.Pair; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; +import com.metamx.druid.query.segment.QuerySegmentSpec; +import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; +import org.joda.time.Interval; /** * The broker-side (also used by server in some cases) API for a specific Query type. This API is still undergoing * evolution and is only semi-stable, so proprietary Query implementations should be ready for the potential * maintenance burden when upgrading versions. */ -public interface QueryToolChest> +public abstract class QueryToolChest> { - public QueryRunner mergeResults(QueryRunner runner); + public abstract QueryRunner mergeResults(QueryRunner runner); /** * This method doesn't belong here, but it's here for now just to make it work. @@ -40,11 +44,15 @@ public interface QueryToolChest> * @param seqOfSequences * @return */ - public Sequence mergeSequences(Sequence> seqOfSequences); - public ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); - public Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); - public TypeReference getResultTypeReference(); - public CacheStrategy getCacheStrategy(QueryType query); - public QueryRunner preMergeQueryDecoration(QueryRunner runner); - public QueryRunner postMergeQueryDecoration(QueryRunner runner); + public abstract Sequence mergeSequences(Sequence> seqOfSequences); + public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); + public abstract Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); + public abstract TypeReference getResultTypeReference(); + public abstract CacheStrategy getCacheStrategy(QueryType query); + public abstract QueryRunner preMergeQueryDecoration(QueryRunner runner); + public abstract QueryRunner postMergeQueryDecoration(QueryRunner runner); + + public Iterable filterSegments(QueryType query, Iterable intervals) { + return intervals; + } } diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index 9dcf6110322..b93f46a4562 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -53,7 +53,7 @@ import java.util.Properties; /** */ -public class GroupByQueryQueryToolChest implements QueryToolChest +public class GroupByQueryQueryToolChest extends QueryToolChest { private static final TypeReference TYPE_REFERENCE = new TypeReference(){}; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 160c23cd958..226fc58969f 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -50,7 +50,7 @@ import java.util.Map; import java.util.Set; -public class SegmentMetadataQueryQueryToolChest implements QueryToolChest +public class SegmentMetadataQueryQueryToolChest extends QueryToolChest { private static final TypeReference TYPE_REFERENCE = new TypeReference(){}; private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4}; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index ce3fcc86114..a321c7486c5 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -64,7 +64,7 @@ import java.util.Set; /** */ -public class SearchQueryQueryToolChest implements QueryToolChest, SearchQuery> +public class SearchQueryQueryToolChest extends QueryToolChest, SearchQuery> { private static final byte SEARCH_QUERY = 0x2; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 5ee6c321bbb..b566cd84de3 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -21,8 +21,11 @@ package com.metamx.druid.query.timeboundary; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; @@ -33,20 +36,26 @@ import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.MetricManipulationFn; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChest; +import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; +import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; +import org.joda.time.Interval; import javax.annotation.Nullable; import java.nio.ByteBuffer; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; /** */ public class TimeBoundaryQueryQueryToolChest - implements QueryToolChest, TimeBoundaryQuery> + extends QueryToolChest, TimeBoundaryQuery> { private static final byte TIMEBOUNDARY_QUERY = 0x3; @@ -57,6 +66,44 @@ public class TimeBoundaryQueryQueryToolChest { }; + + @Override + public Iterable filterSegments(TimeBoundaryQuery query, Iterable input) { + long minMillis = Long.MAX_VALUE; + long maxMillis = Long.MIN_VALUE; + SegmentDescriptor min = null; + SegmentDescriptor max = null; + + // keep track of all segments in a given shard + Map> segmentGroups = Maps.newHashMap(); + + Iterable condensedIntervals = JodaUtils.condenseIntervals(query.getIntervals()); + for(SegmentDescriptor e : input) { + if(Iterables.isEmpty(condensedIntervals) || JodaUtils.overlaps(e.getInterval(), condensedIntervals)) { + final long start = e.getInterval().getStartMillis(); + final long end = e.getInterval().getEndMillis(); + final String version = e.getVersion(); + + if(segmentGroups.containsKey(version)) { + segmentGroups.get(version).add(e); + } else { + segmentGroups.put(version, Sets.newHashSet(e)); + } + + if(min == null || start < minMillis) { + min = e; + minMillis = start; + } + if(max == null || end > maxMillis) { + max = e; + maxMillis = end; + } + } + } + + return Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion())); + } + @Override public QueryRunner> mergeResults( final QueryRunner> runner diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 9c633507ec5..c5f8c280c9a 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -60,7 +60,7 @@ import java.util.Map; /** */ -public class TimeseriesQueryQueryToolChest implements QueryToolChest, TimeseriesQuery> +public class TimeseriesQueryQueryToolChest extends QueryToolChest, TimeseriesQuery> { private static final byte TIMESERIES_QUERY = 0x0; diff --git a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java index 53b14618001..c7feeadc44c 100644 --- a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java +++ b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java @@ -19,12 +19,15 @@ package com.metamx.druid.utils; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.guava.Comparators; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Iterator; import java.util.TreeSet; @@ -63,6 +66,17 @@ public class JodaUtils return retVal; } + public static boolean overlaps(final Interval i, Iterable intervals) { + return Iterables.any(intervals, new Predicate() + { + @Override + public boolean apply(@Nullable Interval input) + { + return input.overlaps(i); + } + }); + } + public static DateTime minDateTime(DateTime... times) { if (times == null) { diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index 84982d05737..165fb402107 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -391,7 +391,7 @@ public class ServerManagerTest } } - public static class NoopQueryToolChest> implements QueryToolChest + public static class NoopQueryToolChest> extends QueryToolChest { @Override public QueryRunner mergeResults(QueryRunner runner) From 695b78b19c082bd2ddc57274b7fb2968784f241f Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Mon, 11 Feb 2013 15:28:17 -0800 Subject: [PATCH 033/121] make findNewSegmentHome return a priority queue so assignment can assign replicants to other servers --- .../druid/master/BalancerCostAnalyzer.java | 31 +++++++++++++------ .../druid/master/DruidMasterBalancer.java | 2 +- .../metamx/druid/master/rules/LoadRule.java | 31 ++++++++----------- .../master/DruidMasterRuleRunnerTest.java | 4 +++ 4 files changed, 40 insertions(+), 28 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index d57a3e2d841..cfd99c142c8 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -20,11 +20,14 @@ package com.metamx.druid.master; import com.google.common.collect.Lists; +import com.google.common.collect.MinMaxPriorityQueue; +import com.metamx.common.Pair; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; +import java.util.Comparator; import java.util.List; import java.util.Random; @@ -188,13 +191,26 @@ public class BalancerCostAnalyzer * A DataSegment that we are proposing to move. * @param serverHolders * An iterable of ServerHolders for a particular tier. - * @return A ServerHolder with the new home for a segment. + * @return A MinMaxPriorityQueue of costs of putting the proposalSegment on the server and ServerHolders. */ - public ServerHolder findNewSegmentHome(DataSegment proposalSegment, Iterable serverHolders) + public MinMaxPriorityQueue> findNewSegmentHome(DataSegment proposalSegment, Iterable serverHolders) { + // Just need a regular priority queue for the min. element. + final MinMaxPriorityQueue> costServerPairs = MinMaxPriorityQueue.orderedBy( + new Comparator>() + { + @Override + public int compare( + Pair o, + Pair o1 + ) + { + return Double.compare(o.lhs, o1.lhs); + } + } + ).create(); + final long proposalSegmentSize = proposalSegment.getSize(); - double minCost = Double.MAX_VALUE; - ServerHolder toServer = null; for (ServerHolder server : serverHolders) { /** Only calculate costs if the server has enough space. */ @@ -215,13 +231,10 @@ public class BalancerCostAnalyzer cost += computeJointSegmentCosts(proposalSegment, segment); } - if (cost < minCost) { - minCost = cost; - toServer = server; - } + costServerPairs.add(Pair.of(cost, server)); } - return toServer; + return costServerPairs; } } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index fa04d93a34a..d766f2d663f 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -110,7 +110,7 @@ public class DruidMasterBalancer implements DruidMasterHelper while (iter < maxSegmentsToMove) { BalancerSegmentHolder segmentToMove = analyzer.pickSegmentToMove(serverHolderList, numSegments); - DruidServer toServer = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList).getServer(); + DruidServer toServer = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList).pollFirst().rhs.getServer(); if (!currentlyMovingSegments.get(tier).containsKey(segmentToMove.getSegment())) { moveSegment(segmentToMove, toServer, params); } diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index d33da8286e2..7c26c8b2d6d 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -21,6 +21,7 @@ package com.metamx.druid.master.rules; import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; +import com.metamx.common.Pair; import com.metamx.druid.client.DataSegment; import com.metamx.druid.master.BalancerCostAnalyzer; import com.metamx.druid.master.DruidMaster; @@ -58,14 +59,18 @@ public abstract class LoadRule implements Rule return stats; } + final List serverHolderList = new ArrayList(serverQueue); + final DateTime referenceTimestamp = params.getBalancerReferenceTimestamp(); + final BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(referenceTimestamp); + MinMaxPriorityQueue> serverCostQueue = analyzer.findNewSegmentHome(segment, serverHolderList); + stats.accumulate( assign( params.getReplicationManager(), expectedReplicants, totalReplicants, - serverQueue, - segment, - params + serverCostQueue, + segment ) ); @@ -78,40 +83,32 @@ public abstract class LoadRule implements Rule final ReplicationThrottler replicationManager, int expectedReplicants, int totalReplicants, - MinMaxPriorityQueue serverQueue, - final DataSegment segment, - final DruidMasterRuntimeParams params + MinMaxPriorityQueue> serverQueue, + final DataSegment segment ) { MasterStats stats = new MasterStats(); - List serverHolderList = new ArrayList(serverQueue); - - List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { - final DateTime referenceTimestamp = params.getBalancerReferenceTimestamp(); - final BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(referenceTimestamp); - ServerHolder holder = analyzer.findNewSegmentHome(segment, serverHolderList); + ServerHolder holder = serverQueue.pollFirst().rhs; if (holder == null) { log.warn( - "Not enough %s servers[%d] or node capacity to assign segment[%s]! Expected Replicants[%d]", + "Not enough %s servers or node capacity to assign segment[%s]! Expected Replicants[%d]", getTier(), - assignedServers.size() + serverQueue.size() + 1, segment.getIdentifier(), expectedReplicants ); break; } + if (holder.isServingSegment(segment) || holder.isLoadingSegment(segment)) { - assignedServers.add(holder); continue; } if (totalReplicants > 0) { // don't throttle if there's only 1 copy of this segment in the cluster if (!replicationManager.canAddReplicant(getTier()) || !replicationManager.registerReplicantCreation(getTier(), segment.getIdentifier())) { - serverQueue.add(holder); break; } } @@ -127,12 +124,10 @@ public abstract class LoadRule implements Rule } } ); - assignedServers.add(holder); stats.addToTieredStat("assignedCount", getTier(), 1); ++totalReplicants; } - serverQueue.addAll(assignedServers); return stats; } diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java index f40d16d93ff..60429dbcae1 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java @@ -528,6 +528,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); @@ -601,6 +602,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); @@ -847,6 +849,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); @@ -1032,6 +1035,7 @@ public class DruidMasterRuleRunnerTest .withAvailableSegments(longerAvailableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) + .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); DruidMasterRuntimeParams afterParams = ruleRunner.run(params); From e9f546c4cabdf489e17f061d076a341c78312078 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 16:51:40 -0800 Subject: [PATCH 034/121] add default implementations to QueryToolChest methods --- .../com/metamx/druid/query/QueryToolChest.java | 15 ++++++++++++--- .../group/GroupByQueryQueryToolChest.java | 18 ------------------ .../SegmentMetadataQueryQueryToolChest.java | 12 ------------ .../search/SearchQueryQueryToolChest.java | 6 ------ .../TimeBoundaryQueryQueryToolChest.java | 12 ------------ .../TimeseriesQueryQueryToolChest.java | 6 ------ .../druid/coordination/ServerManagerTest.java | 18 ------------------ 7 files changed, 12 insertions(+), 75 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 36847f7f77f..7c462cc08d8 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -48,9 +48,18 @@ public abstract class QueryToolChest makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); public abstract TypeReference getResultTypeReference(); - public abstract CacheStrategy getCacheStrategy(QueryType query); - public abstract QueryRunner preMergeQueryDecoration(QueryRunner runner); - public abstract QueryRunner postMergeQueryDecoration(QueryRunner runner); + + public CacheStrategy getCacheStrategy(QueryType query) { + return null; + } + + public QueryRunner preMergeQueryDecoration(QueryRunner runner) { + return runner; + } + + public QueryRunner postMergeQueryDecoration(QueryRunner runner) { + return runner; + } public Iterable filterSegments(QueryType query, Iterable intervals) { return intervals; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index b93f46a4562..1c66774d3bc 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -176,22 +176,4 @@ public class GroupByQueryQueryToolChest extends QueryToolChest getCacheStrategy(GroupByQuery query) - { - return null; - } - - @Override - public QueryRunner preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } } diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 226fc58969f..2b979117800 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -219,18 +219,6 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - private Ordering getOrdering() { return new Ordering() diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index a321c7486c5..cefc32b5e3d 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -262,12 +262,6 @@ public class SearchQueryQueryToolChest extends QueryToolChest> postMergeQueryDecoration(final QueryRunner> runner) - { - return runner; - } - private static class SearchThresholdAdjustingQueryRunner implements QueryRunner> { private final QueryRunner> runner; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index b566cd84de3..a4647026e79 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -215,18 +215,6 @@ public class TimeBoundaryQueryQueryToolChest }; } - @Override - public QueryRunner> preMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - - @Override - public QueryRunner> postMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - public Ordering> getOrdering() { return Ordering.natural(); diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index c5f8c280c9a..538897480b3 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -258,12 +258,6 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest>(runner, Period.months(1)); } - @Override - public QueryRunner> postMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - public Ordering> getOrdering() { return Ordering.natural(); diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index 165fb402107..de148a9219c 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -422,23 +422,5 @@ public class ServerManagerTest { return new TypeReference(){}; } - - @Override - public CacheStrategy getCacheStrategy(QueryType query) - { - return null; - } - - @Override - public QueryRunner preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } } } From 6b87ef2921cf1dd5a1124d872dc32e5668656fbe Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 16:55:21 -0800 Subject: [PATCH 035/121] assume segment intervals have already been filtered based on query interval --- .../TimeBoundaryQueryQueryToolChest.java | 37 ++++++++----------- 1 file changed, 15 insertions(+), 22 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index a4647026e79..8ad7378faf2 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -21,7 +21,6 @@ package com.metamx.druid.query.timeboundary; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -39,15 +38,12 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; -import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -77,27 +73,24 @@ public class TimeBoundaryQueryQueryToolChest // keep track of all segments in a given shard Map> segmentGroups = Maps.newHashMap(); - Iterable condensedIntervals = JodaUtils.condenseIntervals(query.getIntervals()); for(SegmentDescriptor e : input) { - if(Iterables.isEmpty(condensedIntervals) || JodaUtils.overlaps(e.getInterval(), condensedIntervals)) { - final long start = e.getInterval().getStartMillis(); - final long end = e.getInterval().getEndMillis(); - final String version = e.getVersion(); + final long start = e.getInterval().getStartMillis(); + final long end = e.getInterval().getEndMillis(); + final String version = e.getVersion(); - if(segmentGroups.containsKey(version)) { - segmentGroups.get(version).add(e); - } else { - segmentGroups.put(version, Sets.newHashSet(e)); - } + if(segmentGroups.containsKey(version)) { + segmentGroups.get(version).add(e); + } else { + segmentGroups.put(version, Sets.newHashSet(e)); + } - if(min == null || start < minMillis) { - min = e; - minMillis = start; - } - if(max == null || end > maxMillis) { - max = e; - maxMillis = end; - } + if(min == null || start < minMillis) { + min = e; + minMillis = start; + } + if(max == null || end > maxMillis) { + max = e; + maxMillis = end; } } From d4009c8c1ce9a101295209e742f4a2de7fe2c9ad Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 18:40:14 -0800 Subject: [PATCH 036/121] refactor filterSegments to use `TimelineObjectHolder`s --- .../druid/client/CachingClusteredClient.java | 51 +++++-------------- .../metamx/druid/query/QueryToolChest.java | 11 +++- .../TimeBoundaryQueryQueryToolChest.java | 17 ++++--- 3 files changed, 34 insertions(+), 45 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index a07478ab751..20c9ca2fea8 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -139,49 +139,26 @@ public class CachingClusteredClient implements QueryRunner // build set of segments to query Set> segments = Sets.newLinkedHashSet(); + List> serversLookup = Lists.newLinkedList(); + for (Interval interval : rewrittenQuery.getIntervals()) { - List> serversLookup = timeline.lookup(interval); - - for (TimelineObjectHolder holder : serversLookup) { - for (PartitionChunk chunk : holder.getObject()) { - ServerSelector selector = chunk.getObject(); - final SegmentDescriptor descriptor = new SegmentDescriptor( - holder.getInterval(), holder.getVersion(), chunk.getChunkNumber() - ); - - segments.add(Pair.of(selector, descriptor)); - } - } + serversLookup.addAll(timeline.lookup(interval)); } // Let tool chest filter out unneeded segments - final Set filteredSegmentDescriptors = Sets.newLinkedHashSet(toolChest.filterSegments( - query, - Iterables.transform( - segments, new Function, SegmentDescriptor>() - { - @Override - public SegmentDescriptor apply( - @Nullable Pair input - ) - { - return input.rhs; - } - } - ) - )); + final List> filteredServersLookup = + toolChest.filterSegments(query, serversLookup); - // remove unneeded segments from list of segments to query - segments = Sets.newLinkedHashSet(Iterables.filter(segments, new Predicate>() - { - @Override - public boolean apply( - @Nullable Pair input - ) - { - return filteredSegmentDescriptors.contains(input.rhs); + for (TimelineObjectHolder holder : filteredServersLookup) { + for (PartitionChunk chunk : holder.getObject()) { + ServerSelector selector = chunk.getObject(); + final SegmentDescriptor descriptor = new SegmentDescriptor( + holder.getInterval(), holder.getVersion(), chunk.getChunkNumber() + ); + + segments.add(Pair.of(selector, descriptor)); } - })); + } final byte[] queryCacheKey; if(strategy != null) { diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 7c462cc08d8..6822e654471 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -23,12 +23,16 @@ import com.google.common.base.Function; import com.metamx.common.Pair; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; +import com.metamx.druid.TimelineObjectHolder; +import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; import org.joda.time.Interval; +import java.util.List; + /** * The broker-side (also used by server in some cases) API for a specific Query type. This API is still undergoing * evolution and is only semi-stable, so proprietary Query implementations should be ready for the potential @@ -61,7 +65,10 @@ public abstract class QueryToolChest filterSegments(QueryType query, Iterable intervals) { - return intervals; + public List> filterSegments( + QueryType query, + List> segments + ) { + return segments; } } diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 8ad7378faf2..388a3c6df85 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -29,6 +29,8 @@ import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.Query; +import com.metamx.druid.TimelineObjectHolder; +import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.collect.OrderedMergeSequence; import com.metamx.druid.query.BySegmentSkippingQueryRunner; import com.metamx.druid.query.CacheStrategy; @@ -64,16 +66,19 @@ public class TimeBoundaryQueryQueryToolChest @Override - public Iterable filterSegments(TimeBoundaryQuery query, Iterable input) { + public List> filterSegments( + TimeBoundaryQuery query, + List> input + ) { long minMillis = Long.MAX_VALUE; long maxMillis = Long.MIN_VALUE; - SegmentDescriptor min = null; - SegmentDescriptor max = null; + TimelineObjectHolder min = null; + TimelineObjectHolder max = null; // keep track of all segments in a given shard - Map> segmentGroups = Maps.newHashMap(); + Map>> segmentGroups = Maps.newHashMap(); - for(SegmentDescriptor e : input) { + for(TimelineObjectHolder e : input) { final long start = e.getInterval().getStartMillis(); final long end = e.getInterval().getEndMillis(); final String version = e.getVersion(); @@ -94,7 +99,7 @@ public class TimeBoundaryQueryQueryToolChest } } - return Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion())); + return Lists.newArrayList(Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion()))); } @Override From 633cccce6918cece6ebdb9ed2174c9a419dd9872 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 17:02:33 -0800 Subject: [PATCH 037/121] bump up to 0.3.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ec6db24d01c..aa77ea6894b 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT druid druid From 7ab2e0552aa954b973d470bf39eb7c23274fc7e9 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 18:42:43 -0800 Subject: [PATCH 038/121] cleanup imports --- .../java/com/metamx/druid/client/CachingClusteredClient.java | 2 -- .../src/main/java/com/metamx/druid/query/QueryToolChest.java | 4 ---- .../query/timeboundary/TimeBoundaryQueryQueryToolChest.java | 1 - 3 files changed, 7 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 20c9ca2fea8..ad08ede04eb 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -20,7 +20,6 @@ package com.metamx.druid.client; import com.google.common.base.Function; -import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -59,7 +58,6 @@ import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 6822e654471..701a4512403 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -20,16 +20,12 @@ package com.metamx.druid.query; import com.google.common.base.Function; -import com.metamx.common.Pair; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.client.selector.ServerSelector; -import com.metamx.druid.query.segment.QuerySegmentSpec; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; -import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 388a3c6df85..b27a6fd5b80 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -37,7 +37,6 @@ import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.MetricManipulationFn; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChest; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; From 68cd2e10a7c42150cb9f5acdef9ca66d7c98b959 Mon Sep 17 00:00:00 2001 From: xvrl Date: Tue, 12 Feb 2013 09:45:26 -0800 Subject: [PATCH 039/121] remove unnecessary segment grouping --- .../TimeBoundaryQueryQueryToolChest.java | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index b27a6fd5b80..8e41f1f6eb8 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -74,19 +74,9 @@ public class TimeBoundaryQueryQueryToolChest TimelineObjectHolder min = null; TimelineObjectHolder max = null; - // keep track of all segments in a given shard - Map>> segmentGroups = Maps.newHashMap(); - for(TimelineObjectHolder e : input) { final long start = e.getInterval().getStartMillis(); final long end = e.getInterval().getEndMillis(); - final String version = e.getVersion(); - - if(segmentGroups.containsKey(version)) { - segmentGroups.get(version).add(e); - } else { - segmentGroups.put(version, Sets.newHashSet(e)); - } if(min == null || start < minMillis) { min = e; @@ -98,7 +88,7 @@ public class TimeBoundaryQueryQueryToolChest } } - return Lists.newArrayList(Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion()))); + return min == max ? Lists.newArrayList(min) : Lists.newArrayList(min , max); } @Override From 07b0b4611bae8063698e220372678e0907b155e4 Mon Sep 17 00:00:00 2001 From: xvrl Date: Tue, 12 Feb 2013 09:52:21 -0800 Subject: [PATCH 040/121] bump up to 0.3.0 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 6 +++--- examples/pom.xml | 2 +- index-common/pom.xml | 4 ++-- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 9 files changed, 12 insertions(+), 12 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 82c917ebecb..36bb84de0d4 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 8d058d643cf..350828b9316 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 9b2567dc753..88994a529d5 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT @@ -66,4 +66,4 @@ - \ No newline at end of file + diff --git a/examples/pom.xml b/examples/pom.xml index fa195a3f0f4..b24a0f7dd42 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 71a7d3192cc..9d17c6b2c53 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT @@ -94,4 +94,4 @@ - \ No newline at end of file + diff --git a/indexer/pom.xml b/indexer/pom.xml index 413582985a2..d38a3a17769 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index c24a262529d..980aa96d5f0 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/realtime/pom.xml b/realtime/pom.xml index 8a71c169fd4..a8c50128f2b 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index bf0a71f4d75..544108cef5f 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT From fcbac9619391d7ba51e088a7b41e218dd59ebe20 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Tue, 12 Feb 2013 10:47:30 -0800 Subject: [PATCH 041/121] revert change to findNewSegmentHome: now return single ServerHolder --- .../druid/master/BalancerCostAnalyzer.java | 47 +++++++------------ .../druid/master/DruidMasterBalancer.java | 11 +++-- .../metamx/druid/master/rules/LoadRule.java | 17 +++---- .../druid/master/DruidMasterBalancerTest.java | 6 +++ 4 files changed, 39 insertions(+), 42 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index cfd99c142c8..3a08c4a05f3 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -20,14 +20,11 @@ package com.metamx.druid.master; import com.google.common.collect.Lists; -import com.google.common.collect.MinMaxPriorityQueue; -import com.metamx.common.Pair; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; -import java.util.Comparator; import java.util.List; import java.util.Random; @@ -61,7 +58,7 @@ public class BalancerCostAnalyzer * pairwise cost matrix). This is the cost of a cluster if each * segment were to get its own compute node. */ - public double calculateNormalization(List serverHolders) + public double calculateNormalization(final List serverHolders) { double cost = 0; for (ServerHolder server : serverHolders) { @@ -78,7 +75,7 @@ public class BalancerCostAnalyzer * A list of ServerHolders for a particular tier. * @return The initial cost of the Druid tier. */ - public double calculateInitialTotalCost(List serverHolders) + public double calculateInitialTotalCost(final List serverHolders) { double cost = 0; for (ServerHolder server : serverHolders) { @@ -105,7 +102,7 @@ public class BalancerCostAnalyzer * The second DataSegment. * @return The joint cost of placing the two DataSegments together on one node. */ - public double computeJointSegmentCosts(DataSegment segment1, DataSegment segment2) + public double computeJointSegmentCosts(final DataSegment segment1, final DataSegment segment2) { final Interval gap = segment1.getInterval().gap(segment2.getInterval()); @@ -150,7 +147,7 @@ public class BalancerCostAnalyzer * @return A ServerHolder sampled with probability proportional to the * number of segments on that server */ - private ServerHolder sampleServer(List serverHolders, int numSegments) + private ServerHolder sampleServer(final List serverHolders, final int numSegments) { final int num = rand.nextInt(numSegments); int cumulativeSegments = 0; @@ -172,7 +169,7 @@ public class BalancerCostAnalyzer * The total number of segments on a particular tier. * @return A BalancerSegmentHolder sampled uniformly at random. */ - public BalancerSegmentHolder pickSegmentToMove(List serverHolders, int numSegments) + public BalancerSegmentHolder pickSegmentToMove(final List serverHolders, final int numSegments) { /** We want to sample from each server w.p. numSegmentsOnServer / totalSegments */ ServerHolder fromServerHolder = sampleServer(serverHolders, numSegments); @@ -191,30 +188,19 @@ public class BalancerCostAnalyzer * A DataSegment that we are proposing to move. * @param serverHolders * An iterable of ServerHolders for a particular tier. - * @return A MinMaxPriorityQueue of costs of putting the proposalSegment on the server and ServerHolders. + * @return A ServerHolder with the new home for a segment. */ - public MinMaxPriorityQueue> findNewSegmentHome(DataSegment proposalSegment, Iterable serverHolders) + public ServerHolder findNewSegmentHome(final DataSegment proposalSegment, final Iterable serverHolders) { - // Just need a regular priority queue for the min. element. - final MinMaxPriorityQueue> costServerPairs = MinMaxPriorityQueue.orderedBy( - new Comparator>() - { - @Override - public int compare( - Pair o, - Pair o1 - ) - { - return Double.compare(o.lhs, o1.lhs); - } - } - ).create(); - final long proposalSegmentSize = proposalSegment.getSize(); + double minCost = Double.MAX_VALUE; + ServerHolder toServer = null; for (ServerHolder server : serverHolders) { - /** Only calculate costs if the server has enough space. */ - if (proposalSegmentSize > server.getAvailableSize()) { + /** Don't calculate cost if the server doesn't have enough space or is serving/loading the segment. */ + if (proposalSegmentSize > server.getAvailableSize() + || server.isServingSegment(proposalSegment) + || server.isLoadingSegment(proposalSegment)) { break; } @@ -231,10 +217,13 @@ public class BalancerCostAnalyzer cost += computeJointSegmentCosts(proposalSegment, segment); } - costServerPairs.add(Pair.of(cost, server)); + if (cost < minCost) { + minCost = cost; + toServer = server; + } } - return costServerPairs; + return toServer; } } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index d766f2d663f..d82ada0b207 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -109,12 +109,17 @@ public class DruidMasterBalancer implements DruidMasterHelper int iter = 0; while (iter < maxSegmentsToMove) { - BalancerSegmentHolder segmentToMove = analyzer.pickSegmentToMove(serverHolderList, numSegments); - DruidServer toServer = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList).pollFirst().rhs.getServer(); + iter++; + final BalancerSegmentHolder segmentToMove = analyzer.pickSegmentToMove(serverHolderList, numSegments); + final ServerHolder holder = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList); + if (holder == null) { + continue; + } + final DruidServer toServer = holder.getServer(); + if (!currentlyMovingSegments.get(tier).containsKey(segmentToMove.getSegment())) { moveSegment(segmentToMove, toServer, params); } - iter++; } final double initialTotalCost = analyzer.calculateInitialTotalCost(serverHolderList); diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index 7c26c8b2d6d..0dadf86d9da 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -62,14 +62,14 @@ public abstract class LoadRule implements Rule final List serverHolderList = new ArrayList(serverQueue); final DateTime referenceTimestamp = params.getBalancerReferenceTimestamp(); final BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(referenceTimestamp); - MinMaxPriorityQueue> serverCostQueue = analyzer.findNewSegmentHome(segment, serverHolderList); stats.accumulate( assign( params.getReplicationManager(), expectedReplicants, totalReplicants, - serverCostQueue, + analyzer, + serverHolderList, segment ) ); @@ -81,16 +81,17 @@ public abstract class LoadRule implements Rule private MasterStats assign( final ReplicationThrottler replicationManager, - int expectedReplicants, + final int expectedReplicants, int totalReplicants, - MinMaxPriorityQueue> serverQueue, + final BalancerCostAnalyzer analyzer, + final List serverHolderList, final DataSegment segment ) { - MasterStats stats = new MasterStats(); + final MasterStats stats = new MasterStats(); while (totalReplicants < expectedReplicants) { - ServerHolder holder = serverQueue.pollFirst().rhs; + ServerHolder holder = analyzer.findNewSegmentHome(segment, serverHolderList); if (holder == null) { log.warn( @@ -102,10 +103,6 @@ public abstract class LoadRule implements Rule break; } - if (holder.isServingSegment(segment) || holder.isLoadingSegment(segment)) { - continue; - } - if (totalReplicants > 0) { // don't throttle if there's only 1 copy of this segment in the cluster if (!replicationManager.canAddReplicant(getTier()) || !replicationManager.registerReplicantCreation(getTier(), segment.getIdentifier())) { diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 504c4f4cfef..2ff1dade115 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -143,6 +143,7 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer1.getSegments()).andReturn(segments).anyTimes(); + EasyMock.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); EasyMock.expect(druidServer2.getName()).andReturn("to").atLeastOnce(); @@ -150,6 +151,7 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer2.getSegments()).andReturn(new HashMap()).anyTimes(); + EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); EasyMock.replay(druidServer3); @@ -202,6 +204,7 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer1.getSegments()).andReturn(segments).anyTimes(); + EasyMock.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); EasyMock.expect(druidServer2.getName()).andReturn("2").atLeastOnce(); @@ -209,6 +212,7 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer2.getSegments()).andReturn(new HashMap()).anyTimes(); + EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); EasyMock.expect(druidServer3.getName()).andReturn("3").atLeastOnce(); @@ -216,6 +220,7 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer3.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer3.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer3.getSegments()).andReturn(new HashMap()).anyTimes(); + EasyMock.expect(druidServer3.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer3); EasyMock.expect(druidServer4.getName()).andReturn("4").atLeastOnce(); @@ -223,6 +228,7 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer4.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer4.getMaxSize()).andReturn(100L).atLeastOnce(); EasyMock.expect(druidServer4.getSegments()).andReturn(new HashMap()).anyTimes(); + EasyMock.expect(druidServer4.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer4); // Mock stuff that the master needs From 429e23d3b2741211abff1677ef23d48e33f3b55d Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Wed, 13 Feb 2013 08:57:39 -0800 Subject: [PATCH 042/121] change break to continue in BalancerCostAnalyzer to skip calculating cost when it's not necessary --- .../main/java/com/metamx/druid/master/BalancerCostAnalyzer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 3a08c4a05f3..6e7ca4d20a4 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -201,7 +201,7 @@ public class BalancerCostAnalyzer if (proposalSegmentSize > server.getAvailableSize() || server.isServingSegment(proposalSegment) || server.isLoadingSegment(proposalSegment)) { - break; + continue; } /** The contribution to the total cost of a given server by proposing to move the segment to that server is... */ From c5b78e75cfe80b594ad5eb09db94ec830fa1e72b Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Tue, 12 Feb 2013 20:58:12 -0600 Subject: [PATCH 043/121] 1) Move up to Jackson 2.1 2) Some bugs were fixed, but they ended up getting mixed in with the Jackson upgrade :( --- client/pom.xml | 19 +++--- .../main/java/com/metamx/druid/BaseNode.java | 3 +- .../main/java/com/metamx/druid/BaseQuery.java | 3 +- .../src/main/java/com/metamx/druid/Query.java | 6 +- .../java/com/metamx/druid/QueryableNode.java | 9 ++- .../com/metamx/druid/RegisteringNode.java | 3 +- .../metamx/druid/client/BrokerServerView.java | 3 +- .../druid/client/CachingClusteredClient.java | 6 +- .../com/metamx/druid/client/DataSegment.java | 12 ++-- .../druid/client/DirectDruidClient.java | 20 ++++--- .../metamx/druid/client/DruidDataSource.java | 3 +- .../com/metamx/druid/client/DruidServer.java | 6 +- .../com/metamx/druid/client/ZKPhoneBook.java | 3 +- .../com/metamx/druid/http/BrokerNode.java | 6 +- .../druid/http/ClientServletModule.java | 6 +- .../com/metamx/druid/http/QueryServlet.java | 6 +- .../druid/initialization/Initialization.java | 3 +- .../metamx/druid/merge/ClientAppendQuery.java | 6 +- .../druid/merge/ClientDefaultMergeQuery.java | 6 +- .../metamx/druid/merge/ClientDeleteQuery.java | 6 +- .../metamx/druid/merge/ClientMergeQuery.java | 8 ++- .../com/metamx/druid/query/CacheStrategy.java | 3 +- .../metamx/druid/query/QueryToolChest.java | 3 +- .../query/dimension/DefaultDimensionSpec.java | 6 +- .../druid/query/dimension/DimensionSpec.java | 6 +- .../dimension/ExtractionDimensionSpec.java | 6 +- .../query/dimension/LegacyDimensionSpec.java | 3 +- .../query/extraction/DimExtractionFn.java | 5 +- .../extraction/PartialDimExtractionFn.java | 7 ++- .../extraction/RegexDimExtractionFn.java | 7 ++- .../SearchQuerySpecDimExtractionFn.java | 6 +- .../query/extraction/TimeDimExtractionFn.java | 6 +- .../druid/query/filter/AndDimFilter.java | 6 +- .../metamx/druid/query/filter/DimFilter.java | 5 +- .../query/filter/ExtractionDimFilter.java | 6 +- .../druid/query/filter/NotDimFilter.java | 7 ++- .../druid/query/filter/OrDimFilter.java | 6 +- .../druid/query/filter/RegexDimFilter.java | 6 +- .../query/filter/SearchQueryDimFilter.java | 3 +- .../druid/query/filter/SelectorDimFilter.java | 7 ++- .../druid/query/group/GroupByQuery.java | 6 +- .../group/GroupByQueryQueryToolChest.java | 3 +- .../druid/query/metadata/ColumnAnalysis.java | 6 +- .../query/metadata/ColumnIncluderator.java | 5 +- .../metadata/ListColumnIncluderator.java | 6 +- .../druid/query/metadata/SegmentAnalysis.java | 6 +- .../query/metadata/SegmentMetadataQuery.java | 3 +- .../SegmentMetadataQueryQueryToolChest.java | 3 +- .../query/search/FragmentSearchQuerySpec.java | 6 +- .../InsensitiveContainsSearchQuerySpec.java | 7 ++- .../search/LexicographicSearchSortSpec.java | 4 +- .../metamx/druid/query/search/SearchHit.java | 7 ++- .../druid/query/search/SearchQuery.java | 6 +- .../search/SearchQueryQueryToolChest.java | 3 +- .../druid/query/search/SearchQuerySpec.java | 6 +- .../druid/query/search/SearchSortSpec.java | 7 ++- .../query/segment/LegacySegmentSpec.java | 3 +- .../segment/MultipleIntervalSegmentSpec.java | 6 +- .../segment/MultipleSpecificSegmentSpec.java | 6 +- .../druid/query/segment/QuerySegmentSpec.java | 8 ++- .../query/segment/SegmentDescriptor.java | 6 +- .../query/timeboundary/TimeBoundaryQuery.java | 6 +- .../TimeBoundaryQueryQueryToolChest.java | 3 +- .../query/timeseries/TimeseriesQuery.java | 11 ++-- .../TimeseriesQueryQueryToolChest.java | 3 +- .../result/BySegmentResultValueClass.java | 3 +- .../result/BySegmentSearchResultValue.java | 6 +- .../druid/result/MetricValueExtractor.java | 7 ++- .../java/com/metamx/druid/result/Result.java | 8 ++- .../druid/result/SearchResultValue.java | 6 +- .../druid/result/TimeBoundaryResultValue.java | 6 +- .../druid/result/TimeseriesResultValue.java | 4 +- .../metamx/druid/shard/LinearShardSpec.java | 3 +- .../com/metamx/druid/shard/NoneShardSpec.java | 2 +- .../com/metamx/druid/shard/ShardSpec.java | 8 ++- .../druid/shard/SingleDimensionShardSpec.java | 5 +- .../metamx/druid/client/DataSegmentTest.java | 6 +- .../druid/query/QuerySegmentSpecTest.java | 3 +- common/pom.xml | 15 +++-- .../com/metamx/druid/DurationGranularity.java | 6 +- .../com/metamx/druid/PeriodGranularity.java | 6 +- .../com/metamx/druid/QueryGranularity.java | 11 ++-- .../druid/aggregation/AggregatorFactory.java | 8 ++- .../aggregation/CountAggregatorFactory.java | 6 +- .../DoubleSumAggregatorFactory.java | 6 +- .../metamx/druid/aggregation/Histogram.java | 3 +- .../HistogramAggregatorFactory.java | 6 +- .../druid/aggregation/HistogramVisual.java | 9 ++- .../JavaScriptAggregatorFactory.java | 6 +- .../aggregation/LongSumAggregatorFactory.java | 6 +- .../aggregation/MaxAggregatorFactory.java | 6 +- .../aggregation/MinAggregatorFactory.java | 6 +- .../post/ArithmeticPostAggregator.java | 6 +- .../post/ConstantPostAggregator.java | 6 +- .../post/FieldAccessPostAggregator.java | 6 +- .../aggregation/post/PostAggregator.java | 7 ++- .../metamx/druid/db/DbConnectorConfig.java | 3 +- .../com/metamx/druid/input/MapBasedRow.java | 6 +- .../main/java/com/metamx/druid/input/Row.java | 7 ++- .../jackson/CommaListJoinDeserializer.java | 13 ++-- .../jackson/CommaListJoinSerializer.java | 14 +++-- .../druid/jackson/DefaultObjectMapper.java | 60 +++++++++++++------ .../com/metamx/druid/jackson/JodaStuff.java | 28 +++++---- .../metamx/druid/QueryGranularityTest.java | 3 +- .../metamx/druid/histogram/HistogramTest.java | 3 +- .../jackson/DefaultObjectMapperTest.java | 40 +++++++++++++ examples/rand/pom.xml | 19 +++--- .../druid/examples/RandomFirehoseFactory.java | 9 ++- .../examples/RealtimeStandaloneMain.java | 3 +- examples/twitter/pom.xml | 19 +++--- .../examples/RealtimeStandaloneMain.java | 3 +- .../TwitterSpritzerFirehoseFactory.java | 9 ++- index-common/pom.xml | 7 ++- .../index/column/ColumnCapabilitiesImpl.java | 3 +- .../druid/index/column/ColumnDescriptor.java | 6 +- .../druid/index/serde/ColumnPartSerde.java | 6 +- .../index/serde/ComplexColumnPartSerde.java | 6 +- .../DictionaryEncodedColumnPartSerde.java | 6 +- .../serde/FloatGenericColumnPartSerde.java | 6 +- .../serde/LongGenericColumnPartSerde.java | 6 +- .../com/metamx/druid/index/v1/IndexIO.java | 3 +- .../druid/index/v1/serde/Registererer.java | 3 +- .../druid/indexer/data/CSVDataSpec.java | 6 +- .../metamx/druid/indexer/data/DataSpec.java | 6 +- .../druid/indexer/data/DelimitedDataSpec.java | 6 +- .../druid/indexer/data/JSONDataSpec.java | 3 +- .../indexer/data/StringInputRowParser.java | 6 +- .../druid/indexer/data/TimestampSpec.java | 6 +- .../indexer/data/ToLowercaseDataSpec.java | 3 +- indexer/pom.xml | 11 ++-- .../metamx/druid/indexer/DbUpdaterJob.java | 3 +- .../druid/indexer/DeterminePartitionsJob.java | 3 +- .../druid/indexer/HadoopDruidIndexer.java | 2 +- .../indexer/HadoopDruidIndexerConfig.java | 59 +++++++++--------- .../druid/indexer/HadoopDruidIndexerNode.java | 4 +- .../druid/indexer/HadoopyShardSpec.java | 6 +- .../java/com/metamx/druid/indexer/Utils.java | 6 +- .../granularity/ArbitraryGranularitySpec.java | 6 +- .../indexer/granularity/GranularitySpec.java | 6 +- .../granularity/UniformGranularitySpec.java | 6 +- .../indexer/partitions/PartitionsSpec.java | 9 ++- .../path/GranularUnprocessedPathSpec.java | 3 +- .../indexer/path/GranularityPathSpec.java | 3 +- .../metamx/druid/indexer/path/PathSpec.java | 8 ++- .../druid/indexer/path/StaticPathSpec.java | 3 +- .../druid/indexer/rollup/DataRollupSpec.java | 3 +- .../indexer/updater/DbUpdaterJobSpec.java | 3 +- .../druid/indexer/updater/UpdaterJobSpec.java | 5 +- .../indexer/updater/ZkUpdaterJobSpec.java | 3 +- .../indexer/HadoopDruidIndexerConfigTest.java | 7 ++- .../granularity/ArbitraryGranularityTest.java | 3 +- .../granularity/UniformGranularityTest.java | 3 +- merger/pom.xml | 15 ++--- .../druid/merger/common/TaskHolder.java | 6 +- .../druid/merger/common/TaskStatus.java | 6 +- .../druid/merger/common/TaskToolbox.java | 3 +- .../common/index/StaticS3FirehoseFactory.java | 12 ++-- .../common/index/YeOldePlumberSchool.java | 12 ++-- .../merger/common/task/AbstractTask.java | 6 +- .../druid/merger/common/task/AppendTask.java | 6 +- .../merger/common/task/DefaultMergeTask.java | 6 +- .../druid/merger/common/task/DeleteTask.java | 8 ++- .../task/IndexDeterminePartitionsTask.java | 9 ++- .../common/task/IndexGeneratorTask.java | 6 +- .../druid/merger/common/task/IndexTask.java | 6 +- .../druid/merger/common/task/MergeTask.java | 9 ++- .../metamx/druid/merger/common/task/Task.java | 6 +- .../merger/common/task/V8toV9UpgradeTask.java | 3 +- .../merger/coordinator/DbTaskStorage.java | 3 +- .../coordinator/MergerDBCoordinator.java | 3 +- .../merger/coordinator/RemoteTaskRunner.java | 3 +- .../druid/merger/coordinator/TaskContext.java | 6 +- .../merger/coordinator/WorkerWrapper.java | 3 +- .../config/IndexerDbConnectorConfig.java | 3 +- .../http/IndexerCoordinatorNode.java | 6 +- .../http/IndexerCoordinatorServletModule.java | 6 +- .../scaling/EC2AutoScalingStrategy.java | 3 +- .../merger/coordinator/setup/EC2NodeData.java | 7 ++- .../coordinator/setup/GalaxyUserData.java | 5 +- .../coordinator/setup/WorkerSetupData.java | 7 ++- .../coordinator/setup/WorkerSetupManager.java | 3 +- .../druid/merger/worker/TaskMonitor.java | 2 +- .../metamx/druid/merger/worker/Worker.java | 6 +- .../worker/WorkerCuratorCoordinator.java | 3 +- .../druid/merger/worker/http/WorkerNode.java | 6 +- .../coordinator/RemoteTaskRunnerTest.java | 14 +++-- pom.xml | 43 ++++++++----- realtime/pom.xml | 15 ++--- .../metamx/druid/realtime/FireDepartment.java | 7 ++- .../druid/realtime/FireDepartmentConfig.java | 3 +- .../druid/realtime/FirehoseFactory.java | 7 ++- .../druid/realtime/KafkaFirehoseFactory.java | 6 +- .../druid/realtime/MetadataUpdater.java | 3 +- .../metamx/druid/realtime/PlumberSchool.java | 5 +- .../metamx/druid/realtime/RealtimeNode.java | 18 ++++-- .../druid/realtime/RealtimePlumberSchool.java | 15 +++-- .../druid/realtime/S3SegmentPusher.java | 3 +- .../com/metamx/druid/realtime/Schema.java | 6 +- server/pom.xml | 27 +++++---- .../src/main/java/com/metamx/TsvToJson.java | 3 +- .../java/com/metamx/druid/BaseServerNode.java | 3 +- .../DataSegmentChangeRequest.java | 8 ++- .../SegmentChangeRequestDrop.java | 9 ++- .../SegmentChangeRequestLoad.java | 9 ++- .../druid/coordination/ZkCoordinator.java | 3 +- .../metamx/druid/db/DatabaseRuleManager.java | 6 +- .../druid/db/DatabaseSegmentManager.java | 3 +- .../com/metamx/druid/http/ComputeNode.java | 6 +- .../com/metamx/druid/http/MasterMain.java | 3 +- .../druid/http/MasterServletModule.java | 6 +- .../com/metamx/druid/http/SegmentToDrop.java | 5 +- .../com/metamx/druid/http/SegmentToMove.java | 5 +- .../druid/index/brita/SearchQueryFilter.java | 6 +- .../druid/index/v1/IndexGranularity.java | 3 +- .../metamx/druid/loading/S3SegmentPusher.java | 3 +- .../com/metamx/druid/master/DruidMaster.java | 3 +- .../master/DruidMasterSegmentMerger.java | 3 +- .../metamx/druid/master/HttpMergerClient.java | 3 +- .../druid/master/rules/IntervalDropRule.java | 6 +- .../druid/master/rules/IntervalLoadRule.java | 6 +- .../druid/master/rules/PeriodDropRule.java | 6 +- .../druid/master/rules/PeriodLoadRule.java | 6 +- .../com/metamx/druid/master/rules/Rule.java | 6 +- .../SegmentChangeRequestDropTest.java | 6 +- .../SegmentChangeRequestLoadTest.java | 6 +- .../druid/coordination/ServerManagerTest.java | 3 +- .../druid/coordination/ZkCoordinatorTest.java | 3 +- 227 files changed, 1029 insertions(+), 542 deletions(-) create mode 100644 common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java diff --git a/client/pom.xml b/client/pom.xml index 36bb84de0d4..91674a214cc 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-client @@ -106,20 +107,20 @@ icu4j - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile javax.inject diff --git a/client/src/main/java/com/metamx/druid/BaseNode.java b/client/src/main/java/com/metamx/druid/BaseNode.java index 7c7ed5f0984..8902d14a65f 100644 --- a/client/src/main/java/com/metamx/druid/BaseNode.java +++ b/client/src/main/java/com/metamx/druid/BaseNode.java @@ -19,9 +19,10 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; -import org.codehaus.jackson.map.ObjectMapper; + import org.skife.config.ConfigurationObjectFactory; import java.util.Properties; diff --git a/client/src/main/java/com/metamx/druid/BaseQuery.java b/client/src/main/java/com/metamx/druid/BaseQuery.java index 76448ed9f17..9f3ae8d30ed 100644 --- a/client/src/main/java/com/metamx/druid/BaseQuery.java +++ b/client/src/main/java/com/metamx/druid/BaseQuery.java @@ -19,13 +19,14 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.metamx.common.guava.Sequence; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentWalker; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/Query.java b/client/src/main/java/com/metamx/druid/Query.java index 4c4e7f715b4..818400fc262 100644 --- a/client/src/main/java/com/metamx/druid/Query.java +++ b/client/src/main/java/com/metamx/druid/Query.java @@ -19,6 +19,8 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.guava.Sequence; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.group.GroupByQuery; @@ -28,8 +30,8 @@ import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; import com.metamx.druid.query.timeseries.TimeseriesQuery; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index 76e835d4ffe..749b2bd8b40 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -19,6 +19,9 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -47,9 +50,9 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import com.metamx.phonebook.PhoneBook; import org.I0Itec.zkclient.ZkClient; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.jsontype.NamedType; -import org.codehaus.jackson.smile.SmileFactory; + + + import org.mortbay.jetty.Server; import org.skife.config.ConfigurationObjectFactory; diff --git a/client/src/main/java/com/metamx/druid/RegisteringNode.java b/client/src/main/java/com/metamx/druid/RegisteringNode.java index 93d8fd7c8cd..1332bbd2a4b 100644 --- a/client/src/main/java/com/metamx/druid/RegisteringNode.java +++ b/client/src/main/java/com/metamx/druid/RegisteringNode.java @@ -19,8 +19,9 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.index.v1.serde.Registererer; -import org.codehaus.jackson.map.ObjectMapper; + import java.util.Arrays; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java b/client/src/main/java/com/metamx/druid/client/BrokerServerView.java index e37e0bfeb01..98900e9de41 100644 --- a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java +++ b/client/src/main/java/com/metamx/druid/client/BrokerServerView.java @@ -19,6 +19,7 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -28,7 +29,7 @@ import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.http.client.HttpClient; -import org.codehaus.jackson.map.ObjectMapper; + import javax.annotation.Nullable; import java.util.Iterator; diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index ad08ede04eb..6d896d8bf0f 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -19,6 +19,8 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.base.Throwables; @@ -53,8 +55,8 @@ import com.metamx.druid.query.segment.MultipleSpecificSegmentSpec; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.BySegmentResultValueClass; import com.metamx.druid.result.Result; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/client/DataSegment.java b/client/src/main/java/com/metamx/druid/client/DataSegment.java index 9b29f3507ee..b915f7680cd 100644 --- a/client/src/main/java/com/metamx/druid/client/DataSegment.java +++ b/client/src/main/java/com/metamx/druid/client/DataSegment.java @@ -19,6 +19,10 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; @@ -31,10 +35,10 @@ import com.metamx.druid.jackson.CommaListJoinSerializer; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.annotate.JsonDeserialize; -import org.codehaus.jackson.map.annotate.JsonSerialize; + + + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java b/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java index 62782a41892..7c3e23c1395 100644 --- a/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java +++ b/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java @@ -19,6 +19,13 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.core.ObjectCodec; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.type.TypeFactory; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.google.common.io.Closeables; @@ -40,13 +47,12 @@ import com.metamx.http.client.HttpClient; import com.metamx.http.client.io.AppendableByteArrayInputStream; import com.metamx.http.client.response.ClientResponse; import com.metamx.http.client.response.InputStreamResponseHandler; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonToken; -import org.codehaus.jackson.ObjectCodec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.type.TypeFactory; -import org.codehaus.jackson.smile.SmileFactory; -import org.codehaus.jackson.type.JavaType; + + + + + + import org.jboss.netty.handler.codec.http.HttpChunk; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpResponse; diff --git a/client/src/main/java/com/metamx/druid/client/DruidDataSource.java b/client/src/main/java/com/metamx/druid/client/DruidDataSource.java index 6f96e8e8672..e112bbf6234 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidDataSource.java +++ b/client/src/main/java/com/metamx/druid/client/DruidDataSource.java @@ -19,8 +19,9 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.Collections; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/client/DruidServer.java b/client/src/main/java/com/metamx/druid/client/DruidServer.java index 6ecfdb16a64..a71d8a15ac7 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServer.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServer.java @@ -19,10 +19,12 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.metamx.common.logger.Logger; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.HashMap; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java b/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java index af9d552d009..1b8fa9df876 100644 --- a/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java +++ b/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java @@ -19,6 +19,7 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.HashBasedTable; import com.google.common.collect.Maps; @@ -33,7 +34,7 @@ import org.I0Itec.zkclient.IZkChildListener; import org.I0Itec.zkclient.ZkClient; import org.I0Itec.zkclient.exception.ZkNoNodeException; import org.I0Itec.zkclient.exception.ZkNodeExistsException; -import org.codehaus.jackson.map.ObjectMapper; + import java.io.IOException; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/http/BrokerNode.java b/client/src/main/java/com/metamx/druid/http/BrokerNode.java index 2a94a00d76d..76e487efb56 100644 --- a/client/src/main/java/com/metamx/druid/http/BrokerNode.java +++ b/client/src/main/java/com/metamx/druid/http/BrokerNode.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.inject.Guice; @@ -53,8 +55,8 @@ import com.metamx.http.client.HttpClientInit; import com.metamx.metrics.Monitor; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.x.discovery.ServiceDiscovery; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.smile.SmileFactory; + + import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; diff --git a/client/src/main/java/com/metamx/druid/http/ClientServletModule.java b/client/src/main/java/com/metamx/druid/http/ClientServletModule.java index 7ecc5d7ad81..c5b89f37004 100644 --- a/client/src/main/java/com/metamx/druid/http/ClientServletModule.java +++ b/client/src/main/java/com/metamx/druid/http/ClientServletModule.java @@ -19,13 +19,15 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; import com.metamx.druid.client.ClientInventoryManager; import com.metamx.druid.query.segment.QuerySegmentWalker; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import org.codehaus.jackson.jaxrs.JacksonJsonProvider; -import org.codehaus.jackson.map.ObjectMapper; + + import javax.inject.Singleton; diff --git a/client/src/main/java/com/metamx/druid/http/QueryServlet.java b/client/src/main/java/com/metamx/druid/http/QueryServlet.java index 108863bf7af..9a13c50f45c 100644 --- a/client/src/main/java/com/metamx/druid/http/QueryServlet.java +++ b/client/src/main/java/com/metamx/druid/http/QueryServlet.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; @@ -31,8 +33,8 @@ import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.ObjectWriter; + + import org.joda.time.DateTime; import org.mortbay.jetty.Request; diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index cfb2d9302a2..4865208e88e 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -19,6 +19,7 @@ package com.metamx.druid.initialization; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.io.Closeables; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -41,7 +42,7 @@ import com.netflix.curator.x.discovery.ServiceInstance; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.ZkClient; import org.I0Itec.zkclient.ZkConnection; -import org.codehaus.jackson.map.ObjectMapper; + import org.mortbay.jetty.Connector; import org.mortbay.jetty.Server; import org.mortbay.jetty.nio.SelectChannelConnector; diff --git a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java index 441adcffc3b..5a8e3bdb50f 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java @@ -19,9 +19,11 @@ package com.metamx.druid.merge; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java index c5cc8045492..4286cd211cb 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java @@ -19,10 +19,12 @@ package com.metamx.druid.merge; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java index a0733db0a2a..3acf20a0a43 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java @@ -19,8 +19,10 @@ package com.metamx.druid.merge; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; public class ClientDeleteQuery diff --git a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java index 83f269d0db3..3956991ed6b 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java @@ -19,15 +19,17 @@ package com.metamx.druid.merge; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.List; /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class) @JsonSubTypes(value={ @JsonSubTypes.Type(name="append", value=ClientAppendQuery.class) }) diff --git a/client/src/main/java/com/metamx/druid/query/CacheStrategy.java b/client/src/main/java/com/metamx/druid/query/CacheStrategy.java index f8f5098f6ca..b17e95b86e7 100644 --- a/client/src/main/java/com/metamx/druid/query/CacheStrategy.java +++ b/client/src/main/java/com/metamx/druid/query/CacheStrategy.java @@ -19,10 +19,11 @@ package com.metamx.druid.query; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; -import org.codehaus.jackson.type.TypeReference; + /** */ diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 701a4512403..f729016032e 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -19,13 +19,14 @@ package com.metamx.druid.query; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.client.selector.ServerSelector; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java index a936023550f..a3c36476253 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java index 34310ef4c09..bcf5fabf62b 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + /** */ diff --git a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java index fe80c24c94f..6feb011dcd7 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java index 9a4e11a9a5f..b753b04c46b 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonCreator; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java index 0c1ef8b159c..8fe869cb4af 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.extraction; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java index 28ea1ebc37b..f30c4578706 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.extraction; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; import java.util.regex.Matcher; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java index da3a878aa2f..c722b0c52b7 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.extraction; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; import java.util.regex.Matcher; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java index 57e3bfde16b..7dbc2f3c65d 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.extraction; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.search.SearchQuerySpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java index dc2c2eb36cf..c02797cf088 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.extraction; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.ibm.icu.text.SimpleDateFormat; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.text.ParseException; diff --git a/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java index 92696b29c26..6b557fdba27 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Joiner; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java index 9dc819cb8ec..8da47da465c 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.filter; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java index db3376b9de8..21785f06b0c 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java index 8b474d301f3..0753d9f64de 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.filter; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java index 2a056c8dfd6..9105cfe0412 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Joiner; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java index 519814d8262..6ee15d4b0c9 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java index b579acd87c5..7b1b6b0fd15 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java @@ -19,9 +19,10 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.metamx.druid.query.search.SearchQuerySpec; -import org.codehaus.jackson.annotate.JsonProperty; + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java index c778027d48c..11deaeb28e2 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.filter; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java index d30af309473..4c1ac740dca 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java @@ -19,6 +19,8 @@ package com.metamx.druid.query.group; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -33,8 +35,8 @@ import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.segment.LegacySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index 1c66774d3bc..51570e30955 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.group; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; @@ -43,7 +44,7 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java b/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java index 33b894ca07a..9eb8a46bcdb 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java @@ -19,10 +19,12 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.druid.index.column.ValueType; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** */ diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java index 90533c4eaca..c6cfb081835 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.metadata; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java index e74661d6822..a7de6b1920e 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java @@ -19,12 +19,14 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.ByteArrayOutputStream; import java.io.IOException; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java index 1182bfeb9fa..22657e45301 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java @@ -19,8 +19,10 @@ package com.metamx.druid.query.metadata; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java index 7e0d04c0739..707a0a285b2 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java @@ -19,10 +19,11 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.BaseQuery; import com.metamx.druid.Query; import com.metamx.druid.query.segment.QuerySegmentSpec; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 2b979117800..6ede9c7f247 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; @@ -39,7 +40,7 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.ResultMergeQueryRunner; import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java index e0a73b4bc3c..b813b7febbf 100644 --- a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java @@ -19,10 +19,12 @@ package com.metamx.druid.query.search; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Lists; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java index 87a6246a5b2..1d7a836a3ab 100644 --- a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java b/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java index b7dfddef413..83f81bfc349 100644 --- a/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java @@ -19,7 +19,9 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonCreator; + + +import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Comparator; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchHit.java b/client/src/main/java/com/metamx/druid/query/search/SearchHit.java index c1c0727067a..ba896912197 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchHit.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchHit.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import static com.google.common.base.Preconditions.checkNotNull; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java index 4ce3bd72e7f..124d9c5fa62 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java @@ -19,6 +19,8 @@ package com.metamx.druid.query.search; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -29,8 +31,8 @@ import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import javax.annotation.Nullable; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index cefc32b5e3d..f58f31ebad7 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.search; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; @@ -49,7 +50,7 @@ import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java index 3f634bd3530..dbd0e91c48d 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.search; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Predicate; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java b/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java index b68f074e2ce..f98cf41f999 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.util.Comparator; diff --git a/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java index 6d25c152ab4..f4e62ff77c2 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java @@ -19,10 +19,11 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; + import org.joda.time.Interval; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java index f81126857bc..5432d468ed4 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java @@ -19,11 +19,13 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.Query; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.utils.JodaUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import java.util.Collections; diff --git a/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java index 4193934fbc0..fb259c5e5d5 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java @@ -19,13 +19,15 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.metamx.druid.Query; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.utils.JodaUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java index ca42ce8bc8a..98fc8e7b327 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java @@ -19,17 +19,19 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.Query; import com.metamx.druid.query.QueryRunner; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.Interval; import java.util.List; /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type", defaultImpl = LegacySegmentSpec.class) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = LegacySegmentSpec.class) @JsonSubTypes(value={ @JsonSubTypes.Type(name="intervals", value=MultipleIntervalSegmentSpec.class), @JsonSubTypes.Type(name="segments", value=MultipleSpecificSegmentSpec.class) diff --git a/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java b/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java index d918309baa6..90d433c1e5d 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java +++ b/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java @@ -19,8 +19,10 @@ package com.metamx.druid.query.segment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; /** diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java index 071d7695c0f..07fd071d6db 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java @@ -19,6 +19,8 @@ package com.metamx.druid.query.timeboundary; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -28,8 +30,8 @@ import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 8e41f1f6eb8..86e618bcee8 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.timeboundary; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.collect.Lists; @@ -40,7 +41,7 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java index 490e2531edd..a67caf19924 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java @@ -19,6 +19,9 @@ package com.metamx.druid.query.timeseries; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.metamx.druid.BaseQuery; @@ -31,10 +34,10 @@ import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeName; + + + + import java.util.List; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 538897480b3..bef34d9a2da 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.timeseries; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.collect.Lists; @@ -45,7 +46,7 @@ import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java b/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java index fb1e0df8e22..dfaacb572cc 100644 --- a/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java +++ b/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java @@ -19,7 +19,8 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java b/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java index b324ccbab37..e5a069f17a1 100644 --- a/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java @@ -19,9 +19,11 @@ package com.metamx.druid.result; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.druid.query.search.SearchHit; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonValue; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java b/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java index 3baf45c6a67..c99ac0f3ed3 100644 --- a/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java +++ b/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java @@ -19,8 +19,11 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonValue; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import java.lang.Double;import java.lang.Float;import java.lang.Long;import java.lang.Number;import java.lang.Object;import java.lang.Override;import java.lang.String;import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/result/Result.java b/client/src/main/java/com/metamx/druid/result/Result.java index 9b055cd4800..f9c30fc8319 100644 --- a/client/src/main/java/com/metamx/druid/result/Result.java +++ b/client/src/main/java/com/metamx/druid/result/Result.java @@ -19,9 +19,11 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonUnwrapped; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.DateTime; /** diff --git a/client/src/main/java/com/metamx/druid/result/SearchResultValue.java b/client/src/main/java/com/metamx/druid/result/SearchResultValue.java index 40f10ed85ec..c9dd74fc4ec 100644 --- a/client/src/main/java/com/metamx/druid/result/SearchResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/SearchResultValue.java @@ -19,9 +19,11 @@ package com.metamx.druid.result; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.druid.query.search.SearchHit; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonValue; + + import java.util.Iterator; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java b/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java index 74daa14f376..1b1b4c45547 100644 --- a/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java @@ -19,10 +19,12 @@ package com.metamx.druid.result; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.common.IAE; import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonValue; + + import org.joda.time.DateTime; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java b/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java index d68a9855395..55892023def 100644 --- a/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java @@ -19,7 +19,9 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonCreator; + + +import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java b/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java index 97f22ae6660..ae83287f290 100644 --- a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java @@ -1,9 +1,10 @@ package com.metamx.druid.shard; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.LinearPartitionChunk; import com.metamx.druid.partition.PartitionChunk; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java b/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java index 4ed756e1daf..9e40f9f703e 100644 --- a/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java @@ -22,7 +22,7 @@ package com.metamx.druid.shard; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.SingleElementPartitionChunk; -import org.codehaus.jackson.annotate.JsonTypeName; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/shard/ShardSpec.java b/client/src/main/java/com/metamx/druid/shard/ShardSpec.java index df3eb29004a..2db8f15ed5f 100644 --- a/client/src/main/java/com/metamx/druid/shard/ShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/ShardSpec.java @@ -19,17 +19,19 @@ package com.metamx.druid.shard; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.PartitionChunk; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.Map; /** * A Marker interface that exists to combine ShardSpec objects together for Jackson */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type", include=JsonTypeInfo.As.PROPERTY) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", include=JsonTypeInfo.As.PROPERTY) @JsonSubTypes({ @JsonSubTypes.Type(name="single", value=SingleDimensionShardSpec.class), @JsonSubTypes.Type(name="none", value=NoneShardSpec.class), diff --git a/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java b/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java index 7580673b071..0843ec7c61d 100644 --- a/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java @@ -19,11 +19,12 @@ package com.metamx.druid.shard; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.StringPartitionChunk; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; + + import java.util.List; import java.util.Map; diff --git a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java b/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java index 24d5986a8f8..082f66e8aac 100644 --- a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java +++ b/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java @@ -19,6 +19,8 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -27,8 +29,8 @@ import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java b/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java index a05973af729..654ea2e6b79 100644 --- a/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java +++ b/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.metamx.druid.jackson.DefaultObjectMapper; @@ -27,7 +28,7 @@ import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; import com.metamx.druid.query.segment.MultipleSpecificSegmentSpec; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.SegmentDescriptor; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/common/pom.xml b/common/pom.xml index 350828b9316..f334b521775 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-common @@ -72,12 +73,16 @@ guava - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.datatype + jackson-datatype-joda org.jdbi diff --git a/common/src/main/java/com/metamx/druid/DurationGranularity.java b/common/src/main/java/com/metamx/druid/DurationGranularity.java index da485b62e97..b2d5e55be42 100644 --- a/common/src/main/java/com/metamx/druid/DurationGranularity.java +++ b/common/src/main/java/com/metamx/druid/DurationGranularity.java @@ -19,9 +19,11 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Longs; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import java.nio.ByteBuffer; diff --git a/common/src/main/java/com/metamx/druid/PeriodGranularity.java b/common/src/main/java/com/metamx/druid/PeriodGranularity.java index 6fb461a5bc9..3c6ca5fd23e 100644 --- a/common/src/main/java/com/metamx/druid/PeriodGranularity.java +++ b/common/src/main/java/com/metamx/druid/PeriodGranularity.java @@ -19,8 +19,10 @@ package com.metamx.druid; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Chronology; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; diff --git a/common/src/main/java/com/metamx/druid/QueryGranularity.java b/common/src/main/java/com/metamx/druid/QueryGranularity.java index 36a4dd01714..5b94c5e1ae2 100644 --- a/common/src/main/java/com/metamx/druid/QueryGranularity.java +++ b/common/src/main/java/com/metamx/druid/QueryGranularity.java @@ -19,14 +19,17 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + import org.joda.time.DateTime; import org.joda.time.ReadableDuration; -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property = "type", defaultImpl = QueryGranularity.class) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property = "type", defaultImpl = QueryGranularity.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "period", value = PeriodGranularity.class), @JsonSubTypes.Type(name = "duration", value = DurationGranularity.class), diff --git a/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java index 00722fcf5eb..c12793972f2 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java @@ -19,9 +19,11 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.Comparator; import java.util.List; @@ -36,7 +38,7 @@ import java.util.List; * provided to the Aggregator through the MetricSelector object, so whatever creates that object gets to choose how * the data is actually stored and accessed. */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type") +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type") @JsonSubTypes(value={ @JsonSubTypes.Type(name="count", value=CountAggregatorFactory.class), @JsonSubTypes.Type(name="longSum", value=LongSumAggregatorFactory.class), diff --git a/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java index 61dcde40449..a901c3f54e2 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java @@ -19,11 +19,13 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Longs; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.List; diff --git a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java index dc8bc9a9842..92a972d0658 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Doubles; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java b/common/src/main/java/com/metamx/druid/aggregation/Histogram.java index bb27e317eb4..a3ec32d4a15 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java +++ b/common/src/main/java/com/metamx/druid/aggregation/Histogram.java @@ -19,11 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.base.Preconditions; import com.google.common.primitives.Floats; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; -import org.codehaus.jackson.annotate.JsonValue; + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java index 8902ae56bef..af6d23c32ac 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java @@ -19,13 +19,15 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.google.common.primitives.Floats; import com.google.common.primitives.Longs; import com.metamx.druid.processing.MetricSelectorFactory; import org.apache.commons.codec.binary.Base64; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java b/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java index ab4e8836fe1..01c01719854 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java +++ b/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java @@ -19,16 +19,19 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Arrays; public class HistogramVisual { @JsonProperty final public double[] breaks; - @JsonProperty final public double[] counts; + @JsonProperty + final public double[] counts; // an array of the quantiles including the min. and max. @JsonProperty final public double[] quantiles; diff --git a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java index 48e5cf776f5..5c257f8cc74 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java @@ -19,14 +19,16 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Joiner; import com.google.common.collect.Lists; import com.google.common.primitives.Doubles; import com.metamx.common.IAE; import com.metamx.druid.processing.FloatMetricSelector; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.mozilla.javascript.Context; import org.mozilla.javascript.Function; import org.mozilla.javascript.Script; diff --git a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java index 75e59b39e2c..61247e40add 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Longs; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java index 85a5466ec53..76a57f23066 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Doubles; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java index 49b2ad154c9..2720fa2a9f1 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Doubles; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java index cec19d80d78..1166413a3e7 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java @@ -19,11 +19,13 @@ package com.metamx.druid.aggregation.post; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.Iterator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java index f1bbb0d8392..f36761f962f 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java @@ -19,9 +19,11 @@ package com.metamx.druid.aggregation.post; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.Map; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java index 780c720103d..eb3a531e2d1 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation.post; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; import com.metamx.common.ISE; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.Map; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java index 5b1ebc60528..66427785fbc 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java @@ -19,8 +19,11 @@ package com.metamx.druid.aggregation.post; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.util.Comparator; import java.util.Map; diff --git a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java index b6e195cd5b7..808be63267c 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java @@ -19,7 +19,8 @@ package com.metamx.druid.db; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; import org.skife.config.Config; /** diff --git a/common/src/main/java/com/metamx/druid/input/MapBasedRow.java b/common/src/main/java/com/metamx/druid/input/MapBasedRow.java index d823762eddd..5449ff2f865 100644 --- a/common/src/main/java/com/metamx/druid/input/MapBasedRow.java +++ b/common/src/main/java/com/metamx/druid/input/MapBasedRow.java @@ -19,13 +19,15 @@ package com.metamx.druid.input; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.IAE; import com.metamx.common.exception.FormattedException; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/common/src/main/java/com/metamx/druid/input/Row.java b/common/src/main/java/com/metamx/druid/input/Row.java index add4623d498..353d3e52f06 100644 --- a/common/src/main/java/com/metamx/druid/input/Row.java +++ b/common/src/main/java/com/metamx/druid/input/Row.java @@ -19,8 +19,11 @@ package com.metamx.druid.input; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.util.List; diff --git a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java index 1932746b3d0..dd18713d797 100644 --- a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java +++ b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java @@ -19,10 +19,15 @@ package com.metamx.druid.jackson; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonProcessingException; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.deser.std.StdScalarDeserializer; + + + + + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer; import java.io.IOException; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java index 3b270ad4aa1..a837c88bbe7 100644 --- a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java +++ b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java @@ -19,18 +19,22 @@ package com.metamx.druid.jackson; +import com.fasterxml.jackson.core.JsonGenerationException; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer; import com.google.common.base.Joiner; -import org.codehaus.jackson.JsonGenerationException; -import org.codehaus.jackson.JsonGenerator; -import org.codehaus.jackson.map.SerializerProvider; -import org.codehaus.jackson.map.ser.std.ScalarSerializerBase; + + + + import java.io.IOException; import java.util.List; /** */ -public class CommaListJoinSerializer extends ScalarSerializerBase> +public class CommaListJoinSerializer extends StdScalarSerializer> { private static final Joiner joiner = Joiner.on(","); diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index c80958eff88..84514bd6c4c 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -19,26 +19,45 @@ package com.metamx.druid.jackson; +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.Version; +import com.fasterxml.jackson.databind.DeserializationConfig; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationConfig; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.fasterxml.jackson.datatype.joda.JodaModule; import com.google.common.base.Throwables; import com.metamx.common.Granularity; import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; -import org.codehaus.jackson.JsonFactory; -import org.codehaus.jackson.JsonGenerator; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonProcessingException; -import org.codehaus.jackson.Version; -import org.codehaus.jackson.map.DeserializationConfig; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.JsonDeserializer; -import org.codehaus.jackson.map.JsonSerializer; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.SerializationConfig; -import org.codehaus.jackson.map.SerializerProvider; -import org.codehaus.jackson.map.Serializers; -import org.codehaus.jackson.map.module.SimpleModule; -import org.codehaus.jackson.map.ser.std.ToStringSerializer; + + + + + + + + + + + + + + + import org.joda.time.DateTimeZone; +import org.skife.jdbi.v2.sqlobject.customizers.Mapper; import java.io.IOException; import java.nio.ByteOrder; @@ -153,9 +172,12 @@ public class DefaultObjectMapper extends ObjectMapper ); registerModule(serializerModule); - configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); - configure(SerializationConfig.Feature.AUTO_DETECT_GETTERS, false); - configure(SerializationConfig.Feature.AUTO_DETECT_FIELDS, false); - configure(SerializationConfig.Feature.INDENT_OUTPUT, false); + configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + configure(MapperFeature.AUTO_DETECT_GETTERS, false); + configure(MapperFeature.AUTO_DETECT_CREATORS, false); + configure(MapperFeature.AUTO_DETECT_FIELDS, false); + configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false); + configure(MapperFeature.AUTO_DETECT_SETTERS, false); + configure(SerializationFeature.INDENT_OUTPUT, false); } } diff --git a/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java b/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java index 50d39378390..f06bee8c76d 100644 --- a/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java +++ b/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java @@ -19,19 +19,21 @@ package com.metamx.druid.jackson; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonProcessingException; -import org.codehaus.jackson.JsonToken; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.KeyDeserializer; -import org.codehaus.jackson.map.deser.std.StdDeserializer; -import org.codehaus.jackson.map.ext.JodaDeserializers; -import org.codehaus.jackson.map.module.SimpleModule; -import org.codehaus.jackson.map.ser.std.ToStringSerializer; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.KeyDeserializer; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.fasterxml.jackson.datatype.joda.deser.DurationDeserializer; +import com.fasterxml.jackson.datatype.joda.deser.PeriodDeserializer; import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; +import org.joda.time.Duration; import org.joda.time.Interval; -import org.joda.time.ReadableInstant; +import org.joda.time.Period; import org.joda.time.format.ISODateTimeFormat; import java.io.IOException; @@ -47,6 +49,10 @@ public class JodaStuff module.addSerializer(DateTime.class, ToStringSerializer.instance); module.addDeserializer(Interval.class, new JodaStuff.IntervalDeserializer()); module.addSerializer(Interval.class, ToStringSerializer.instance); + module.addDeserializer(Period.class, new PeriodDeserializer()); + module.addSerializer(Period.class, ToStringSerializer.instance); + module.addDeserializer(Duration.class, new DurationDeserializer()); + module.addSerializer(Duration.class, ToStringSerializer.instance); return module; } diff --git a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java index de0a297c569..ec95cb6b586 100644 --- a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java +++ b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java @@ -19,10 +19,11 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Days; diff --git a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java index c280507e207..f221b0e26df 100644 --- a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java +++ b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java @@ -19,10 +19,11 @@ package com.metamx.druid.histogram; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; import com.metamx.druid.aggregation.Histogram; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java b/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java new file mode 100644 index 00000000000..aff4ad208c6 --- /dev/null +++ b/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java @@ -0,0 +1,40 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.jackson; + +import com.fasterxml.jackson.databind.ObjectMapper; +import junit.framework.Assert; +import org.joda.time.DateTime; +import org.junit.Test; + +/** + */ +public class DefaultObjectMapperTest +{ + ObjectMapper mapper = new DefaultObjectMapper(); + + @Test + public void testDateTime() throws Exception + { + final DateTime time = new DateTime(); + + Assert.assertEquals(String.format("\"%s\"", time), mapper.writeValueAsString(time)); + } +} diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index f431915bbac..592a713cf3a 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -1,5 +1,6 @@ - + 4.0.0 com.metamx.druid druid-examples-rand @@ -75,20 +76,20 @@ icu4j - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile org.jdbi diff --git a/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java b/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java index 66c071a79fe..520b0188171 100644 --- a/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java +++ b/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java @@ -1,5 +1,8 @@ package druid.examples; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.metamx.druid.guava.Runnables; @@ -7,9 +10,9 @@ import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; + + + import java.io.IOException; import java.util.HashMap; diff --git a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java index 4ffd7119431..ecdf2606a3b 100644 --- a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -1,5 +1,6 @@ package druid.examples; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; @@ -13,7 +14,7 @@ import com.metamx.druid.realtime.MetadataUpdaterConfig; import com.metamx.druid.realtime.RealtimeNode; import com.metamx.druid.loading.SegmentPusher; import com.metamx.phonebook.PhoneBook; -import org.codehaus.jackson.map.jsontype.NamedType; + import java.io.File; import java.io.IOException; diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 9ab850948d9..a96af1c08a0 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -1,5 +1,6 @@ - + 4.0.0 com.metamx.druid druid-examples-twitter @@ -75,20 +76,20 @@ icu4j - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile org.jdbi diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index fd072a427f3..e936d481489 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -1,5 +1,6 @@ package druid.examples; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; @@ -14,7 +15,7 @@ import com.metamx.druid.realtime.RealtimeNode; import com.metamx.druid.loading.SegmentPusher; import com.metamx.phonebook.PhoneBook; import druid.examples.twitter.TwitterSpritzerFirehoseFactory; -import org.codehaus.jackson.map.jsontype.NamedType; + import java.io.File; import java.io.IOException; diff --git a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index 992cd239487..1bef29dfc6c 100644 --- a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -1,14 +1,17 @@ package druid.examples.twitter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; + + + import twitter4j.ConnectionLifeCycleListener; import twitter4j.HashtagEntity; import twitter4j.Status; diff --git a/index-common/pom.xml b/index-common/pom.xml index 9d17c6b2c53..fee0b62da1f 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-index-common @@ -52,8 +53,8 @@ extendedset - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core net.java.dev.jets3t diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java b/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java index 2be3a50a595..69cb7ed628a 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java @@ -19,7 +19,8 @@ package com.metamx.druid.index.column; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java b/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java index 7f9b7ac7ee2..587c2455652 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java @@ -19,12 +19,14 @@ package com.metamx.druid.index.column; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.IAE; import com.metamx.druid.index.serde.ColumnPartSerde; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java index d60428fbe3e..d9e53011429 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java @@ -19,9 +19,11 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.index.column.ColumnBuilder; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java index 760fcbb3f20..0dc17bdb0dd 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java @@ -19,12 +19,14 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; import com.metamx.druid.index.v1.serde.ComplexMetrics; import com.metamx.druid.kv.GenericIndexed; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java index d30c6c6bbc0..5b542803c6c 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java @@ -19,6 +19,8 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.IAE; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.column.ValueType; @@ -27,8 +29,8 @@ import com.metamx.druid.kv.GenericIndexed; import com.metamx.druid.kv.VSizeIndexed; import com.metamx.druid.kv.VSizeIndexedInts; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java index 87cb95758db..2d0d5a0dc09 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java @@ -19,11 +19,13 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.column.ValueType; import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java index 0d140751e4c..8304062b05b 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java @@ -19,11 +19,13 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.column.ValueType; import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index 34a8e06962d..6d3badd4c35 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -19,6 +19,7 @@ package com.metamx.druid.index.v1; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -61,7 +62,7 @@ import com.metamx.druid.kv.VSizeIndexedInts; import com.metamx.druid.utils.SerializerUtils; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Interval; import java.io.ByteArrayOutputStream; diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java b/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java index 4a90f35a288..066d6bf47ed 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java @@ -19,7 +19,8 @@ package com.metamx.druid.index.v1.serde; -import org.codehaus.jackson.map.ObjectMapper; + +import com.fasterxml.jackson.databind.ObjectMapper; /** * This is a "factory" interface for registering handlers in the system. It exists because I'm unaware of diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java index 05613567b87..cf2a2d56d0d 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java @@ -19,11 +19,13 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.common.parsers.CSVParser; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java index 668dbc00fff..149a1946c42 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java index d372d46696f..177f0a2c358 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java @@ -19,11 +19,13 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.common.parsers.DelimitedParser; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java index 11228bc132f..74255c2b261 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java @@ -19,9 +19,10 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.parsers.JSONParser; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java b/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java index 3721f2c14f7..e04bd3d1d7c 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java @@ -19,6 +19,8 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -27,8 +29,8 @@ import com.metamx.common.parsers.Parser; import com.metamx.common.parsers.ToLowerCaseParser; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java index 4905f268ed4..0f52bae8bd2 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java @@ -19,10 +19,12 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.metamx.common.parsers.ParserUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import java.util.Map; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java index 2156314b4c6..2882d1a813a 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java @@ -19,9 +19,10 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.common.parsers.Parser; import com.metamx.common.parsers.ToLowerCaseParser; -import org.codehaus.jackson.annotate.JsonValue; + import java.util.List; diff --git a/indexer/pom.xml b/indexer/pom.xml index d38a3a17769..2af06cfbb0c 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-indexer @@ -79,12 +80,12 @@ - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind org.jdbi diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java index 720242466e4..a7dbee1f4e2 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.metamx.common.logger.Logger; @@ -30,7 +31,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index 13490e3ef00..d5bb8a4925a 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Joiner; @@ -63,7 +64,7 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.DateTimeComparator; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java index b020a99fa4e..f9749b79861 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java @@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableList; import com.metamx.common.Pair; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import java.util.List; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index 3d682dadce0..eed2339114b 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -19,6 +19,11 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; @@ -54,11 +59,11 @@ import com.metamx.druid.utils.JodaUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; -import org.codehaus.jackson.JsonGenerator; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + + + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.format.ISODateTimeFormat; @@ -97,24 +102,24 @@ public class HadoopDruidIndexerConfig public static HadoopDruidIndexerConfig fromMap(Map argSpec) { - if (argSpec.containsKey("registererers")) { - List registererers = Lists.transform( - MapUtils.getList(argSpec, "registererers"), - new Function() + List registererers = Lists.transform( + MapUtils.getList(argSpec, "registererers", ImmutableList.of()), + new Function() + { + @Override + public Registererer apply(@Nullable Object input) { - @Override - public Registererer apply(@Nullable Object input) - { - try { - return (Registererer) Class.forName((String) input).newInstance(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } + try { + return (Registererer) Class.forName((String) input).newInstance(); + } + catch (Exception e) { + throw Throwables.propagate(e); } } - ); + } + ); + if (!registererers.isEmpty()) { RegisteringNode.registerHandlers(registererers, Arrays.asList(jsonMapper)); } @@ -125,13 +130,7 @@ public class HadoopDruidIndexerConfig public static HadoopDruidIndexerConfig fromFile(File file) { try { - return fromMap( - (Map) jsonMapper.readValue( - file, new TypeReference>() - { - } - ) - ); + return fromMap((Map) jsonMapper.readValue(file, new TypeReference>(){})); } catch (IOException e) { throw Throwables.propagate(e); @@ -191,14 +190,14 @@ public class HadoopDruidIndexerConfig public HadoopDruidIndexerConfig( final @JsonProperty("intervals") List intervals, final @JsonProperty("dataSource") String dataSource, - final @JsonProperty("timestampColumnName") String timestampColumnName, + final @JsonProperty("timestampColumn") String timestampColumnName, final @JsonProperty("timestampFormat") String timestampFormat, final @JsonProperty("dataSpec") DataSpec dataSpec, final @JsonProperty("segmentGranularity") Granularity segmentGranularity, final @JsonProperty("granularitySpec") GranularitySpec granularitySpec, final @JsonProperty("pathSpec") PathSpec pathSpec, - final @JsonProperty("jobOutputDir") String jobOutputDir, - final @JsonProperty("segmentOutputDir") String segmentOutputDir, + final @JsonProperty("workingPath") String jobOutputDir, + final @JsonProperty("segmentOutputPath") String segmentOutputDir, final @JsonProperty("version") DateTime version, final @JsonProperty("partitionDimension") String partitionDimension, final @JsonProperty("targetPartitionSize") Long targetPartitionSize, @@ -221,7 +220,7 @@ public class HadoopDruidIndexerConfig this.pathSpec = pathSpec; this.jobOutputDir = jobOutputDir; this.segmentOutputDir = segmentOutputDir; - this.version = version; + this.version = version == null ? new DateTime() : version; this.partitionsSpec = partitionsSpec; this.leaveIntermediate = leaveIntermediate; this.cleanupOnFailure = cleanupOnFailure; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java index 15354d0be67..62441560573 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java @@ -19,11 +19,13 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import org.codehaus.jackson.map.jsontype.NamedType; + +import org.joda.time.DateTime; import org.joda.time.Interval; import java.io.File; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java index d7f40087ba8..c0d692c7c5f 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.shard.ShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** * ShardSpec + a shard ID that is unique across this run. The shard ID is used for grouping and partitioning. diff --git a/indexer/src/main/java/com/metamx/druid/indexer/Utils.java b/indexer/src/main/java/com/metamx/druid/indexer/Utils.java index aa1c3ed2f4b..97b046076e3 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/Utils.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/Utils.java @@ -19,6 +19,8 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; import com.metamx.common.ISE; @@ -30,8 +32,8 @@ import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.ReflectionUtils; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import java.io.IOException; import java.io.InputStream; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java b/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java index 151055271ec..e79af1e1df5 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java @@ -19,13 +19,15 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.collect.Iterators; import com.google.common.collect.PeekingIterator; import com.google.common.collect.Sets; import com.metamx.common.guava.Comparators; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java b/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java index 30ce2df1171..ee73f4fdb25 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Optional; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java b/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java index 51d2f37d437..22696a69c5d 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java @@ -19,6 +19,8 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; @@ -26,8 +28,8 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.Granularity; import com.metamx.common.guava.Comparators; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java index 2d00cf71f06..7988fb49c5b 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java @@ -1,7 +1,11 @@ package com.metamx.druid.indexer.partitions; -import org.codehaus.jackson.annotate.JsonIgnore; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; import javax.annotation.Nullable; @@ -14,6 +18,7 @@ public class PartitionsSpec private final boolean assumeGrouped; + @JsonCreator public PartitionsSpec( @JsonProperty("partitionDimension") @Nullable String partitionDimension, @JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize, diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java index 47e1676f848..3bcfb30c31d 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -32,7 +33,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; -import org.codehaus.jackson.annotate.JsonProperty; + import org.jets3t.service.model.S3Object; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java index aeca725bd1c..fb5b9e47ffe 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; import com.metamx.common.Granularity; import com.metamx.common.guava.Comparators; @@ -30,7 +31,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.format.DateTimeFormat; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java index 776f51c99bd..4a0b7278ad7 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java @@ -19,16 +19,18 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.indexer.HadoopDruidIndexerConfig; import org.apache.hadoop.mapreduce.Job; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.io.IOException; /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type") +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type") @JsonSubTypes(value={ @JsonSubTypes.Type(name="granular_unprocessed", value=GranularUnprocessedPathSpec.class), @JsonSubTypes.Type(name="granularity", value=GranularityPathSpec.class), diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java index f46135c32cc..9a244c2f74a 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java @@ -19,11 +19,12 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.logger.Logger; import com.metamx.druid.indexer.HadoopDruidIndexerConfig; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.codehaus.jackson.annotate.JsonProperty; + import java.io.IOException; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java index e2e1f9ce032..a223589db08 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java @@ -19,10 +19,11 @@ package com.metamx.druid.indexer.rollup; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.Granularity; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.List; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java index 9bf40d86657..7a8892e9160 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java @@ -19,8 +19,9 @@ package com.metamx.druid.indexer.updater; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.db.DbConnectorConfig; -import org.codehaus.jackson.annotate.JsonProperty; + /** */ diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java index 3523a198ba6..2957c1b3bf7 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java @@ -19,8 +19,9 @@ package com.metamx.druid.indexer.updater; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java index be1668afa93..1199e1ddf44 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java @@ -19,7 +19,8 @@ package com.metamx.druid.indexer.updater; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java index f4db1148327..5fdff8ce8b8 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -19,12 +19,13 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.druid.indexer.granularity.UniformGranularitySpec; import com.metamx.druid.indexer.partitions.PartitionsSpec; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -39,8 +40,8 @@ public class HadoopDruidIndexerConfigTest try { cfg = jsonMapper.readValue( - "{" - + "\"granularitySpec\":{" + "{" + + " \"granularitySpec\":{" + " \"type\":\"uniform\"," + " \"gran\":\"hour\"," + " \"intervals\":[\"2012-01-01/P1D\"]" diff --git a/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java b/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java index 0044d7d13e6..ecf6918b289 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java @@ -19,11 +19,12 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java b/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java index ab21be5f9f5..ea29d5ae727 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java @@ -19,12 +19,13 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.common.Granularity; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/merger/pom.xml b/merger/pom.xml index 980aa96d5f0..7f91b1e8b92 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-merger @@ -113,16 +114,16 @@ guice-servlet - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind javax.inject diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java index d9479a7dbbf..9abb60d063a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java @@ -19,10 +19,12 @@ package com.metamx.druid.merger.common; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskContext; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** */ diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java index 39b38a18a93..d6cb18093b8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java @@ -19,13 +19,15 @@ package com.metamx.druid.merger.common; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.task.Task; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Collections; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index 74a546cf696..0cebe1fc91c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.common; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.druid.loading.S3SegmentPuller; import com.metamx.druid.loading.S3SegmentGetterConfig; @@ -28,7 +29,7 @@ import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.loading.SegmentPusher; import com.metamx.emitter.service.ServiceEmitter; -import org.codehaus.jackson.map.ObjectMapper; + import org.jets3t.service.impl.rest.httpclient.RestS3Service; import java.io.File; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java index 28b78f78ad6..1ea40a95d2a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java @@ -19,6 +19,10 @@ package com.metamx.druid.merger.common.index; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; @@ -31,10 +35,10 @@ import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; import org.apache.commons.io.IOUtils; import org.apache.commons.io.LineIterator; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; -import org.codehaus.jackson.map.annotate.JacksonInject; + + + + import org.jets3t.service.S3Service; import org.jets3t.service.model.S3Object; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index 634fe65ebaf..7d456d29e01 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -19,6 +19,10 @@ package com.metamx.druid.merger.common.index; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -39,10 +43,10 @@ import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.PlumberSchool; import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.Sink; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; -import org.codehaus.jackson.map.annotate.JacksonInject; + + + + import org.joda.time.Interval; import java.io.File; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index 6de3eb0d73f..ae383f49f60 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -19,12 +19,14 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.coordinator.TaskContext; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; public abstract class AbstractTask implements Task diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java index f1153e5c43c..b3c90fa1330 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; @@ -33,8 +35,8 @@ import com.metamx.druid.index.v1.IndexableAdapter; import com.metamx.druid.index.v1.QueryableIndexIndexableAdapter; import com.metamx.druid.index.v1.Rowboat; import com.metamx.druid.index.v1.RowboatFilteringIndexAdapter; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import javax.annotation.Nullable; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java index e17db4b980e..4ba3ab38030 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -28,8 +30,8 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import javax.annotation.Nullable; import java.io.File; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 5f08298cd2d..5f37ad2853e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import com.metamx.druid.QueryGranularity; @@ -33,9 +35,9 @@ import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.ObjectMapper; + + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 55edfdbc3bc..110fdbf86f6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -38,8 +40,8 @@ import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import javax.annotation.Nullable; @@ -49,7 +51,8 @@ import java.util.Set; public class IndexDeterminePartitionsTask extends AbstractTask { - @JsonProperty private final FirehoseFactory firehoseFactory; + @JsonProperty + private final FirehoseFactory firehoseFactory; @JsonProperty private final Schema schema; @JsonProperty private final long targetPartitionSize; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index b89142ef19a..b3da8978bd3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; @@ -36,8 +38,8 @@ import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.Schema; import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.realtime.Sink; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 6074765ddbf..ca514cec52b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; @@ -31,8 +33,8 @@ import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.realtime.FirehoseFactory; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 20fefa0014f..21f6c1e6416 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -19,6 +19,9 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Objects; @@ -43,9 +46,9 @@ import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import org.apache.commons.codec.digest.DigestUtils; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 807172d11ae..60d1cf21e39 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -19,14 +19,16 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.merger.common.task.IndexDeterminePartitionsTask; import com.metamx.druid.merger.common.task.IndexGeneratorTask; import com.metamx.druid.merger.common.task.IndexTask; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java index 1a718fdaf20..32e8bfd71dd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java @@ -1,9 +1,10 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java index c1dbce4fc61..41032cc831f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; @@ -29,7 +30,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java index bc53ef0d4f7..f226a19be4f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; @@ -27,7 +28,7 @@ import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.db.DbConnectorConfig; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.skife.jdbi.v2.DBI; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index a95f64cb623..4157d6a66ca 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Predicate; @@ -48,7 +49,7 @@ import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; import com.netflix.curator.utils.ZKPaths; import org.apache.zookeeper.CreateMode; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java index fcbb2450cbc..d3a32aca462 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java @@ -19,9 +19,11 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Set; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index c6353698cbd..8203781bbf9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -27,7 +28,7 @@ import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.worker.Worker; import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java index 26f0105771d..5b342883bb2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java @@ -19,8 +19,9 @@ package com.metamx.druid.merger.coordinator.config; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.db.DbConnectorConfig; -import org.codehaus.jackson.annotate.JsonProperty; + import org.skife.config.Config; public abstract class IndexerDbConnectorConfig extends DbConnectorConfig diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 3dae4046764..7d075cbe9e8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -21,6 +21,8 @@ package com.metamx.druid.merger.coordinator.http; import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.services.ec2.AmazonEC2Client; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -89,8 +91,8 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; -import org.codehaus.jackson.map.InjectableValues; -import org.codehaus.jackson.map.ObjectMapper; + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java index 4cc1df9fa6f..1012beb830b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.coordinator.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; @@ -26,8 +28,8 @@ import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import org.codehaus.jackson.jaxrs.JacksonJsonProvider; -import org.codehaus.jackson.map.ObjectMapper; + + import javax.inject.Singleton; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index 8d51da61afd..16b629d9abb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -28,6 +28,7 @@ import com.amazonaws.services.ec2.model.Reservation; import com.amazonaws.services.ec2.model.RunInstancesRequest; import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; @@ -36,7 +37,7 @@ import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.EmittingLogger; import org.apache.commons.codec.binary.Base64; -import org.codehaus.jackson.map.ObjectMapper; + import javax.annotation.Nullable; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java index 8d302df25f6..4c2b86f4f6f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java @@ -19,8 +19,11 @@ package com.metamx.druid.merger.coordinator.setup; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java index 876a2635273..76061637312 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java @@ -19,8 +19,9 @@ package com.metamx.druid.merger.coordinator.setup; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java index 8395fa2d6c8..fada73cb40e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java @@ -19,8 +19,11 @@ package com.metamx.druid.merger.coordinator.setup; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java index 5e43e68ae66..89a0dd2d5c1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator.setup; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.common.ISE; @@ -28,7 +29,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; import org.apache.commons.collections.MapUtils; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Duration; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.FoldController; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index 5fc49788fcd..c41f4ac8be4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -32,7 +32,7 @@ import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.commons.io.FileUtils; -import org.codehaus.jackson.map.ObjectMapper; + import java.io.File; import java.util.concurrent.ExecutorService; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java b/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java index a1ebf273521..6b349de38f7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java @@ -19,9 +19,11 @@ package com.metamx.druid.merger.worker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.merger.worker.config.WorkerConfig; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** * A container for worker metadata. diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java index d4237da7a9e..ab3a35d53de 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.worker; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -32,7 +33,7 @@ import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.state.ConnectionState; import com.netflix.curator.framework.state.ConnectionStateListener; import org.apache.zookeeper.CreateMode; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import java.util.Arrays; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 0799a8de37c..237ae09d854 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.worker.http; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.concurrent.ScheduledExecutorFactory; @@ -59,8 +61,8 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; -import org.codehaus.jackson.map.InjectableValues; -import org.codehaus.jackson.map.ObjectMapper; + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index b7107ed72d2..ded6b55d64d 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -1,5 +1,10 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -29,10 +34,10 @@ import com.netflix.curator.retry.ExponentialBackoffRetry; import com.netflix.curator.test.TestingCluster; import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.zookeeper.CreateMode; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.jsontype.NamedType; + + + + import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -495,6 +500,7 @@ public class RemoteTaskRunnerTest private final List segments; private final List aggregators; + @JsonCreator public TestTask( @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, diff --git a/pom.xml b/pom.xml index aa77ea6894b..7b1e3f9e062 100644 --- a/pom.xml +++ b/pom.xml @@ -38,6 +38,7 @@ UTF-8 + 0.20.0 @@ -58,17 +59,17 @@ com.metamx emitter - 0.0.7 + 0.2.0 com.metamx http-client - 0.6.1 + 0.7.0 com.metamx java-util - 0.19.1 + ${metamx.java-util.version} com.metamx @@ -162,24 +163,34 @@ 4.8.1 - org.codehaus.jackson - jackson-core-asl - 1.9.9 + com.fasterxml.jackson.core + jackson-annotations + 2.1.2 - org.codehaus.jackson - jackson-jaxrs - 1.9.9 + com.fasterxml.jackson.core + jackson-core + 2.1.3 - org.codehaus.jackson - jackson-mapper-asl - 1.9.9 + com.fasterxml.jackson.core + jackson-databind + 2.1.3 - org.codehaus.jackson - jackson-smile - 1.9.9 + com.fasterxml.jackson.datatype + jackson-datatype-joda + 2.1.2 + + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + 2.1.3 + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + 2.1.3 javax.inject @@ -269,7 +280,7 @@ java-util test-jar test - 0.16.0 + ${metamx.java-util.version} diff --git a/realtime/pom.xml b/realtime/pom.xml index a8c50128f2b..26c23ad9133 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-realtime @@ -82,16 +83,16 @@ guava - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile org.jdbi diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java index b5709de1d6b..aab4509bbe5 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java @@ -19,8 +19,11 @@ package com.metamx.druid.realtime; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.io.IOException; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java index b2afbbc4884..d98997b5051 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java @@ -19,8 +19,9 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.Period; /** diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java index e090e088d08..40d0e662b49 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java @@ -19,8 +19,11 @@ package com.metamx.druid.realtime; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.io.IOException; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java index 05714ddfdbd..12c74ad6b16 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java @@ -19,6 +19,8 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import com.metamx.common.exception.FormattedException; @@ -30,8 +32,8 @@ import kafka.consumer.ConsumerConfig; import kafka.consumer.KafkaMessageStream; import kafka.javaapi.consumer.ConsumerConnector; import kafka.message.Message; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.CharBuffer; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java b/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java index 5bb10b36753..2d377124cc3 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java @@ -19,13 +19,14 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.phonebook.PhoneBook; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java index 0f0d87b3490..5fcc1f29f7d 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java @@ -19,8 +19,9 @@ package com.metamx.druid.realtime; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index f503e80ade3..d8fc7ebeea7 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -19,6 +19,12 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Maps; @@ -47,12 +53,12 @@ import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.Monitor; -import org.codehaus.jackson.map.BeanProperty; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.InjectableValues; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.smile.SmileFactory; -import org.codehaus.jackson.type.TypeReference; + + + + + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java index f4df5e054f8..47a4ef0f40f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -19,6 +19,11 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; @@ -53,11 +58,11 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import org.apache.commons.io.FileUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; -import org.codehaus.jackson.map.annotate.JacksonInject; + + + + + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java index 1b22f1e3a78..5a74b17e223 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java @@ -19,9 +19,10 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.loading.S3SegmentPusherConfig; import com.metamx.druid.loading.SegmentPusher; -import org.codehaus.jackson.map.ObjectMapper; + import org.jets3t.service.impl.rest.httpclient.RestS3Service; /** diff --git a/realtime/src/main/java/com/metamx/druid/realtime/Schema.java b/realtime/src/main/java/com/metamx/druid/realtime/Schema.java index b54c8efbbf6..ee3d63a14f8 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/Schema.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/Schema.java @@ -19,14 +19,16 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Arrays; diff --git a/server/pom.xml b/server/pom.xml index 544108cef5f..66309b73c30 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-server @@ -107,20 +108,20 @@ guice-servlet - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile javax.inject @@ -154,10 +155,10 @@ joda-time joda-time - - com.google.code.findbugs - jsr305 - + + com.google.code.findbugs + jsr305 + log4j log4j diff --git a/server/src/main/java/com/metamx/TsvToJson.java b/server/src/main/java/com/metamx/TsvToJson.java index 9d4417fe597..65effd5077e 100644 --- a/server/src/main/java/com/metamx/TsvToJson.java +++ b/server/src/main/java/com/metamx/TsvToJson.java @@ -19,11 +19,12 @@ package com.metamx; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.collect.Maps; import com.metamx.common.IAE; import com.metamx.common.ISE; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.DateTimeZone; diff --git a/server/src/main/java/com/metamx/druid/BaseServerNode.java b/server/src/main/java/com/metamx/druid/BaseServerNode.java index cdc7a68e4f6..dbbe286e270 100644 --- a/server/src/main/java/com/metamx/druid/BaseServerNode.java +++ b/server/src/main/java/com/metamx/druid/BaseServerNode.java @@ -19,6 +19,7 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; @@ -29,7 +30,7 @@ import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import org.codehaus.jackson.map.ObjectMapper; + import org.skife.config.ConfigurationObjectFactory; import java.nio.ByteBuffer; diff --git a/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java b/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java index 4e68ad2995e..a6d342d8419 100644 --- a/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java +++ b/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java @@ -19,13 +19,15 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="action") +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="action") @JsonSubTypes(value={ @JsonSubTypes.Type(name="load", value=SegmentChangeRequestLoad.class), @JsonSubTypes.Type(name="drop", value=SegmentChangeRequestDrop.class) diff --git a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java index 0093907d26e..b231781e394 100644 --- a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java +++ b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java @@ -19,10 +19,13 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonUnwrapped; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonUnwrapped; + + + /** */ diff --git a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java index c98106efdfb..ecf35513af9 100644 --- a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java @@ -19,10 +19,13 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonUnwrapped; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonUnwrapped; + + + /** */ diff --git a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java index 1fb756daafe..57d64e0ba32 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java +++ b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java @@ -19,6 +19,7 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; @@ -33,7 +34,7 @@ import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import java.io.File; diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index 2840eb037a8..e76b372978f 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -19,6 +19,8 @@ package com.metamx.druid.db; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -29,8 +31,8 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.master.rules.PeriodLoadRule; import com.metamx.druid.master.rules.Rule; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index ef620cd1e38..b17a2f63299 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -19,6 +19,7 @@ package com.metamx.druid.db; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -31,7 +32,7 @@ import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index c6d284403c0..2230932d9a1 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -46,8 +48,8 @@ import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.metrics.Monitor; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.smile.SmileFactory; + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index 51df502fa64..5ba559d4656 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -19,6 +19,7 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -67,7 +68,7 @@ import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.x.discovery.ServiceDiscovery; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.ZkClient; -import org.codehaus.jackson.map.ObjectMapper; + import org.mortbay.jetty.Server; import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.DefaultServlet; diff --git a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java index 1d33550635c..47395f73eeb 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java +++ b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.coordination.DruidClusterInfo; @@ -27,8 +29,8 @@ import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.master.DruidMaster; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import org.codehaus.jackson.jaxrs.JacksonJsonProvider; -import org.codehaus.jackson.map.ObjectMapper; + + import javax.inject.Singleton; diff --git a/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java b/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java index 3fdcd5dbdc5..a609329b0fd 100644 --- a/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java +++ b/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java @@ -19,8 +19,9 @@ package com.metamx.druid.http; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/server/src/main/java/com/metamx/druid/http/SegmentToMove.java b/server/src/main/java/com/metamx/druid/http/SegmentToMove.java index b35c3ce4d7c..d54a78b8bba 100644 --- a/server/src/main/java/com/metamx/druid/http/SegmentToMove.java +++ b/server/src/main/java/com/metamx/druid/http/SegmentToMove.java @@ -19,8 +19,9 @@ package com.metamx.druid.http; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java b/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java index f97d6dde379..2bf090850b6 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java @@ -19,10 +19,12 @@ package com.metamx.druid.index.brita; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Predicate; import com.metamx.druid.query.search.SearchQuerySpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java b/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java index 3069984739b..4628028ee0b 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java @@ -19,8 +19,9 @@ package com.metamx.druid.index.v1; +import com.fasterxml.jackson.annotation.JsonCreator; import com.metamx.druid.QueryGranularity; -import org.codehaus.jackson.annotate.JsonCreator; + import org.joda.time.DateTime; import org.joda.time.Days; import org.joda.time.Hours; diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java index 4d64c3ed464..5af4b905719 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java @@ -19,6 +19,7 @@ package com.metamx.druid.loading; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.common.io.Closeables; @@ -29,7 +30,7 @@ import com.metamx.druid.index.v1.IndexIO; import com.metamx.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; -import org.codehaus.jackson.map.ObjectMapper; + import org.jets3t.service.S3ServiceException; import org.jets3t.service.acl.gs.GSAccessControlList; import org.jets3t.service.impl.rest.httpclient.RestS3Service; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 3d9c8ecffe0..281c4cf5c3d 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -19,6 +19,7 @@ package com.metamx.druid.master; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; @@ -49,7 +50,7 @@ import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.exception.ZkNodeExistsException; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Duration; import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java index e7f3d5f41ed..f3d93d0e6fd 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java @@ -19,6 +19,7 @@ package com.metamx.druid.master; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -41,7 +42,7 @@ import com.metamx.http.client.HttpClientConfig; import com.metamx.http.client.HttpClientInit; import com.metamx.http.client.response.ToStringResponseHandler; import com.netflix.curator.x.discovery.ServiceProvider; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java b/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java index 32c77fdff3d..07a0f8dba6e 100644 --- a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java +++ b/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java @@ -19,13 +19,14 @@ package com.metamx.druid.master; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merge.ClientAppendQuery; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.HttpResponseHandler; import com.netflix.curator.x.discovery.ServiceProvider; -import org.codehaus.jackson.map.ObjectMapper; + import java.net.URL; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java index 0acdd8bc2f8..1e9bef477b5 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java @@ -19,9 +19,11 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java index 5aa984ccba8..bb50ef352fa 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java @@ -19,10 +19,12 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java index 152f074dc3a..22f1794189d 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java @@ -19,9 +19,11 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java index 051967e65ab..a332c302dd7 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java @@ -19,10 +19,12 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/master/rules/Rule.java b/server/src/main/java/com/metamx/druid/master/rules/Rule.java index a6eced93c68..3d2ea2f59c1 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/Rule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/Rule.java @@ -19,12 +19,14 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.client.DataSegment; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterRuntimeParams; import com.metamx.druid.master.MasterStats; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.DateTime; /** diff --git a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java index 62cb939e0ed..38a63d57f0c 100644 --- a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java @@ -19,13 +19,15 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java index 122b779e922..41ed201242c 100644 --- a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java @@ -19,13 +19,15 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index de148a9219c..762662741a6 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.collect.ImmutableList; @@ -60,7 +61,7 @@ import com.metamx.druid.result.SearchResultValue; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java index 5165647b24f..99019ef5568 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.util.concurrent.MoreExecutors; @@ -33,7 +34,7 @@ import com.metamx.druid.loading.NoopSegmentLoader; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.NoopQueryRunnerFactoryConglomerate; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.map.ObjectMapper; + import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Assert; From ee193f0ca7e8b9f7676b7388114470c5beb67135 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 13 Feb 2013 09:50:30 -0800 Subject: [PATCH 044/121] DeterminePartitionsJob: Select partition dimension to minimize segment size variance when cardinality is low --- .../druid/indexer/DeterminePartitionsJob.java | 36 ++++++++++++++++--- 1 file changed, 32 insertions(+), 4 deletions(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index d5bb8a4925a..4b4a5c8b995 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -500,6 +500,7 @@ public class DeterminePartitionsJob implements Jobby { private static final double SHARD_COMBINE_THRESHOLD = 0.25; private static final double SHARD_OVERSIZE_THRESHOLD = 1.5; + private static final int HIGH_CARDINALITY_THRESHOLD = 3000000; @Override protected void innerReduce( @@ -634,7 +635,9 @@ public class DeterminePartitionsJob implements Jobby final int totalRows = dimPartitionss.values().iterator().next().getRows(); - int maxCardinality = -1; + int maxCardinality = Integer.MIN_VALUE; + long minVariance = Long.MAX_VALUE; + DimPartitions minVariancePartitions = null; DimPartitions maxCardinalityPartitions = null; for(final DimPartitions dimPartitions : dimPartitionss.values()) { @@ -660,10 +663,18 @@ public class DeterminePartitionsJob implements Jobby continue; } - if(dimPartitions.getCardinality() > maxCardinality) { - maxCardinality = dimPartitions.getCardinality(); + final int cardinality = dimPartitions.getCardinality(); + final long variance = dimPartitions.getVariance(); + + if(cardinality > maxCardinality) { + maxCardinality = cardinality; maxCardinalityPartitions = dimPartitions; } + + if(variance < minVariance) { + minVariance = variance; + minVariancePartitions = dimPartitions; + } } if(maxCardinalityPartitions == null) { @@ -675,8 +686,12 @@ public class DeterminePartitionsJob implements Jobby context, config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)), config.isOverwriteFiles() ); + final DimPartitions chosenPartitions = maxCardinality > HIGH_CARDINALITY_THRESHOLD + ? maxCardinalityPartitions + : minVariancePartitions; + final List chosenShardSpecs = Lists.transform( - maxCardinalityPartitions.partitions, new Function() + chosenPartitions.partitions, new Function() { @Override public ShardSpec apply(DimPartition dimPartition) @@ -752,6 +767,19 @@ public class DeterminePartitionsJob implements Jobby return sum; } + public long getVariance() + { + final long meanRows = getRows() / partitions.size(); + + long variance = 0; + for(final DimPartition dimPartition : partitions) { + variance += (dimPartition.rows - meanRows) * (dimPartition.rows - meanRows); + } + + variance /= partitions.size(); + return variance; + } + public int getRows() { int sum = 0; From e3822f6ca795cb7fc8385b97147959d8b16ce582 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 13 Feb 2013 10:16:03 -0800 Subject: [PATCH 045/121] DeterminePartitionsJob: Fix docs --- .../com/metamx/druid/indexer/DeterminePartitionsJob.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index 4b4a5c8b995..d4ee1941396 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -79,13 +79,16 @@ import java.util.Set; /** * Determines appropriate ShardSpecs for a job by determining whether or not partitioning is necessary, and if so, - * choosing the highest cardinality dimension that satisfies the criteria: + * choosing the best dimension that satisfies the criteria: * *

    *
  • Must have exactly one value per row.
  • *
  • Must not generate oversized partitions. A dimension with N rows having the same value will necessarily * put all those rows in the same partition, and that partition may be much larger than the target size.
  • *
+ * + * "Best" means a very high cardinality dimension, or, if none exist, the dimension that minimizes segment size + * variance. */ public class DeterminePartitionsJob implements Jobby { From 1b85eaad3e39aea91461b0406a6a70b112ef09dd Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 8 Feb 2013 16:50:23 -0800 Subject: [PATCH 046/121] simplify query toolchest interface --- .../metamx/druid/query/QueryToolChest.java | 12 ++---- .../TimeBoundaryQueryQueryToolChest.java | 39 +++++-------------- .../java/com/metamx/druid/LogicalSegment.java | 8 ++++ .../metamx/druid/TimelineObjectHolder.java | 3 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- 6 files changed, 25 insertions(+), 41 deletions(-) create mode 100644 common/src/main/java/com/metamx/druid/LogicalSegment.java diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index f729016032e..e2d227729d3 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -22,12 +22,9 @@ package com.metamx.druid.query; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; +import com.metamx.druid.LogicalSegment; import com.metamx.druid.Query; -import com.metamx.druid.TimelineObjectHolder; -import com.metamx.druid.client.selector.ServerSelector; import com.metamx.emitter.service.ServiceMetricEvent; - - import java.util.List; /** @@ -50,7 +47,7 @@ public abstract class QueryToolChest makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); public abstract TypeReference getResultTypeReference(); - public CacheStrategy getCacheStrategy(QueryType query) { + public CacheStrategy getCacheStrategy(QueryType query) { return null; } @@ -62,10 +59,7 @@ public abstract class QueryToolChest> filterSegments( - QueryType query, - List> segments - ) { + public List filterSegments(QueryType query, List segments) { return segments; } } diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 86e618bcee8..4ff1ffdf533 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -22,16 +22,14 @@ package com.metamx.druid.query.timeboundary; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Ordering; -import com.google.common.collect.Sets; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import com.metamx.druid.LogicalSegment; import com.metamx.druid.Query; -import com.metamx.druid.TimelineObjectHolder; -import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.collect.OrderedMergeSequence; import com.metamx.druid.query.BySegmentSkippingQueryRunner; import com.metamx.druid.query.CacheStrategy; @@ -47,8 +45,6 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.List; -import java.util.Map; -import java.util.Set; /** */ @@ -64,32 +60,17 @@ public class TimeBoundaryQueryQueryToolChest { }; - @Override - public List> filterSegments( - TimeBoundaryQuery query, - List> input - ) { - long minMillis = Long.MAX_VALUE; - long maxMillis = Long.MIN_VALUE; - TimelineObjectHolder min = null; - TimelineObjectHolder max = null; - - for(TimelineObjectHolder e : input) { - final long start = e.getInterval().getStartMillis(); - final long end = e.getInterval().getEndMillis(); - - if(min == null || start < minMillis) { - min = e; - minMillis = start; - } - if(max == null || end > maxMillis) { - max = e; - maxMillis = end; - } + public List filterSegments(TimeBoundaryQuery query, List input) + { + if(input.size() <= 1) { + return input; } - return min == max ? Lists.newArrayList(min) : Lists.newArrayList(min , max); + return Lists.newArrayList( + Iterables.getFirst(input, null), + Iterables.getLast(input) + ); } @Override diff --git a/common/src/main/java/com/metamx/druid/LogicalSegment.java b/common/src/main/java/com/metamx/druid/LogicalSegment.java new file mode 100644 index 00000000000..5550fb44966 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/LogicalSegment.java @@ -0,0 +1,8 @@ +package com.metamx.druid; + +import org.joda.time.Interval; + +public interface LogicalSegment +{ + public Interval getInterval(); +} diff --git a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java index 403fe8bb2cb..d93e5ed8f6b 100644 --- a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java +++ b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java @@ -24,7 +24,7 @@ import org.joda.time.Interval; /** */ -public class TimelineObjectHolder +public class TimelineObjectHolder implements LogicalSegment { private final Interval interval; private final VersionType version; @@ -41,6 +41,7 @@ public class TimelineObjectHolder this.object = object; } + @Override public Interval getInterval() { return interval; diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 592a713cf3a..c496170be99 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -10,7 +10,7 @@ com.metamx druid-examples - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a96af1c08a0..a327c94881d 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -10,7 +10,7 @@ com.metamx druid-examples - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT From a616f861283a780c1db6d14ebbe04f7be457bb67 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Thu, 14 Feb 2013 09:09:22 -0800 Subject: [PATCH 047/121] add DateTime import back --- server/src/main/java/com/metamx/druid/master/DruidMaster.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 6c4a386f05c..3894606a7ff 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -51,6 +51,7 @@ import com.metamx.phonebook.PhoneBookPeon; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.exception.ZkNodeExistsException; +import org.joda.time.DateTime; import org.joda.time.Duration; import javax.annotation.Nullable; From 6998d604a293f752af8649c7f12b31f2801decaa Mon Sep 17 00:00:00 2001 From: xvrl Date: Thu, 14 Feb 2013 12:41:34 -0800 Subject: [PATCH 048/121] fixes #80 --- .../query/timeboundary/TimeBoundaryQueryQueryToolChest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 4ff1ffdf533..9701d9eee76 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -41,7 +41,6 @@ import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; import org.joda.time.DateTime; - import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.List; From ee4236dac2e5cbec65ef65f9c6ba365637ee9561 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 14 Feb 2013 17:10:27 -0600 Subject: [PATCH 049/121] 1) Update to mmx build of jackson 2.1 branch to get bug-fix for jackson-databind #167 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a1f1dd515ff..dc4e48422c8 100644 --- a/pom.xml +++ b/pom.xml @@ -175,7 +175,7 @@ com.fasterxml.jackson.core jackson-databind - 2.1.3 + 2.1.4-mmx-2 com.fasterxml.jackson.datatype From 4f11eb5209a98e909fc3aa0b6b08cc9ed6463c32 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 14 Feb 2013 18:24:51 -0600 Subject: [PATCH 050/121] 1) Fix alert sent from line DruidMasterBalancer:71 to not include all of the segments that a server is holding --- .../main/java/com/metamx/druid/master/DruidMasterBalancer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 4ad3a839b4b..c3853af91f0 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -70,7 +70,7 @@ public class DruidMasterBalancer implements DruidMasterHelper if (holder.getLifetime() <= 0) { log.makeAlert("[%s]: Balancer move segments queue has a segment stuck", tier) .addData("segment", holder.getSegment().getIdentifier()) - .addData("server", holder.getServer()) + .addData("server", holder.getServer().getStringProps()) .emit(); } } From e56bc27f93fc355782713a51b3eafe2af3d0bef8 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 16:27:12 -0800 Subject: [PATCH 051/121] [maven-release-plugin] prepare release druid-0.2.8 --- client/pom.xml | 5 ++--- common/pom.xml | 5 ++--- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 5 ++--- examples/twitter/pom.xml | 5 ++--- index-common/pom.xml | 5 ++--- indexer/pom.xml | 5 ++--- merger/pom.xml | 5 ++--- pom.xml | 2 +- realtime/pom.xml | 5 ++--- server/pom.xml | 5 ++--- 12 files changed, 22 insertions(+), 31 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 588da3fe4de..117b634c7a8 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-client @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/common/pom.xml b/common/pom.xml index 369aa402c4d..7c29586abb0 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-common @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 9b2567dc753..4aa4a010a02 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.8-SNAPSHOT + 0.2.8 com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/examples/pom.xml b/examples/pom.xml index fa195a3f0f4..356773392f2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 592a713cf3a..21a6f40303c 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 com.metamx.druid druid-examples-rand @@ -10,7 +9,7 @@ com.metamx druid-examples - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a96af1c08a0..7f3ee6e940c 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 com.metamx.druid druid-examples-twitter @@ -10,7 +9,7 @@ com.metamx druid-examples - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/index-common/pom.xml b/index-common/pom.xml index 084bc314ba1..77641cc61c2 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-index-common @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/indexer/pom.xml b/indexer/pom.xml index 5ff19d88607..75bf11e4f0b 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-indexer @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/merger/pom.xml b/merger/pom.xml index da776d6e512..26d410f3059 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-merger @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/pom.xml b/pom.xml index dc4e48422c8..95214118926 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.8-SNAPSHOT + 0.2.8 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index c321afa695d..8e7744e01f7 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-realtime @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/server/pom.xml b/server/pom.xml index 9b4a0707961..2478372e2dd 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-server @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 From 152bf201ec35b56b3334e058330ed763adbbf260 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 16:27:20 -0800 Subject: [PATCH 052/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 117b634c7a8..8329d3bb8a4 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 7c29586abb0..978164b92e5 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 4aa4a010a02..f52cc4d954b 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.8 + 0.2.9-SNAPSHOT com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 356773392f2..aa8cb80def3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 21a6f40303c..978f04351df 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 7f3ee6e940c..924c439aa09 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 77641cc61c2..4b19856be9c 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 75bf11e4f0b..bb4a90b692f 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 26d410f3059..93b979a1460 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/pom.xml b/pom.xml index 95214118926..e0e1f947cd2 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.8 + 0.2.9-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 8e7744e01f7..dd267cee537 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 2478372e2dd..4210d090b5a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT From 48538d045409f3d996bc06fe3a191ca7ccfb05af Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 14 Feb 2013 22:59:17 -0600 Subject: [PATCH 053/121] 1) Fix bug in index converter when column has cardinality 0 --- .../com/metamx/druid/index/v1/IndexIO.java | 59 +++++++++++-------- 1 file changed, 34 insertions(+), 25 deletions(-) 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 6d3badd4c35..5a2c7307ddd 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 @@ -370,6 +370,7 @@ public class IndexIO } LinkedHashSet skippedFiles = Sets.newLinkedHashSet(); + Set skippedDimensions = Sets.newLinkedHashSet(); for (String filename : v8SmooshedFiles.getInternalFilenames()) { log.info("Processing file[%s]", filename); if (filename.startsWith("dim_")) { @@ -392,6 +393,12 @@ public class IndexIO dimBuffer, GenericIndexed.stringStrategy ); + if (dictionary.size() == 0) { + log.info("Dimension[%s] had cardinality 0, equivalent to no column, so skipping.", dimension); + skippedDimensions.add(dimension); + continue; + } + VSizeIndexedInts singleValCol = null; VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer()); GenericIndexed bitmaps = bitmapIndexes.get(dimension); @@ -555,35 +562,37 @@ public class IndexIO channel.write(ByteBuffer.wrap(specBytes)); serdeficator.write(channel); channel.close(); - } else if ("index.drd".equals(filename)) { - final ByteBuffer indexBuffer = v8SmooshedFiles.mapFile(filename); - - indexBuffer.get(); // Skip the version byte - final GenericIndexed dims = GenericIndexed.read( - indexBuffer, GenericIndexed.stringStrategy - ); - final GenericIndexed availableMetrics = GenericIndexed.read( - indexBuffer, GenericIndexed.stringStrategy - ); - final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); - - Set columns = Sets.newTreeSet(); - columns.addAll(Lists.newArrayList(dims)); - columns.addAll(Lists.newArrayList(availableMetrics)); - - GenericIndexed cols = GenericIndexed.fromIterable(columns, GenericIndexed.stringStrategy); - - final int numBytes = cols.getSerializedSize() + dims.getSerializedSize() + 16; - final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); - cols.writeToChannel(writer); - dims.writeToChannel(writer); - serializerUtils.writeLong(writer, dataInterval.getStartMillis()); - serializerUtils.writeLong(writer, dataInterval.getEndMillis()); - writer.close(); } else { skippedFiles.add(filename); } } + + final ByteBuffer indexBuffer = v8SmooshedFiles.mapFile("index.drd"); + + indexBuffer.get(); // Skip the version byte + final GenericIndexed dims = GenericIndexed.read( + indexBuffer, GenericIndexed.stringStrategy + ); + final GenericIndexed availableMetrics = GenericIndexed.read( + indexBuffer, GenericIndexed.stringStrategy + ); + final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); + + Set columns = Sets.newTreeSet(); + columns.addAll(Lists.newArrayList(dims)); + columns.addAll(Lists.newArrayList(availableMetrics)); + columns.removeAll(skippedDimensions); + + GenericIndexed cols = GenericIndexed.fromIterable(columns, GenericIndexed.stringStrategy); + + final int numBytes = cols.getSerializedSize() + dims.getSerializedSize() + 16; + final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); + cols.writeToChannel(writer); + dims.writeToChannel(writer); + serializerUtils.writeLong(writer, dataInterval.getStartMillis()); + serializerUtils.writeLong(writer, dataInterval.getEndMillis()); + writer.close(); + log.info("Skipped files[%s]", skippedFiles); v9Smoosher.close(); From 22d1f2a0c09bed8b77983ed9aa271fbd05070857 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 21:07:17 -0800 Subject: [PATCH 054/121] IndexMergerTest: Empty column test --- .../druid/index/v1/IndexMergerTest.java | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java index 097762106a2..86d03f6dd76 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.index.v1; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.io.Files; @@ -111,4 +112,33 @@ public class IndexMergerTest FileUtils.deleteQuietly(mergedDir); } } + + @Test + public void testPersistEmptyColumn() throws Exception + { + final IncrementalIndex toPersist = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); + final File tmpDir = Files.createTempDir(); + + try { + toPersist.add( + new MapBasedInputRow( + 1L, + ImmutableList.of("dim1", "dim2"), + ImmutableMap.of("dim1", ImmutableList.of(), "dim2", "foo") + ) + ); + + final QueryableIndex merged = IndexIO.loadIndex( + IndexMerger.persist(toPersist, tmpDir) + ); + + Assert.assertEquals(1, merged.getTimeColumn().getLength()); + Assert.assertEquals(ImmutableList.of("dim1", "dim2"), ImmutableList.copyOf(merged.getAvailableDimensions())); + Assert.assertEquals(null, merged.getColumn("dim1")); + } finally { + FileUtils.deleteQuietly(tmpDir); + } + + + } } From ba7df5b7092dbc85a687c3036091d872e8eb1dce Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 21:23:41 -0800 Subject: [PATCH 055/121] [maven-release-plugin] prepare release druid-0.2.9 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 8329d3bb8a4..a5689080ff3 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/common/pom.xml b/common/pom.xml index 978164b92e5..f906565c5fb 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index f52cc4d954b..08d00318d79 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.9-SNAPSHOT + 0.2.9 com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/examples/pom.xml b/examples/pom.xml index aa8cb80def3..0c9853282a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 978f04351df..5b97b7088d9 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 924c439aa09..771c0e72e5c 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/index-common/pom.xml b/index-common/pom.xml index 4b19856be9c..f6879ba8336 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/indexer/pom.xml b/indexer/pom.xml index bb4a90b692f..485cde8a6bb 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/merger/pom.xml b/merger/pom.xml index 93b979a1460..8378522c36c 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/pom.xml b/pom.xml index e0e1f947cd2..a2fb097fd71 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.9-SNAPSHOT + 0.2.9 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index dd267cee537..042bcfc26f0 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/server/pom.xml b/server/pom.xml index 4210d090b5a..af0009bbd9d 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 From 3fed8ee0cee1c9e164731e067b1f800f2afcf7d4 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 21:23:46 -0800 Subject: [PATCH 056/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index a5689080ff3..302778496bc 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index f906565c5fb..f2f878a6da1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 08d00318d79..90716448ca1 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.9 + 0.2.10-SNAPSHOT com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 0c9853282a7..bfbdeb2f735 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 5b97b7088d9..20bf45f8cae 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 771c0e72e5c..42be38a8916 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index f6879ba8336..49db8c7b958 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 485cde8a6bb..ee05ae99737 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 8378522c36c..cc230b8e921 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/pom.xml b/pom.xml index a2fb097fd71..3166c2e845b 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.9 + 0.2.10-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 042bcfc26f0..0c3e8ab773c 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index af0009bbd9d..8e715da4e8e 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT From 548c901c0b21a21fe5f93c115a8750e00b39cebf Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 23:34:49 -0800 Subject: [PATCH 057/121] Additional fix for columns with cardinality 0 --- .../com/metamx/druid/index/v1/IndexIO.java | 28 +++++++++---- .../druid/index/v1/IndexMergerTest.java | 39 ++++++++++++++----- 2 files changed, 49 insertions(+), 18 deletions(-) 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 5a2c7307ddd..d26e73f5b3c 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 @@ -21,6 +21,7 @@ package com.metamx.druid.index.v1; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -62,7 +63,6 @@ import com.metamx.druid.kv.VSizeIndexedInts; import com.metamx.druid.utils.SerializerUtils; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; - import org.joda.time.Interval; import java.io.ByteArrayOutputStream; @@ -369,8 +369,8 @@ public class IndexIO ); } - LinkedHashSet skippedFiles = Sets.newLinkedHashSet(); - Set skippedDimensions = Sets.newLinkedHashSet(); + final LinkedHashSet skippedFiles = Sets.newLinkedHashSet(); + final Set skippedDimensions = Sets.newLinkedHashSet(); for (String filename : v8SmooshedFiles.getInternalFilenames()) { log.info("Processing file[%s]", filename); if (filename.startsWith("dim_")) { @@ -570,25 +570,37 @@ public class IndexIO final ByteBuffer indexBuffer = v8SmooshedFiles.mapFile("index.drd"); indexBuffer.get(); // Skip the version byte - final GenericIndexed dims = GenericIndexed.read( + final GenericIndexed dims8 = GenericIndexed.read( indexBuffer, GenericIndexed.stringStrategy ); + final GenericIndexed dims9 = GenericIndexed.fromIterable( + Iterables.filter( + dims8, new Predicate() + { + @Override + public boolean apply(String s) + { + return !skippedDimensions.contains(s); + } + } + ), + GenericIndexed.stringStrategy + ); final GenericIndexed availableMetrics = GenericIndexed.read( indexBuffer, GenericIndexed.stringStrategy ); final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); Set columns = Sets.newTreeSet(); - columns.addAll(Lists.newArrayList(dims)); + columns.addAll(Lists.newArrayList(dims9)); columns.addAll(Lists.newArrayList(availableMetrics)); - columns.removeAll(skippedDimensions); GenericIndexed cols = GenericIndexed.fromIterable(columns, GenericIndexed.stringStrategy); - final int numBytes = cols.getSerializedSize() + dims.getSerializedSize() + 16; + final int numBytes = cols.getSerializedSize() + dims9.getSerializedSize() + 16; final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); cols.writeToChannel(writer); - dims.writeToChannel(writer); + dims9.writeToChannel(writer); serializerUtils.writeLong(writer, dataInterval.getStartMillis()); serializerUtils.writeLong(writer, dataInterval.getEndMillis()); writer.close(); diff --git a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java index 86d03f6dd76..407cd36253f 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java @@ -33,6 +33,7 @@ import org.junit.Test; import java.io.File; import java.util.Arrays; +import java.util.List; /** */ @@ -116,11 +117,14 @@ public class IndexMergerTest @Test public void testPersistEmptyColumn() throws Exception { - final IncrementalIndex toPersist = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); - final File tmpDir = Files.createTempDir(); + final IncrementalIndex toPersist1 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); + final IncrementalIndex toPersist2 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); + final File tmpDir1 = Files.createTempDir(); + final File tmpDir2 = Files.createTempDir(); + final File tmpDir3 = Files.createTempDir(); try { - toPersist.add( + toPersist1.add( new MapBasedInputRow( 1L, ImmutableList.of("dim1", "dim2"), @@ -128,17 +132,32 @@ public class IndexMergerTest ) ); - final QueryableIndex merged = IndexIO.loadIndex( - IndexMerger.persist(toPersist, tmpDir) + toPersist2.add( + new MapBasedInputRow( + 1L, + ImmutableList.of("dim1", "dim2"), + ImmutableMap.of("dim1", ImmutableList.of(), "dim2", "bar") + ) ); + final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1)); + final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2)); + final QueryableIndex merged = IndexIO.loadIndex( + IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3) + ); + + Assert.assertEquals(1, index1.getTimeColumn().getLength()); + Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(index1.getAvailableDimensions())); + + Assert.assertEquals(1, index2.getTimeColumn().getLength()); + Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(index2.getAvailableDimensions())); + Assert.assertEquals(1, merged.getTimeColumn().getLength()); - Assert.assertEquals(ImmutableList.of("dim1", "dim2"), ImmutableList.copyOf(merged.getAvailableDimensions())); - Assert.assertEquals(null, merged.getColumn("dim1")); + Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(merged.getAvailableDimensions())); } finally { - FileUtils.deleteQuietly(tmpDir); + FileUtils.deleteQuietly(tmpDir1); + FileUtils.deleteQuietly(tmpDir2); + FileUtils.deleteQuietly(tmpDir3); } - - } } From 5ee7c0c264c99c4d704231429455867c16094117 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 15 Feb 2013 12:15:53 -0600 Subject: [PATCH 058/121] 1) Remove stray import of java.util.List --- .../src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java index 407cd36253f..9cfe9d22379 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java @@ -33,7 +33,6 @@ import org.junit.Test; import java.io.File; import java.util.Arrays; -import java.util.List; /** */ From 780410446c019f748e23d586496c3267932c8126 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 15 Feb 2013 11:19:13 -0800 Subject: [PATCH 059/121] even more simple --- .../query/timeboundary/TimeBoundaryQueryQueryToolChest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 9701d9eee76..e8735522f89 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -22,7 +22,6 @@ package com.metamx.druid.query.timeboundary; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.metamx.common.guava.MergeSequence; @@ -67,8 +66,8 @@ public class TimeBoundaryQueryQueryToolChest } return Lists.newArrayList( - Iterables.getFirst(input, null), - Iterables.getLast(input) + input.get(0), + input.get(input.size() - 1) ); } From e042c29173c0c139dfa18e5dd44b45e20bc67acb Mon Sep 17 00:00:00 2001 From: Deep Ganguli Date: Fri, 15 Feb 2013 11:39:49 -0800 Subject: [PATCH 060/121] Fixed typos in comments, changed prefix on s3Paths to s3n from s3://, cleaned up umbrellaIntervals code in JodaUtils, modified the DbUpdater to push segment meta-data to the database in batches. --- .../com/metamx/druid/utils/JodaUtils.java | 25 ++----- .../metamx/druid/indexer/DbUpdaterJob.java | 66 +++++++++++-------- .../merger/common/task/HadoopIndexTask.java | 4 +- 3 files changed, 46 insertions(+), 49 deletions(-) diff --git a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java index 3932ef08153..3de006d6efd 100644 --- a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java +++ b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java @@ -19,7 +19,6 @@ package com.metamx.druid.utils; -import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.guava.Comparators; @@ -66,27 +65,17 @@ public class JodaUtils public static Interval umbrellaInterval(Iterable intervals) { - DateTime minStart = null; - DateTime maxEnd = null; + ArrayList startDates = Lists.newArrayList(); + ArrayList endDates = Lists.newArrayList(); for (Interval interval : intervals) { - DateTime curStart = interval.getStart(); - DateTime curEnd = interval.getEnd(); - - if (minStart == null || maxEnd == null) { - minStart = curStart; - maxEnd = curEnd; - } - - if (curStart.isBefore(minStart)) { - minStart = curStart; - } - - if (curEnd.isAfter(maxEnd)) { - maxEnd = curEnd; - } + startDates.add(interval.getStart()); + endDates.add(interval.getEnd()); } + DateTime minStart = minDateTime(startDates.toArray(new DateTime[]{})); + DateTime maxEnd = maxDateTime(endDates.toArray(new DateTime[]{})); + if (minStart == null || maxEnd == null) { throw new IllegalArgumentException("Empty list of intervals"); } diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java index 530042d289c..45021618d76 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java @@ -20,6 +20,7 @@ package com.metamx.druid.indexer; +import com.google.common.collect.ImmutableMap; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.db.DbConnector; @@ -29,6 +30,7 @@ import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.PreparedBatch; import org.skife.jdbi.v2.tweak.HandleCallback; import java.util.List; @@ -59,39 +61,45 @@ public class DbUpdaterJob implements Jobby { final List segments = IndexGeneratorJob.getPublishedSegments(config); - for (final DataSegment segment : segments) { - - dbi.withHandle( - new HandleCallback() + dbi.withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception { - @Override - public Void withHandle(Handle handle) throws Exception - { - handle.createStatement( - String.format( - "INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - spec.getSegmentTable() - ) - ) - .bind("id", segment.getIdentifier()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", new DateTime().toString()) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", segment.getShardSpec().getPartitionNum()) - .bind("version", segment.getVersion()) - .bind("used", true) - .bind("payload", jsonMapper.writeValueAsString(segment)) - .execute(); + final PreparedBatch batch = handle.prepareBatch( + String.format( + "INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + spec.getSegmentTable() + ) + ); + for (final DataSegment segment : segments) { + + batch.add( + new ImmutableMap.Builder() + .put("id", segment.getIdentifier()) + .put("dataSource", segment.getDataSource()) + .put("created_date", new DateTime().toString()) + .put("start", segment.getInterval().getStart().toString()) + .put("end", segment.getInterval().getEnd().toString()) + .put("partitioned", segment.getShardSpec().getPartitionNum()) + .put("version", segment.getVersion()) + .put("used", true) + .put("payload", jsonMapper.writeValueAsString(segment)) + .build() + ); + + log.info("Published %s", segment.getIdentifier()); - return null; } - } - ); + batch.execute(); + + return null; + } + } + ); - log.info("Published %s", segment.getIdentifier()); - } return true; } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java index ec80157600c..60165173f79 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java @@ -28,7 +28,7 @@ public class HadoopIndexTask extends AbstractTask * @param config is used by the HadoopDruidIndexerJob to set up the appropriate parameters * for creating Druid index segments. It may be modified. *

- * Here, we will ensure that the UpDaterJobSpec field of the config is set to null, such that the + * Here, we will ensure that the UpdaterJobSpec field of the config is set to null, such that the * job does not push a list of published segments the database. Instead, we will use the method * IndexGeneratorJob.getPublishedSegments() to simply return a list of the published * segments, and let the indexing service report these segments to the database. @@ -67,7 +67,7 @@ public class HadoopIndexTask extends AbstractTask // Hack alert! Bypassing SegmentPusher... S3SegmentPusher segmentPusher = (S3SegmentPusher) toolbox.getSegmentPusher(); String s3Path = String.format( - "s3://%s/%s/%s", + "s3n%s/%s/%s", segmentPusher.getConfig().getBucket(), segmentPusher.getConfig().getBaseKey(), getDataSource() From 0d99cee3c3fff7802009d4a3609acd10a07102e7 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 15 Feb 2013 14:25:44 -0600 Subject: [PATCH 061/121] 1) Whitespace ftw! --- .../timeboundary/TimeBoundaryQueryQueryToolChest.java | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index e8735522f89..dcf09526a31 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -38,8 +38,8 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; - import org.joda.time.DateTime; + import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.List; @@ -65,10 +65,7 @@ public class TimeBoundaryQueryQueryToolChest return input; } - return Lists.newArrayList( - input.get(0), - input.get(input.size() - 1) - ); + return Lists.newArrayList(input.get(0), input.get(input.size() - 1)); } @Override From 6bbc992101d37b5ddd225d5bd05b5cf763b821a0 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 15 Feb 2013 13:03:32 -0800 Subject: [PATCH 062/121] [maven-release-plugin] prepare release druid-0.3.0 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 5 ++--- examples/twitter/pom.xml | 5 ++--- index-common/pom.xml | 5 ++--- indexer/pom.xml | 5 ++--- merger/pom.xml | 5 ++--- pom.xml | 2 +- realtime/pom.xml | 5 ++--- server/pom.xml | 2 +- 12 files changed, 19 insertions(+), 25 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 602d4b9cbb7..c82151f4498 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/common/pom.xml b/common/pom.xml index 0cb9bf4a429..78562c476cd 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 88994a529d5..e9ccc47ec39 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.0-SNAPSHOT + 0.3.0 com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/examples/pom.xml b/examples/pom.xml index b24a0f7dd42..dc880f384e2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index c496170be99..afb3e13decb 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 com.metamx.druid druid-examples-rand @@ -10,7 +9,7 @@ com.metamx druid-examples - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a327c94881d..7e04b88abf1 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 com.metamx.druid druid-examples-twitter @@ -10,7 +9,7 @@ com.metamx druid-examples - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/index-common/pom.xml b/index-common/pom.xml index fee0b62da1f..67c71003775 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-index-common @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/indexer/pom.xml b/indexer/pom.xml index 2af06cfbb0c..e587f76f5d3 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-indexer @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/merger/pom.xml b/merger/pom.xml index 7f91b1e8b92..1d007233120 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-merger @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/pom.xml b/pom.xml index 0e2e7f1b0c0..85ddb331552 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.0-SNAPSHOT + 0.3.0 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 26c23ad9133..215d3bf0cbe 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-realtime @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/server/pom.xml b/server/pom.xml index 284c76ff474..049318700ef 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 From 824e3c0eb2956b619028df4c2f8ff287f170fc1c Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 15 Feb 2013 13:03:39 -0800 Subject: [PATCH 063/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index c82151f4498..59cc716195c 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 78562c476cd..710c88d7274 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index e9ccc47ec39..01078c607f4 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.0 + 0.3.1-SNAPSHOT com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index dc880f384e2..4220ed243bc 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index afb3e13decb..ae5e1e767e9 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 7e04b88abf1..c557c46800b 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 67c71003775..c67359cb14e 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index e587f76f5d3..0a32246ce06 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 1d007233120..e1c33c4b13b 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/pom.xml b/pom.xml index 85ddb331552..b9b34479851 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.0 + 0.3.1-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 215d3bf0cbe..19553828205 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 049318700ef..09c44aea4b0 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT From d1626576c069cc9ee84c113b0bba04ed25c91207 Mon Sep 17 00:00:00 2001 From: James Estes Date: Sat, 9 Feb 2013 22:33:00 -0700 Subject: [PATCH 064/121] Working toward making it easier to add new SegmentPullers. 1) Move the local cacheFile logic out of the S3 pullers into the SingleSegmentLoader 2) Make the S3SegmentPuller just pull down the file 3) Make the Loader do the unzip, ungzip, or rename 4) 2 and 3 make S3ZippedSegmentPuller not necessary (still there, just deprecated and empty) 4) Tweak the TaskToolbox so that the Pullers returned by getSegmentGetters behave the same as they did before --- .../druid/merger/common/TaskToolbox.java | 39 +++-- .../druid/initialization/ServerInit.java | 12 +- .../metamx/druid/loading/S3SegmentPuller.java | 144 +++++----------- .../druid/loading/S3ZippedSegmentPuller.java | 163 +----------------- .../metamx/druid/loading/SegmentPuller.java | 2 +- .../druid/loading/SingleSegmentLoader.java | 141 ++++++++++++++- 6 files changed, 219 insertions(+), 282 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index 0cebe1fc91c..d775921d560 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -21,13 +21,15 @@ package com.metamx.druid.merger.common; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.S3SegmentPuller; -import com.metamx.druid.loading.S3SegmentGetterConfig; -import com.metamx.druid.loading.S3ZippedSegmentPuller; import com.metamx.druid.loading.SegmentPuller; +import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.SingleSegmentLoader; +import com.metamx.druid.loading.StorageAdapterLoadingException; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.emitter.service.ServiceEmitter; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -88,19 +90,28 @@ public class TaskToolbox public Map getSegmentGetters(final Task task) { - final S3SegmentGetterConfig getterConfig = new S3SegmentGetterConfig() - { - @Override - public File getCacheDirectory() - { - return new File(config.getTaskDir(task), "fetched_segments"); - } - }; + LoaderPullerAdapter puller = new LoaderPullerAdapter(new File(config.getTaskDir(task), "fetched_segments")); return ImmutableMap.builder() - .put("s3", new S3SegmentPuller(s3Client, getterConfig)) - .put("s3_union", new S3SegmentPuller(s3Client, getterConfig)) - .put("s3_zip", new S3ZippedSegmentPuller(s3Client, getterConfig)) + .put("s3", puller) + .put("s3_union", puller) + .put("s3_zip", puller) .build(); } + + class LoaderPullerAdapter implements SegmentPuller{ + private SingleSegmentLoader loader; + public LoaderPullerAdapter(File cacheDir){ + loader = new SingleSegmentLoader(new S3SegmentPuller(s3Client), new MMappedQueryableIndexFactory(), cacheDir); + } + @Override + public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException { + return loader.getSegmentFiles(loadSpec); + } + + @Override + public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { + return -1; + } + } } diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 1d727f9abe3..3a510e8b23c 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -35,7 +35,6 @@ import com.metamx.druid.query.group.GroupByQueryEngineConfig; import com.metamx.druid.Query; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.loading.QueryableLoaderConfig; -import com.metamx.druid.loading.S3ZippedSegmentPuller; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; @@ -69,8 +68,8 @@ public class ServerInit { DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); - final S3SegmentPuller segmentGetter = new S3SegmentPuller(s3Client, config); - final S3ZippedSegmentPuller zippedGetter = new S3ZippedSegmentPuller(s3Client, config); + final S3SegmentPuller segmentGetter = new S3SegmentPuller(s3Client); + final QueryableIndexFactory factory; if ("mmap".equals(config.getQueryableFactoryType())) { factory = new MMappedQueryableIndexFactory(); @@ -78,11 +77,12 @@ public class ServerInit throw new ISE("Unknown queryableFactoryType[%s]", config.getQueryableFactoryType()); } + SingleSegmentLoader segmentLoader = new SingleSegmentLoader(segmentGetter, factory, config.getCacheDirectory()); delegateLoader.setLoaderTypes( ImmutableMap.builder() - .put("s3", new SingleSegmentLoader(segmentGetter, factory)) - .put("s3_zip", new SingleSegmentLoader(zippedGetter, factory)) - .build() + .put("s3", segmentLoader) + .put("s3_zip", segmentLoader) + .build() ); return delegateLoader; diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java index 380489548d5..f85a489b1fe 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java @@ -25,17 +25,15 @@ import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.common.s3.S3Utils; -import org.apache.commons.io.FileUtils; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; import org.joda.time.DateTime; import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; import java.util.Map; -import java.util.zip.GZIPInputStream; /** */ @@ -48,133 +46,85 @@ public class S3SegmentPuller implements SegmentPuller private static final String KEY = "key"; private final RestS3Service s3Client; - private final S3SegmentGetterConfig config; @Inject public S3SegmentPuller( - RestS3Service s3Client, - S3SegmentGetterConfig config + RestS3Service s3Client ) { this.s3Client = s3Client; - this.config = config; } @Override public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { - Map loadSpec = segment.getLoadSpec(); - String s3Bucket = MapUtils.getString(loadSpec, "bucket"); - String s3Path = MapUtils.getString(loadSpec, "key"); + S3Coords s3Coords = new S3Coords(segment); - log.info("Loading index at path[s3://%s/%s]", s3Bucket, s3Path); + log.info("Loading index at path[%s]", s3Coords); - S3Object s3Obj = null; + if(!isObjectInBucket(s3Coords)){ + throw new StorageAdapterLoadingException("IndexFile[%s] does not exist.", s3Coords); + } + + long currTime = System.currentTimeMillis(); File tmpFile = null; + S3Object s3Obj = null; + try { - if (!s3Client.isObjectInBucket(s3Bucket, s3Path)) { - throw new StorageAdapterLoadingException("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3Path); - } + s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); + tmpFile = File.createTempFile(s3Coords.bucket, new DateTime().toString() + s3Coords.path.replace('/', '_')); + log.info("Downloading file[%s] to local tmpFile[%s] for segment[%s]", s3Coords, tmpFile, segment); - File cacheFile = new File(config.getCacheDirectory(), computeCacheFilePath(s3Bucket, s3Path)); - - if (cacheFile.exists()) { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(s3Bucket), s3Path); - DateTime cacheFileLastModified = new DateTime(cacheFile.lastModified()); - DateTime s3ObjLastModified = new DateTime(objDetails.getLastModifiedDate().getTime()); - if (cacheFileLastModified.isAfter(s3ObjLastModified)) { - log.info( - "Found cacheFile[%s] with modified[%s], which is after s3Obj[%s]. Using.", - cacheFile, - cacheFileLastModified, - s3ObjLastModified - ); - return cacheFile.getParentFile(); - } - FileUtils.deleteDirectory(cacheFile.getParentFile()); - } - - long currTime = System.currentTimeMillis(); - - tmpFile = File.createTempFile(s3Bucket, new DateTime().toString()); - log.info( - "Downloading file[s3://%s/%s] to local tmpFile[%s] for cacheFile[%s]", - s3Bucket, s3Path, tmpFile, cacheFile - ); - - s3Obj = s3Client.getObject(new S3Bucket(s3Bucket), s3Path); StreamUtils.copyToFileAndClose(s3Obj.getDataInputStream(), tmpFile, DEFAULT_TIMEOUT); final long downloadEndTime = System.currentTimeMillis(); - log.info("Download of file[%s] completed in %,d millis", cacheFile, downloadEndTime - currTime); + log.info("Download of file[%s] completed in %,d millis", tmpFile, downloadEndTime - currTime); - if (!cacheFile.getParentFile().mkdirs()) { - log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); - } - cacheFile.delete(); - - if (s3Path.endsWith("gz")) { - log.info("Decompressing file[%s] to [%s]", tmpFile, cacheFile); - StreamUtils.copyToFileAndClose( - new GZIPInputStream(new FileInputStream(tmpFile)), - cacheFile - ); - if (!tmpFile.delete()) { - log.error("Could not delete tmpFile[%s].", tmpFile); - } - } else { - log.info("Rename tmpFile[%s] to cacheFile[%s]", tmpFile, cacheFile); - if (!tmpFile.renameTo(cacheFile)) { - log.warn("Error renaming tmpFile[%s] to cacheFile[%s]. Copying instead.", tmpFile, cacheFile); - - StreamUtils.copyToFileAndClose(new FileInputStream(tmpFile), cacheFile); - if (!tmpFile.delete()) { - log.error("Could not delete tmpFile[%s].", tmpFile); - } - } - } - - long endTime = System.currentTimeMillis(); - log.info("Local processing of file[%s] done in %,d millis", cacheFile, endTime - downloadEndTime); - - return cacheFile.getParentFile(); + return tmpFile; } catch (Exception e) { + if(tmpFile!=null && tmpFile.exists()){ + tmpFile.delete(); + } throw new StorageAdapterLoadingException(e, e.getMessage()); } finally { S3Utils.closeStreamsQuietly(s3Obj); - if (tmpFile != null && tmpFile.exists()) { - log.warn("Deleting tmpFile[%s] in finally block. Why?", tmpFile); - tmpFile.delete(); - } } } - private String computeCacheFilePath(String s3Bucket, String s3Path) - { - return String.format( - "%s/%s", s3Bucket, s3Path.endsWith("gz") ? s3Path.substring(0, s3Path.length() - ".gz".length()) : s3Path - ); + private boolean isObjectInBucket(S3Coords coords) throws StorageAdapterLoadingException { + try { + return s3Client.isObjectInBucket(coords.bucket, coords.path); + } catch (ServiceException e) { + throw new StorageAdapterLoadingException(e, "Problem communicating with S3 checking bucket/path[%s]", coords); + } } @Override - public boolean cleanSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - File cacheFile = new File( - config.getCacheDirectory(), - computeCacheFilePath(MapUtils.getString(loadSpec, BUCKET), MapUtils.getString(loadSpec, KEY)) - ); - + public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { + S3Coords coords = new S3Coords(segment); try { - final File parentFile = cacheFile.getParentFile(); - log.info("Recursively deleting file[%s]", parentFile); - FileUtils.deleteDirectory(parentFile); - } - catch (IOException e) { + S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); + return objDetails.getLastModifiedDate().getTime(); + } catch (S3ServiceException e) { throw new StorageAdapterLoadingException(e, e.getMessage()); } + } - return true; + private class S3Coords { + String bucket; + String path; + + public S3Coords(DataSegment segment) { + Map loadSpec = segment.getLoadSpec(); + bucket = MapUtils.getString(loadSpec, BUCKET); + path = MapUtils.getString(loadSpec, KEY); + if(path.startsWith("/")){ + path = path.substring(1); + } + } + public String toString(){ + return String.format("s3://%s/%s", bucket, path); + } } } diff --git a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java index 8fd8ebd4542..a3a7c724687 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java @@ -19,169 +19,14 @@ package com.metamx.druid.loading; -import com.google.common.io.Closeables; -import com.metamx.common.MapUtils; -import com.metamx.common.StreamUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.common.s3.S3Utils; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; -import org.jets3t.service.model.S3Object; -import org.joda.time.DateTime; - -import java.io.BufferedInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.util.Map; -import java.util.zip.ZipEntry; -import java.util.zip.ZipInputStream; /** + * @deprecated */ -public class S3ZippedSegmentPuller implements SegmentPuller +public class S3ZippedSegmentPuller extends S3SegmentPuller { - private static final Logger log = new Logger(S3ZippedSegmentPuller.class); - - private static final String BUCKET = "bucket"; - private static final String KEY = "key"; - - private final RestS3Service s3Client; - private final S3SegmentGetterConfig config; - - public S3ZippedSegmentPuller( - RestS3Service s3Client, - S3SegmentGetterConfig config - ) - { - this.s3Client = s3Client; - this.config = config; - } - - @Override - public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - String s3Bucket = MapUtils.getString(loadSpec, "bucket"); - String s3Path = MapUtils.getString(loadSpec, "key"); - - if (s3Path.startsWith("/")) { - s3Path = s3Path.substring(1); - } - - log.info("Loading index at path[s3://%s/%s]", s3Bucket, s3Path); - - S3Object s3Obj = null; - File tmpFile = null; - try { - if (!s3Client.isObjectInBucket(s3Bucket, s3Path)) { - throw new StorageAdapterLoadingException("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3Path); - } - - File cacheFile = new File(config.getCacheDirectory(), computeCacheFilePath(s3Bucket, s3Path)); - - if (cacheFile.exists()) { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(s3Bucket), s3Path); - DateTime cacheFileLastModified = new DateTime(cacheFile.lastModified()); - DateTime s3ObjLastModified = new DateTime(objDetails.getLastModifiedDate().getTime()); - if (cacheFileLastModified.isAfter(s3ObjLastModified)) { - log.info( - "Found cacheFile[%s] with modified[%s], which is after s3Obj[%s]. Using.", - cacheFile, - cacheFileLastModified, - s3ObjLastModified - ); - return cacheFile; - } - FileUtils.deleteDirectory(cacheFile); - } - - long currTime = System.currentTimeMillis(); - - tmpFile = File.createTempFile(s3Bucket, new DateTime().toString()); - log.info( - "Downloading file[s3://%s/%s] to local tmpFile[%s] for cacheFile[%s]", - s3Bucket, s3Path, tmpFile, cacheFile - ); - - s3Obj = s3Client.getObject(new S3Bucket(s3Bucket), s3Path); - StreamUtils.copyToFileAndClose(s3Obj.getDataInputStream(), tmpFile); - final long downloadEndTime = System.currentTimeMillis(); - log.info("Download of file[%s] completed in %,d millis", cacheFile, downloadEndTime - currTime); - - if (cacheFile.exists()) { - FileUtils.deleteDirectory(cacheFile); - } - cacheFile.mkdirs(); - - ZipInputStream zipIn = null; - OutputStream out = null; - ZipEntry entry = null; - try { - zipIn = new ZipInputStream(new BufferedInputStream(new FileInputStream(tmpFile))); - while ((entry = zipIn.getNextEntry()) != null) { - out = new FileOutputStream(new File(cacheFile, entry.getName())); - IOUtils.copy(zipIn, out); - zipIn.closeEntry(); - Closeables.closeQuietly(out); - out = null; - } - } - finally { - Closeables.closeQuietly(out); - Closeables.closeQuietly(zipIn); - } - - long endTime = System.currentTimeMillis(); - log.info("Local processing of file[%s] done in %,d millis", cacheFile, endTime - downloadEndTime); - - log.info("Deleting tmpFile[%s]", tmpFile); - tmpFile.delete(); - - return cacheFile; - } - catch (Exception e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - finally { - S3Utils.closeStreamsQuietly(s3Obj); - if (tmpFile != null && tmpFile.exists()) { - log.warn("Deleting tmpFile[%s] in finally block. Why?", tmpFile); - tmpFile.delete(); - } - } - } - - private String computeCacheFilePath(String s3Bucket, String s3Path) - { - return new File(String.format("%s/%s", s3Bucket, s3Path)).getParent(); - } - - @Override - public boolean cleanSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - File cacheFile = new File( - config.getCacheDirectory(), - computeCacheFilePath( - MapUtils.getString(loadSpec, BUCKET), - MapUtils.getString(loadSpec, KEY) - ) - ); - - try { - log.info("Deleting directory[%s]", cacheFile); - FileUtils.deleteDirectory(cacheFile); - } - catch (IOException e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - - return true; + public S3ZippedSegmentPuller(RestS3Service s3Client) { + super(s3Client); } } diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java index 9cba65f425c..3e5f1b1a161 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java @@ -29,5 +29,5 @@ import java.util.Map; public interface SegmentPuller { public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; - public boolean cleanSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; + long getLastModified(DataSegment segment) throws StorageAdapterLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index 19c3981e988..ae62cfda1e9 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -19,40 +19,171 @@ package com.metamx.druid.loading; +import com.google.common.base.Joiner; +import com.google.common.io.Closeables; import com.google.inject.Inject; +import com.metamx.common.StreamUtils; +import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; import com.metamx.druid.index.Segment; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; + +import java.io.*; +import java.util.zip.GZIPInputStream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; /** */ public class SingleSegmentLoader implements SegmentLoader { + private static final Logger log = new Logger(SingleSegmentLoader.class); + private final SegmentPuller segmentPuller; private final QueryableIndexFactory factory; + private File cacheDirectory; + private static final Joiner JOINER = Joiner.on("/").skipNulls(); @Inject public SingleSegmentLoader( - SegmentPuller segmentPuller, - QueryableIndexFactory factory - ) + SegmentPuller segmentPuller, + QueryableIndexFactory factory, + File cacheDirectory) { this.segmentPuller = segmentPuller; this.factory = factory; + this.cacheDirectory = cacheDirectory; } @Override public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException { - final QueryableIndex index = factory.factorize(segmentPuller.getSegmentFiles(segment)); + File segmentFiles = getSegmentFiles(segment); + final QueryableIndex index = factory.factorize(segmentFiles); return new QueryableIndexSegment(segment.getIdentifier(), index); } + public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { + File cacheFile = getCacheFile(segment); + if (cacheFile.exists()) { + long localLastModified = cacheFile.lastModified(); + long remoteLastModified = segmentPuller.getLastModified(segment); + if(remoteLastModified > 0 && localLastModified >= remoteLastModified){ + log.info( + "Found cacheFile[%s] with modified[%s], which is same or after remote[%s]. Using.", + cacheFile, + localLastModified, + remoteLastModified + ); + return cacheFile.getParentFile(); + } + } + + File pulledFile = segmentPuller.getSegmentFiles(segment); + + if(!cacheFile.getParentFile().mkdirs()){ + log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); + } + if (cacheFile.exists()) { + cacheFile.delete(); + } + + if(pulledFile.getName().endsWith(".zip")){ + unzip(pulledFile, cacheFile.getParentFile()); + } else if(pulledFile.getName().endsWith(".gz")){ + gunzip(pulledFile, cacheFile); + } else { + moveToCache(pulledFile, cacheFile); + } + + return cacheFile.getParentFile(); + } + + private File getCacheFile(DataSegment segment) { + String outputKey = JOINER.join( + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart(), + segment.getInterval().getEnd() + ), + segment.getVersion(), + segment.getShardSpec().getPartitionNum() + ); + + return new File(cacheDirectory, outputKey); + } + + private void moveToCache(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + log.info("Rename pulledFile[%s] to cacheFile[%s]", pulledFile, cacheFile); + if(!pulledFile.renameTo(cacheFile)){ + log.warn("Error renaming pulledFile[%s] to cacheFile[%s]. Copying instead.", pulledFile, cacheFile); + + try { + StreamUtils.copyToFileAndClose(new FileInputStream(pulledFile), cacheFile); + } catch (IOException e) { + throw new StorageAdapterLoadingException(e,"Problem moving pulledFile[%s] to cache[%s]", pulledFile, cacheFile); + } + if (!pulledFile.delete()) { + log.error("Could not delete pulledFile[%s].", pulledFile); + } + } + } + + private void unzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + log.info("Unzipping file[%s] to [%s]", pulledFile, cacheFile); + ZipInputStream zipIn = null; + OutputStream out = null; + ZipEntry entry = null; + try { + zipIn = new ZipInputStream(new BufferedInputStream(new FileInputStream(pulledFile))); + while ((entry = zipIn.getNextEntry()) != null) { + out = new FileOutputStream(new File(cacheFile, entry.getName())); + IOUtils.copy(zipIn, out); + zipIn.closeEntry(); + Closeables.closeQuietly(out); + out = null; + } + } catch(IOException e) { + throw new StorageAdapterLoadingException(e,"Problem unzipping[%s]", pulledFile); + } + finally { + Closeables.closeQuietly(out); + Closeables.closeQuietly(zipIn); + } + } + + private void gunzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + log.info("Gunzipping file[%s] to [%s]", pulledFile, cacheFile); + try { + StreamUtils.copyToFileAndClose( + new GZIPInputStream(new FileInputStream(pulledFile)), + cacheFile + ); + } catch (IOException e) { + throw new StorageAdapterLoadingException(e,"Problem gunzipping[%s]", pulledFile); + } + if (!pulledFile.delete()) { + log.error("Could not delete tmpFile[%s].", pulledFile); + } + } + @Override public void cleanup(DataSegment segment) throws StorageAdapterLoadingException { - segmentPuller.cleanSegmentFiles(segment); + File cacheFile = getCacheFile(segment).getParentFile(); + + try { + log.info("Deleting directory[%s]", cacheFile); + FileUtils.deleteDirectory(cacheFile); + } + catch (IOException e) { + throw new StorageAdapterLoadingException(e, e.getMessage()); + } } + } From dc3459d3f931e3d230a97f23da38152e947798b0 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Wed, 13 Feb 2013 19:15:49 -0600 Subject: [PATCH 065/121] 1) Initial commit of refactorings on top of housejester's to simplify the zipping and simplify the creation of other methods of loading data --- .../metamx/druid/utils/CompressionUtils.java | 90 ++++++++++ .../examples/RealtimeStandaloneMain.java | 6 +- .../examples/RealtimeStandaloneMain.java | 6 +- .../druid/merger/common/TaskToolbox.java | 48 ++--- .../common/index/YeOldePlumberSchool.java | 10 +- .../common/task/IndexGeneratorTask.java | 6 +- .../druid/merger/common/task/MergeTask.java | 10 +- .../http/IndexerCoordinatorNode.java | 8 +- .../druid/merger/worker/http/WorkerNode.java | 8 +- .../metamx/druid/realtime/RealtimeNode.java | 22 +-- .../druid/realtime/RealtimePlumberSchool.java | 12 +- .../druid/realtime/S3SegmentPusher.java | 6 +- .../druid/coordination/ServerManager.java | 14 +- .../druid/coordination/ZkCoordinator.java | 6 +- .../druid/initialization/ServerInit.java | 4 +- ...mentPuller.java => DataSegmentPuller.java} | 22 ++- ...mentPusher.java => DataSegmentPusher.java} | 2 +- .../loading/DelegatingSegmentLoader.java | 8 +- .../loading/MMappedQueryableIndexFactory.java | 8 +- .../druid/loading/QueryableIndexFactory.java | 2 +- .../druid/loading/S3DataSegmentPuller.java | 170 ++++++++++++++++++ ...ntPusher.java => S3DataSegmentPusher.java} | 12 +- .../metamx/druid/loading/S3SegmentPuller.java | 130 -------------- .../druid/loading/S3ZippedSegmentPuller.java | 32 ---- .../metamx/druid/loading/SegmentLoader.java | 4 +- ...tion.java => SegmentLoadingException.java} | 6 +- .../druid/loading/SingleSegmentLoader.java | 102 ++++------- .../druid/coordination/ServerManagerTest.java | 15 +- .../druid/loading/NoopSegmentLoader.java | 4 +- 29 files changed, 410 insertions(+), 363 deletions(-) create mode 100644 common/src/main/java/com/metamx/druid/utils/CompressionUtils.java rename server/src/main/java/com/metamx/druid/loading/{SegmentPuller.java => DataSegmentPuller.java} (55%) rename server/src/main/java/com/metamx/druid/loading/{SegmentPusher.java => DataSegmentPusher.java} (96%) create mode 100644 server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java rename server/src/main/java/com/metamx/druid/loading/{S3SegmentPusher.java => S3DataSegmentPusher.java} (96%) delete mode 100644 server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java delete mode 100644 server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java rename server/src/main/java/com/metamx/druid/loading/{StorageAdapterLoadingException.java => SegmentLoadingException.java} (88%) diff --git a/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java new file mode 100644 index 00000000000..2b87d0a866d --- /dev/null +++ b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java @@ -0,0 +1,90 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.utils; + +import com.google.common.io.ByteStreams; +import com.google.common.io.Closeables; +import com.metamx.common.ISE; +import com.metamx.common.StreamUtils; +import com.metamx.common.logger.Logger; +import sun.misc.IOUtils; + +import java.io.BufferedInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.zip.GZIPInputStream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; + +/** + */ +public class CompressionUtils +{ + private static final Logger log = new Logger(CompressionUtils.class); + + public static void unzip(File pulledFile, File outDir) throws IOException + { + if (!(outDir.exists() && outDir.isDirectory())) { + throw new ISE("outDir[%s] must exist and be a directory"); + } + + log.info("Unzipping file[%s] to [%s]", pulledFile, outDir); + InputStream in = null; + try { + in = new BufferedInputStream(new FileInputStream(pulledFile)); + unzip(in, outDir); + } + finally { + Closeables.closeQuietly(in); + } + } + + public static void unzip(InputStream in, File outDir) throws IOException + { + ZipInputStream zipIn = new ZipInputStream(in); + + ZipEntry entry; + while ((entry = zipIn.getNextEntry()) != null) { + OutputStream out = null; + try { + out = new FileOutputStream(new File(outDir, entry.getName())); + ByteStreams.copy(zipIn, out); + zipIn.closeEntry(); + } + finally { + Closeables.closeQuietly(out); + } + } + } + + public static void gunzip(File pulledFile, File outDir) throws IOException + { + log.info("Gunzipping file[%s] to [%s]", pulledFile, outDir); + StreamUtils.copyToFileAndClose(new GZIPInputStream(new FileInputStream(pulledFile)), outDir); + if (!pulledFile.delete()) { + log.error("Could not delete tmpFile[%s].", pulledFile); + } + } + +} diff --git a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java index ecdf2606a3b..92eb86cc801 100644 --- a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -12,7 +12,7 @@ import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.realtime.MetadataUpdater; import com.metamx.druid.realtime.MetadataUpdaterConfig; import com.metamx.druid.realtime.RealtimeNode; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.phonebook.PhoneBook; @@ -72,8 +72,8 @@ public class RealtimeStandaloneMain // dummyMetadataUpdater will not send updates to db because standalone demo has no db rn.setMetadataUpdater(dummyMetadataUpdater); - rn.setSegmentPusher( - new SegmentPusher() + rn.setDataSegmentPusher( + new DataSegmentPusher() { @Override public DataSegment push(File file, DataSegment segment) throws IOException diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index e936d481489..ca5b9f64fd4 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -12,7 +12,7 @@ import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.realtime.MetadataUpdater; import com.metamx.druid.realtime.MetadataUpdaterConfig; import com.metamx.druid.realtime.RealtimeNode; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.phonebook.PhoneBook; import druid.examples.twitter.TwitterSpritzerFirehoseFactory; @@ -74,8 +74,8 @@ public class RealtimeStandaloneMain // dummyMetadataUpdater will not send updates to db because standalone demo has no db rn.setMetadataUpdater(dummyMetadataUpdater); - rn.setSegmentPusher( - new SegmentPusher() + rn.setDataSegmentPusher( + new DataSegmentPusher() { @Override public DataSegment push(File file, DataSegment segment) throws IOException diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index d775921d560..f77b08ce713 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -20,14 +20,13 @@ package com.metamx.druid.merger.common; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.MMappedQueryableIndexFactory; -import com.metamx.druid.loading.S3SegmentPuller; -import com.metamx.druid.loading.SegmentPuller; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; -import com.metamx.druid.loading.StorageAdapterLoadingException; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.emitter.service.ServiceEmitter; @@ -35,6 +34,7 @@ import com.metamx.emitter.service.ServiceEmitter; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import java.io.File; +import java.util.List; import java.util.Map; /** @@ -45,14 +45,14 @@ public class TaskToolbox private final IndexerCoordinatorConfig config; private final ServiceEmitter emitter; private final RestS3Service s3Client; - private final SegmentPusher segmentPusher; + private final DataSegmentPusher segmentPusher; private final ObjectMapper objectMapper; public TaskToolbox( IndexerCoordinatorConfig config, ServiceEmitter emitter, RestS3Service s3Client, - SegmentPusher segmentPusher, + DataSegmentPusher segmentPusher, ObjectMapper objectMapper ) { @@ -78,7 +78,7 @@ public class TaskToolbox return s3Client; } - public SegmentPusher getSegmentPusher() + public DataSegmentPusher getSegmentPusher() { return segmentPusher; } @@ -88,30 +88,20 @@ public class TaskToolbox return objectMapper; } - public Map getSegmentGetters(final Task task) + public Map getSegments(final Task task, List segments) + throws SegmentLoadingException { - LoaderPullerAdapter puller = new LoaderPullerAdapter(new File(config.getTaskDir(task), "fetched_segments")); + final SingleSegmentLoader loader = new SingleSegmentLoader( + new S3DataSegmentPuller(s3Client), + new MMappedQueryableIndexFactory(), + new File(config.getTaskDir(task), "fetched_segments") + ); - return ImmutableMap.builder() - .put("s3", puller) - .put("s3_union", puller) - .put("s3_zip", puller) - .build(); - } - - class LoaderPullerAdapter implements SegmentPuller{ - private SingleSegmentLoader loader; - public LoaderPullerAdapter(File cacheDir){ - loader = new SingleSegmentLoader(new S3SegmentPuller(s3Client), new MMappedQueryableIndexFactory(), cacheDir); - } - @Override - public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException { - return loader.getSegmentFiles(loadSpec); + Map retVal = Maps.newLinkedHashMap(); + for (DataSegment segment : segments) { + retVal.put(segment, loader.getSegmentFiles(segment)); } - @Override - public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { - return -1; - } + return retVal; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index 7d456d29e01..703dbe898f7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -35,7 +35,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.realtime.FireDepartmentMetrics; import com.metamx.druid.realtime.FireHydrant; @@ -61,7 +61,7 @@ public class YeOldePlumberSchool implements PlumberSchool { private final Interval interval; private final String version; - private final SegmentPusher segmentPusher; + private final DataSegmentPusher dataSegmentPusher; private final File tmpSegmentDir; private static final Logger log = new Logger(YeOldePlumberSchool.class); @@ -70,13 +70,13 @@ public class YeOldePlumberSchool implements PlumberSchool public YeOldePlumberSchool( @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, - @JacksonInject("segmentPusher") SegmentPusher segmentPusher, + @JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher, @JacksonInject("tmpSegmentDir") File tmpSegmentDir ) { this.interval = interval; this.version = version; - this.segmentPusher = segmentPusher; + this.dataSegmentPusher = dataSegmentPusher; this.tmpSegmentDir = tmpSegmentDir; } @@ -149,7 +149,7 @@ public class YeOldePlumberSchool implements PlumberSchool .withVersion(version) .withBinaryVersion(IndexIO.getVersionFromDir(fileToUpload)); - segmentPusher.push(fileToUpload, segmentToUpload); + dataSegmentPusher.push(fileToUpload, segmentToUpload); log.info( "Uploaded segment[%s]", diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index b3da8978bd3..e56b9c4967d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -27,6 +27,7 @@ import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.input.InputRow; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.index.YeOldePlumberSchool; @@ -36,7 +37,6 @@ import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.Schema; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.realtime.Sink; @@ -111,7 +111,7 @@ public class IndexGeneratorTask extends AbstractTask // We need to track published segments. final List pushedSegments = new CopyOnWriteArrayList(); - final SegmentPusher wrappedSegmentPusher = new SegmentPusher() + final DataSegmentPusher wrappedDataSegmentPusher = new DataSegmentPusher() { @Override public DataSegment push(File file, DataSegment segment) throws IOException @@ -128,7 +128,7 @@ public class IndexGeneratorTask extends AbstractTask final Plumber plumber = new YeOldePlumberSchool( getInterval(), context.getVersion(), - wrappedSegmentPusher, + wrappedDataSegmentPusher, tmpDir ).findPlumber(schema, metrics); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 21f6c1e6416..2cfec4e5d2a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -31,13 +31,11 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.loading.SegmentPuller; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -147,13 +145,7 @@ public abstract class MergeTask extends AbstractTask // download segments to merge - final Map segmentGetters = toolbox.getSegmentGetters(this); - final Map gettedSegments = Maps.newHashMap(); - for (final DataSegment segment : segments) { - Map loadSpec = segment.getLoadSpec(); - SegmentPuller segmentPuller = segmentGetters.get(loadSpec.get("type")); - gettedSegments.put(segment, segmentPuller.getSegmentFiles(segment)); - } + final Map gettedSegments = toolbox.getSegments(this, segments); // merge files together final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 7d075cbe9e8..ec5404086d9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -49,9 +49,9 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -403,12 +403,12 @@ public class IndexerCoordinatorNode extends RegisteringNode PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ); - final SegmentPusher segmentPusher = new S3SegmentPusher( + final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, configFactory.build(S3SegmentPusherConfig.class), jsonMapper ); - taskToolbox = new TaskToolbox(config, emitter, s3Client, segmentPusher, jsonMapper); + taskToolbox = new TaskToolbox(config, emitter, s3Client, dataSegmentPusher, jsonMapper); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 237ae09d854..a152f0f003f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -36,6 +36,7 @@ import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -44,9 +45,8 @@ import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; import com.metamx.druid.merger.worker.config.WorkerConfig; -import com.metamx.druid.loading.S3SegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -290,12 +290,12 @@ public class WorkerNode extends RegisteringNode PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ); - final SegmentPusher segmentPusher = new S3SegmentPusher( + final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, configFactory.build(S3SegmentPusherConfig.class), jsonMapper ); - taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, segmentPusher, jsonMapper); + taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, dataSegmentPusher, jsonMapper); } } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index d8fc7ebeea7..536074d6247 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -46,9 +46,9 @@ import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; @@ -86,7 +86,7 @@ public class RealtimeNode extends BaseServerNode private final Map injectablesMap = Maps.newLinkedHashMap(); private MetadataUpdater metadataUpdater = null; - private SegmentPusher segmentPusher = null; + private DataSegmentPusher dataSegmentPusher = null; private List fireDepartments = null; private ServerView view = null; @@ -117,10 +117,10 @@ public class RealtimeNode extends BaseServerNode return this; } - public RealtimeNode setSegmentPusher(SegmentPusher segmentPusher) + public RealtimeNode setDataSegmentPusher(DataSegmentPusher dataSegmentPusher) { - Preconditions.checkState(this.segmentPusher == null, "Cannot set segmentPusher once it has already been set."); - this.segmentPusher = segmentPusher; + Preconditions.checkState(this.dataSegmentPusher == null, "Cannot set segmentPusher once it has already been set."); + this.dataSegmentPusher = dataSegmentPusher; return this; } @@ -144,10 +144,10 @@ public class RealtimeNode extends BaseServerNode return metadataUpdater; } - public SegmentPusher getSegmentPusher() + public DataSegmentPusher getDataSegmentPusher() { initializeSegmentPusher(); - return segmentPusher; + return dataSegmentPusher; } public List getFireDepartments() @@ -220,7 +220,7 @@ public class RealtimeNode extends BaseServerNode } injectables.put("queryRunnerFactoryConglomerate", getConglomerate()); - injectables.put("segmentPusher", segmentPusher); + injectables.put("segmentPusher", dataSegmentPusher); injectables.put("metadataUpdater", metadataUpdater); injectables.put("serverView", view); injectables.put("serviceEmitter", getEmitter()); @@ -256,7 +256,7 @@ public class RealtimeNode extends BaseServerNode private void initializeSegmentPusher() { - if (segmentPusher == null) { + if (dataSegmentPusher == null) { final Properties props = getProps(); final RestS3Service s3Client; try { @@ -271,7 +271,7 @@ public class RealtimeNode extends BaseServerNode throw Throwables.propagate(e); } - segmentPusher = new S3SegmentPusher(s3Client, getConfigFactory().build(S3SegmentPusherConfig.class), getJsonMapper()); + dataSegmentPusher = new S3DataSegmentPusher(s3Client, getConfigFactory().build(S3SegmentPusherConfig.class), getJsonMapper()); } } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java index 47a4ef0f40f..70c12eaaa45 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -48,7 +48,7 @@ import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; @@ -95,7 +95,7 @@ public class RealtimePlumberSchool implements PlumberSchool private volatile RejectionPolicyFactory rejectionPolicyFactory = null; private volatile QueryRunnerFactoryConglomerate conglomerate = null; - private volatile SegmentPusher segmentPusher = null; + private volatile DataSegmentPusher dataSegmentPusher = null; private volatile MetadataUpdater metadataUpdater = null; private volatile ServerView serverView = null; private ServiceEmitter emitter; @@ -130,9 +130,9 @@ public class RealtimePlumberSchool implements PlumberSchool } @JacksonInject("segmentPusher") - public void setSegmentPusher(SegmentPusher segmentPusher) + public void setDataSegmentPusher(DataSegmentPusher dataSegmentPusher) { - this.segmentPusher = segmentPusher; + this.dataSegmentPusher = dataSegmentPusher; } @JacksonInject("metadataUpdater") @@ -325,7 +325,7 @@ public class RealtimePlumberSchool implements PlumberSchool QueryableIndex index = IndexIO.loadIndex(mergedFile); - DataSegment segment = segmentPusher.push( + DataSegment segment = dataSegmentPusher.push( mergedFile, sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions())) ); @@ -512,7 +512,7 @@ public class RealtimePlumberSchool implements PlumberSchool private void verifyState() { Preconditions.checkNotNull(conglomerate, "must specify a queryRunnerFactoryConglomerate to do this action."); - Preconditions.checkNotNull(segmentPusher, "must specify a segmentPusher to do this action."); + Preconditions.checkNotNull(dataSegmentPusher, "must specify a segmentPusher to do this action."); Preconditions.checkNotNull(metadataUpdater, "must specify a metadataUpdater to do this action."); Preconditions.checkNotNull(serverView, "must specify a serverView to do this action."); Preconditions.checkNotNull(emitter, "must specify a serviceEmitter to do this action."); diff --git a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java index 5a74b17e223..2e40c398bdc 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java @@ -20,16 +20,18 @@ package com.metamx.druid.realtime; import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import org.jets3t.service.impl.rest.httpclient.RestS3Service; /** * A placeholder class to make the move of the SegmentPushers to a new package backwards compatible + * + * Exists in 0.2, can be removed from 0.3 on */ @Deprecated -public class S3SegmentPusher extends com.metamx.druid.loading.S3SegmentPusher implements SegmentPusher +public class S3SegmentPusher extends com.metamx.druid.loading.S3DataSegmentPusher implements DataSegmentPusher { public S3SegmentPusher( RestS3Service s3Client, diff --git a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java index 3bde07908c7..38e7d1e4a39 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -30,7 +30,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.collect.CountingMap; import com.metamx.druid.index.Segment; import com.metamx.druid.loading.SegmentLoader; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.PartitionHolder; import com.metamx.druid.query.BySegmentQueryRunner; @@ -104,24 +104,24 @@ public class ServerManager implements QuerySegmentWalker } } - public void loadSegment(final DataSegment segment) throws StorageAdapterLoadingException + public void loadSegment(final DataSegment segment) throws SegmentLoadingException { final Segment adapter; try { adapter = segmentLoader.getSegment(segment); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { try { segmentLoader.cleanup(segment); } - catch (StorageAdapterLoadingException e1) { + catch (SegmentLoadingException e1) { // ignore } throw e; } if (adapter == null) { - throw new StorageAdapterLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); + throw new SegmentLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); } synchronized (lock) { @@ -139,7 +139,7 @@ public class ServerManager implements QuerySegmentWalker ); if ((entry != null) && (entry.getChunk(segment.getShardSpec().getPartitionNum()) != null)) { log.info("Told to load a adapter for a segment[%s] that already exists", segment.getIdentifier()); - throw new StorageAdapterLoadingException("Segment already exists[%s]", segment.getIdentifier()); + throw new SegmentLoadingException("Segment already exists[%s]", segment.getIdentifier()); } loadedIntervals.add( @@ -154,7 +154,7 @@ public class ServerManager implements QuerySegmentWalker } } - public void dropSegment(final DataSegment segment) throws StorageAdapterLoadingException + public void dropSegment(final DataSegment segment) throws SegmentLoadingException { String dataSource = segment.getDataSource(); synchronized (lock) { diff --git a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java index 57d64e0ba32..1951205975c 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java +++ b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java @@ -29,7 +29,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.phonebook.PhoneBook; @@ -245,14 +245,14 @@ public class ZkCoordinator implements DataSegmentChangeHandler } catch (IOException e) { removeSegment(segment); - throw new StorageAdapterLoadingException( + throw new SegmentLoadingException( "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile ); } yp.announce(servedSegmentsLocation, segment.getIdentifier(), segment); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { log.error(e, "Failed to load segment[%s]", segment); emitter.emit( new AlertEvent.Builder().build( diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 3a510e8b23c..82a5f263608 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -28,7 +28,7 @@ import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.loading.DelegatingSegmentLoader; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; -import com.metamx.druid.loading.S3SegmentPuller; +import com.metamx.druid.loading.S3DataSegmentPuller; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.query.group.GroupByQueryEngine; import com.metamx.druid.query.group.GroupByQueryEngineConfig; @@ -68,7 +68,7 @@ public class ServerInit { DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); - final S3SegmentPuller segmentGetter = new S3SegmentPuller(s3Client); + final S3DataSegmentPuller segmentGetter = new S3DataSegmentPuller(s3Client); final QueryableIndexFactory factory; if ("mmap".equals(config.getQueryableFactoryType())) { diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java similarity index 55% rename from server/src/main/java/com/metamx/druid/loading/SegmentPuller.java rename to server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java index 3e5f1b1a161..b821c653a6e 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java @@ -22,12 +22,26 @@ package com.metamx.druid.loading; import com.metamx.druid.client.DataSegment; import java.io.File; -import java.util.Map; /** */ -public interface SegmentPuller +public interface DataSegmentPuller { - public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; - long getLastModified(DataSegment segment) throws StorageAdapterLoadingException; + /** + * Pull down segment files for the given DataSegment and put them in the given directory. + * + * @param segment The segment to pull down files for + * @param dir The directory to store the files in + * @throws SegmentLoadingException if there are any errors + */ + public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException; + + /** + * Returns the last modified time of the given segment. + * + * @param segment The segment to check the last modified time for + * @return the last modified time in millis from the epoch + * @throws SegmentLoadingException if there are any errors + */ + public long getLastModified(DataSegment segment) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java similarity index 96% rename from server/src/main/java/com/metamx/druid/loading/SegmentPusher.java rename to server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java index 3700215efc1..5369480d6b9 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java @@ -24,7 +24,7 @@ import com.metamx.druid.client.DataSegment; import java.io.File; import java.io.IOException; -public interface SegmentPusher +public interface DataSegmentPusher { public DataSegment push(File file, DataSegment segment) throws IOException; } diff --git a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java index d576e59ae82..0f8e1e7074f 100644 --- a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java @@ -44,24 +44,24 @@ public class DelegatingSegmentLoader implements SegmentLoader } @Override - public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(DataSegment segment) throws SegmentLoadingException { return getLoader(segment.getLoadSpec()).getSegment(segment); } @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws SegmentLoadingException { getLoader(segment.getLoadSpec()).cleanup(segment); } - private SegmentLoader getLoader(Map loadSpec) throws StorageAdapterLoadingException + private SegmentLoader getLoader(Map loadSpec) throws SegmentLoadingException { String type = MapUtils.getString(loadSpec, "type"); SegmentLoader loader = loaderTypes.get(type); if (loader == null) { - throw new StorageAdapterLoadingException("Unknown loader type[%s]. Known types are %s", type, loaderTypes.keySet()); + throw new SegmentLoadingException("Unknown loader type[%s]. Known types are %s", type, loaderTypes.keySet()); } return loader; } diff --git a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java index 648813d62ac..9f8594a30d2 100644 --- a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java @@ -34,7 +34,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory private static final Logger log = new Logger(MMappedQueryableIndexFactory.class); @Override - public QueryableIndex factorize(File parentDir) throws StorageAdapterLoadingException + public QueryableIndex factorize(File parentDir) throws SegmentLoadingException { try { if (! IndexIO.canBeMapped(parentDir)) { @@ -46,11 +46,11 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory IndexIO.storeLatest(IndexIO.readIndex(parentDir), canBeMappedDir); if (! IndexIO.canBeMapped(canBeMappedDir)) { - throw new StorageAdapterLoadingException("WTF!? newly written file[%s] cannot be mapped!?", canBeMappedDir); + throw new SegmentLoadingException("WTF!? newly written file[%s] cannot be mapped!?", canBeMappedDir); } for (File file : canBeMappedDir.listFiles()) { if (! file.renameTo(new File(parentDir, file.getName()))) { - throw new StorageAdapterLoadingException("Couldn't rename[%s] to [%s]", canBeMappedDir, parentDir); + throw new SegmentLoadingException("Couldn't rename[%s] to [%s]", canBeMappedDir, parentDir); } } FileUtils.deleteDirectory(canBeMappedDir); @@ -66,7 +66,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory catch (IOException e2) { log.error(e, "Problem deleting parentDir[%s]", parentDir); } - throw new StorageAdapterLoadingException(e, e.getMessage()); + throw new SegmentLoadingException(e, e.getMessage()); } } } diff --git a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java index d7f60309aa6..276bbc2028a 100644 --- a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java @@ -27,5 +27,5 @@ import java.io.File; */ public interface QueryableIndexFactory { - public QueryableIndex factorize(File parentDir) throws StorageAdapterLoadingException; + public QueryableIndex factorize(File parentDir) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java new file mode 100644 index 00000000000..011e1633ca1 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java @@ -0,0 +1,170 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.loading; + +import com.google.common.io.ByteStreams; +import com.google.common.io.Closeables; +import com.google.common.io.Files; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.common.s3.S3Utils; +import com.metamx.druid.utils.CompressionUtils; +import org.apache.commons.io.FileUtils; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.model.S3Bucket; +import org.jets3t.service.model.S3Object; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; +import java.util.zip.GZIPInputStream; + +/** + */ +public class S3DataSegmentPuller implements DataSegmentPuller +{ + private static final Logger log = new Logger(S3DataSegmentPuller.class); + + private static final String BUCKET = "bucket"; + private static final String KEY = "key"; + + private final RestS3Service s3Client; + + @Inject + public S3DataSegmentPuller( + RestS3Service s3Client + ) + { + this.s3Client = s3Client; + } + + @Override + public void getSegmentFiles(DataSegment segment, File outDir) throws SegmentLoadingException + { + S3Coords s3Coords = new S3Coords(segment); + + log.info("Pulling index at path[%s] to outDir[%s]", s3Coords, outDir); + + if (!isObjectInBucket(s3Coords)) { + throw new SegmentLoadingException("IndexFile[%s] does not exist.", s3Coords); + } + + if (!outDir.exists()) { + outDir.mkdirs(); + } + + if (!outDir.isDirectory()) { + throw new ISE("outDir[%s] must be a directory.", outDir); + } + + long startTime = System.currentTimeMillis(); + S3Object s3Obj = null; + + try { + s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); + + InputStream in = null; + try { + in = s3Obj.getDataInputStream(); + final String key = s3Obj.getKey(); + if (key.endsWith(".zip")) { + CompressionUtils.unzip(in, outDir); + } else if (key.endsWith(".gz")) { + final File outFile = new File(outDir, toFilename(key, ".gz")); + ByteStreams.copy(new GZIPInputStream(in), Files.newOutputStreamSupplier(outFile)); + } else { + ByteStreams.copy(in, Files.newOutputStreamSupplier(new File(outDir, toFilename(key, "")))); + } + log.info("Pull of file[%s] completed in %,d millis", s3Obj, System.currentTimeMillis() - startTime); + } + catch (IOException e) { + FileUtils.deleteDirectory(outDir); + throw new SegmentLoadingException(e, "Problem decompressing object[%s]", s3Obj); + } + finally { + Closeables.closeQuietly(in); + } + } + catch (Exception e) { + throw new SegmentLoadingException(e, e.getMessage()); + } + finally { + S3Utils.closeStreamsQuietly(s3Obj); + } + + } + + private String toFilename(String key, final String suffix) + { + String filename = key.substring(key.lastIndexOf("/") + 1); // characters after last '/' + filename = filename.substring(0, filename.length() - suffix.length()); // remove the suffix from the end + return filename; + } + + private boolean isObjectInBucket(S3Coords coords) throws SegmentLoadingException + { + try { + return s3Client.isObjectInBucket(coords.bucket, coords.path); + } + catch (ServiceException e) { + throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords); + } + } + + @Override + public long getLastModified(DataSegment segment) throws SegmentLoadingException + { + S3Coords coords = new S3Coords(segment); + try { + S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); + return objDetails.getLastModifiedDate().getTime(); + } + catch (S3ServiceException e) { + throw new SegmentLoadingException(e, e.getMessage()); + } + } + + private static class S3Coords + { + String bucket; + String path; + + public S3Coords(DataSegment segment) + { + Map loadSpec = segment.getLoadSpec(); + bucket = MapUtils.getString(loadSpec, BUCKET); + path = MapUtils.getString(loadSpec, KEY); + if (path.startsWith("/")) { + path = path.substring(1); + } + } + + public String toString() + { + return String.format("s3://%s/%s", bucket, path); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java similarity index 96% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java rename to server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index 5af4b905719..89a15b056ec 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -41,19 +41,19 @@ import java.security.NoSuchAlgorithmException; import java.util.zip.ZipEntry; import java.util.zip.ZipOutputStream; -public class S3SegmentPusher implements SegmentPusher +public class S3DataSegmentPusher implements DataSegmentPusher { - private static final EmittingLogger log = new EmittingLogger(S3SegmentPusher.class); + private static final EmittingLogger log = new EmittingLogger(S3DataSegmentPusher.class); private static final Joiner JOINER = Joiner.on("/").skipNulls(); private final RestS3Service s3Client; private final S3SegmentPusherConfig config; private final ObjectMapper jsonMapper; - public S3SegmentPusher( - RestS3Service s3Client, - S3SegmentPusherConfig config, - ObjectMapper jsonMapper + public S3DataSegmentPusher( + RestS3Service s3Client, + S3SegmentPusherConfig config, + ObjectMapper jsonMapper ) { this.s3Client = s3Client; diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java deleted file mode 100644 index f85a489b1fe..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java +++ /dev/null @@ -1,130 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.loading; - -import com.google.inject.Inject; -import com.metamx.common.MapUtils; -import com.metamx.common.StreamUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.common.s3.S3Utils; -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; -import org.jets3t.service.model.S3Object; -import org.joda.time.DateTime; - -import java.io.File; -import java.util.Map; - -/** - */ -public class S3SegmentPuller implements SegmentPuller -{ - private static final Logger log = new Logger(S3SegmentPuller.class); - private static final long DEFAULT_TIMEOUT = 5 * 60 * 1000; - - private static final String BUCKET = "bucket"; - private static final String KEY = "key"; - - private final RestS3Service s3Client; - - @Inject - public S3SegmentPuller( - RestS3Service s3Client - ) - { - this.s3Client = s3Client; - } - - @Override - public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - S3Coords s3Coords = new S3Coords(segment); - - log.info("Loading index at path[%s]", s3Coords); - - if(!isObjectInBucket(s3Coords)){ - throw new StorageAdapterLoadingException("IndexFile[%s] does not exist.", s3Coords); - } - - long currTime = System.currentTimeMillis(); - File tmpFile = null; - S3Object s3Obj = null; - - try { - s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); - tmpFile = File.createTempFile(s3Coords.bucket, new DateTime().toString() + s3Coords.path.replace('/', '_')); - log.info("Downloading file[%s] to local tmpFile[%s] for segment[%s]", s3Coords, tmpFile, segment); - - StreamUtils.copyToFileAndClose(s3Obj.getDataInputStream(), tmpFile, DEFAULT_TIMEOUT); - final long downloadEndTime = System.currentTimeMillis(); - log.info("Download of file[%s] completed in %,d millis", tmpFile, downloadEndTime - currTime); - - return tmpFile; - } - catch (Exception e) { - if(tmpFile!=null && tmpFile.exists()){ - tmpFile.delete(); - } - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - finally { - S3Utils.closeStreamsQuietly(s3Obj); - } - } - - private boolean isObjectInBucket(S3Coords coords) throws StorageAdapterLoadingException { - try { - return s3Client.isObjectInBucket(coords.bucket, coords.path); - } catch (ServiceException e) { - throw new StorageAdapterLoadingException(e, "Problem communicating with S3 checking bucket/path[%s]", coords); - } - } - - @Override - public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { - S3Coords coords = new S3Coords(segment); - try { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); - return objDetails.getLastModifiedDate().getTime(); - } catch (S3ServiceException e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - } - - private class S3Coords { - String bucket; - String path; - - public S3Coords(DataSegment segment) { - Map loadSpec = segment.getLoadSpec(); - bucket = MapUtils.getString(loadSpec, BUCKET); - path = MapUtils.getString(loadSpec, KEY); - if(path.startsWith("/")){ - path = path.substring(1); - } - } - public String toString(){ - return String.format("s3://%s/%s", bucket, path); - } - } -} diff --git a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java deleted file mode 100644 index a3a7c724687..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.loading; - -import org.jets3t.service.impl.rest.httpclient.RestS3Service; - -/** - * @deprecated - */ -public class S3ZippedSegmentPuller extends S3SegmentPuller -{ - public S3ZippedSegmentPuller(RestS3Service s3Client) { - super(s3Client); - } -} diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java index 1ca54b89106..20fa5592ac2 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java @@ -26,6 +26,6 @@ import com.metamx.druid.index.Segment; */ public interface SegmentLoader { - public Segment getSegment(DataSegment loadSpec) throws StorageAdapterLoadingException; - public void cleanup(DataSegment loadSpec) throws StorageAdapterLoadingException; + public Segment getSegment(DataSegment loadSpec) throws SegmentLoadingException; + public void cleanup(DataSegment loadSpec) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java similarity index 88% rename from server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java rename to server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java index d0f0ba4be93..d52fd6e3a82 100644 --- a/server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java @@ -21,9 +21,9 @@ package com.metamx.druid.loading; /** */ -public class StorageAdapterLoadingException extends Exception +public class SegmentLoadingException extends Exception { - public StorageAdapterLoadingException( + public SegmentLoadingException( String formatString, Object... objs ) @@ -31,7 +31,7 @@ public class StorageAdapterLoadingException extends Exception super(String.format(formatString, objs)); } - public StorageAdapterLoadingException( + public SegmentLoadingException( Throwable cause, String formatString, Object... objs diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index ae62cfda1e9..9aebfbd35f3 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -20,7 +20,6 @@ package com.metamx.druid.loading; import com.google.common.base.Joiner; -import com.google.common.io.Closeables; import com.google.inject.Inject; import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; @@ -29,12 +28,8 @@ import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; import com.metamx.druid.index.Segment; import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; import java.io.*; -import java.util.zip.GZIPInputStream; -import java.util.zip.ZipEntry; -import java.util.zip.ZipInputStream; /** */ @@ -42,24 +37,25 @@ public class SingleSegmentLoader implements SegmentLoader { private static final Logger log = new Logger(SingleSegmentLoader.class); - private final SegmentPuller segmentPuller; + private final DataSegmentPuller dataSegmentPuller; private final QueryableIndexFactory factory; private File cacheDirectory; private static final Joiner JOINER = Joiner.on("/").skipNulls(); @Inject public SingleSegmentLoader( - SegmentPuller segmentPuller, - QueryableIndexFactory factory, - File cacheDirectory) + DataSegmentPuller dataSegmentPuller, + QueryableIndexFactory factory, + File cacheDirectory + ) { - this.segmentPuller = segmentPuller; + this.dataSegmentPuller = dataSegmentPuller; this.factory = factory; this.cacheDirectory = cacheDirectory; } @Override - public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(DataSegment segment) throws SegmentLoadingException { File segmentFiles = getSegmentFiles(segment); final QueryableIndex index = factory.factorize(segmentFiles); @@ -67,43 +63,37 @@ public class SingleSegmentLoader implements SegmentLoader return new QueryableIndexSegment(segment.getIdentifier(), index); } - public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { + public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException + { File cacheFile = getCacheFile(segment); if (cacheFile.exists()) { long localLastModified = cacheFile.lastModified(); - long remoteLastModified = segmentPuller.getLastModified(segment); - if(remoteLastModified > 0 && localLastModified >= remoteLastModified){ + long remoteLastModified = dataSegmentPuller.getLastModified(segment); + if (remoteLastModified > 0 && localLastModified >= remoteLastModified) { log.info( "Found cacheFile[%s] with modified[%s], which is same or after remote[%s]. Using.", cacheFile, localLastModified, remoteLastModified ); - return cacheFile.getParentFile(); + return cacheFile; } } - File pulledFile = segmentPuller.getSegmentFiles(segment); + dataSegmentPuller.getSegmentFiles(segment, cacheFile); - if(!cacheFile.getParentFile().mkdirs()){ + if (!cacheFile.getParentFile().mkdirs()) { log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); } if (cacheFile.exists()) { cacheFile.delete(); } - if(pulledFile.getName().endsWith(".zip")){ - unzip(pulledFile, cacheFile.getParentFile()); - } else if(pulledFile.getName().endsWith(".gz")){ - gunzip(pulledFile, cacheFile); - } else { - moveToCache(pulledFile, cacheFile); - } - - return cacheFile.getParentFile(); + return cacheFile; } - private File getCacheFile(DataSegment segment) { + private File getCacheFile(DataSegment segment) + { String outputKey = JOINER.join( segment.getDataSource(), String.format( @@ -118,15 +108,22 @@ public class SingleSegmentLoader implements SegmentLoader return new File(cacheDirectory, outputKey); } - private void moveToCache(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + private void moveToCache(File pulledFile, File cacheFile) throws SegmentLoadingException + { log.info("Rename pulledFile[%s] to cacheFile[%s]", pulledFile, cacheFile); - if(!pulledFile.renameTo(cacheFile)){ + if (!pulledFile.renameTo(cacheFile)) { log.warn("Error renaming pulledFile[%s] to cacheFile[%s]. Copying instead.", pulledFile, cacheFile); try { StreamUtils.copyToFileAndClose(new FileInputStream(pulledFile), cacheFile); - } catch (IOException e) { - throw new StorageAdapterLoadingException(e,"Problem moving pulledFile[%s] to cache[%s]", pulledFile, cacheFile); + } + catch (IOException e) { + throw new SegmentLoadingException( + e, + "Problem moving pulledFile[%s] to cache[%s]", + pulledFile, + cacheFile + ); } if (!pulledFile.delete()) { log.error("Could not delete pulledFile[%s].", pulledFile); @@ -134,46 +131,8 @@ public class SingleSegmentLoader implements SegmentLoader } } - private void unzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { - log.info("Unzipping file[%s] to [%s]", pulledFile, cacheFile); - ZipInputStream zipIn = null; - OutputStream out = null; - ZipEntry entry = null; - try { - zipIn = new ZipInputStream(new BufferedInputStream(new FileInputStream(pulledFile))); - while ((entry = zipIn.getNextEntry()) != null) { - out = new FileOutputStream(new File(cacheFile, entry.getName())); - IOUtils.copy(zipIn, out); - zipIn.closeEntry(); - Closeables.closeQuietly(out); - out = null; - } - } catch(IOException e) { - throw new StorageAdapterLoadingException(e,"Problem unzipping[%s]", pulledFile); - } - finally { - Closeables.closeQuietly(out); - Closeables.closeQuietly(zipIn); - } - } - - private void gunzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { - log.info("Gunzipping file[%s] to [%s]", pulledFile, cacheFile); - try { - StreamUtils.copyToFileAndClose( - new GZIPInputStream(new FileInputStream(pulledFile)), - cacheFile - ); - } catch (IOException e) { - throw new StorageAdapterLoadingException(e,"Problem gunzipping[%s]", pulledFile); - } - if (!pulledFile.delete()) { - log.error("Could not delete tmpFile[%s].", pulledFile); - } - } - @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws SegmentLoadingException { File cacheFile = getCacheFile(segment).getParentFile(); @@ -182,8 +141,7 @@ public class SingleSegmentLoader implements SegmentLoader FileUtils.deleteDirectory(cacheFile); } catch (IOException e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); + throw new SegmentLoadingException(e, e.getMessage()); } } - } diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index 762662741a6..d10566c5da4 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -31,7 +31,6 @@ import com.metamx.common.Pair; import com.metamx.common.guava.ConcatSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Capabilities; import com.metamx.druid.Druids; import com.metamx.druid.Query; import com.metamx.druid.QueryGranularity; @@ -39,12 +38,9 @@ import com.metamx.druid.StorageAdapter; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.Segment; -import com.metamx.druid.index.brita.Filter; import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.index.v1.SegmentIdAttachedStorageAdapter; -import com.metamx.druid.index.v1.processing.Cursor; import com.metamx.druid.loading.SegmentLoader; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.ConcatQueryRunner; @@ -54,7 +50,6 @@ import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.query.QueryToolChest; -import com.metamx.druid.query.search.SearchHit; import com.metamx.druid.query.search.SearchQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; @@ -62,7 +57,6 @@ import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceMetricEvent; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -72,7 +66,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.concurrent.ExecutorService; /** @@ -101,7 +94,7 @@ public class ServerManagerTest } @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws SegmentLoadingException { } @@ -245,7 +238,7 @@ public class ServerManagerTest ) ); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { throw new RuntimeException(e); } } @@ -267,7 +260,7 @@ public class ServerManagerTest ) ); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { throw new RuntimeException(e); } } diff --git a/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java index 29d784d3631..ca41c4dfec6 100644 --- a/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java +++ b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java @@ -30,7 +30,7 @@ import org.joda.time.Interval; public class NoopSegmentLoader implements SegmentLoader { @Override - public Segment getSegment(final DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(final DataSegment segment) throws SegmentLoadingException { return new Segment() { @@ -61,7 +61,7 @@ public class NoopSegmentLoader implements SegmentLoader } @Override - public void cleanup(DataSegment loadSpec) throws StorageAdapterLoadingException + public void cleanup(DataSegment loadSpec) throws SegmentLoadingException { } } From f8c54a72c2b61319ee8cbdd7b15bceb132a82703 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Tue, 19 Feb 2013 19:22:59 -0600 Subject: [PATCH 066/121] 1) Changes to allow for local storage --- .../druid/jackson/DefaultObjectMapper.java | 2 +- .../metamx/druid/utils/CompressionUtils.java | 38 ++++++- .../examples/RealtimeStandaloneMain.java | 6 +- .../druid/indexer/DeterminePartitionsJob.java | 3 +- .../indexer/HadoopDruidIndexerConfig.java | 3 +- .../druid/indexer/IndexGeneratorJob.java | 3 +- .../druid/merger/common/TaskToolbox.java | 10 +- .../common/index/YeOldePlumberSchool.java | 21 +++- .../druid/merger/common/task/DeleteTask.java | 6 +- .../druid/merger/common/task/MergeTask.java | 5 +- .../http/IndexerCoordinatorNode.java | 4 +- .../druid/merger/worker/http/WorkerNode.java | 4 +- .../druid/realtime/FireDepartmentConfig.java | 2 + .../metamx/druid/realtime/RealtimeNode.java | 36 ++++-- .../druid/realtime/RealtimePlumberSchool.java | 15 ++- .../druid/realtime/S3SegmentPusher.java | 4 +- .../com/metamx/druid/http/ComputeNode.java | 4 +- .../druid/initialization/ServerInit.java | 22 ++-- .../druid/loading/DataSegmentPusherUtil.java | 44 ++++++++ .../druid/loading/LocalDataSegmentPuller.java | 105 ++++++++++++++++++ .../druid/loading/LocalDataSegmentPusher.java | 96 ++++++++++++++++ ...java => LocalDataSegmentPusherConfig.java} | 6 +- .../loading/MMappedQueryableIndexFactory.java | 2 +- .../druid/loading/S3DataSegmentPusher.java | 51 ++------- ...ig.java => S3DataSegmentPusherConfig.java} | 2 +- ...erConfig.java => SegmentLoaderConfig.java} | 13 ++- .../druid/loading/SingleSegmentLoader.java | 46 ++++---- 27 files changed, 433 insertions(+), 120 deletions(-) create mode 100644 server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java create mode 100644 server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java create mode 100644 server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java rename server/src/main/java/com/metamx/druid/loading/{S3SegmentGetterConfig.java => LocalDataSegmentPusherConfig.java} (86%) rename server/src/main/java/com/metamx/druid/loading/{S3SegmentPusherConfig.java => S3DataSegmentPusherConfig.java} (95%) rename server/src/main/java/com/metamx/druid/loading/{QueryableLoaderConfig.java => SegmentLoaderConfig.java} (76%) diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index 84514bd6c4c..293f80d900d 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -174,7 +174,7 @@ public class DefaultObjectMapper extends ObjectMapper configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); configure(MapperFeature.AUTO_DETECT_GETTERS, false); - configure(MapperFeature.AUTO_DETECT_CREATORS, false); +// configure(MapperFeature.AUTO_DETECT_CREATORS, false); https://github.com/FasterXML/jackson-databind/issues/170 configure(MapperFeature.AUTO_DETECT_FIELDS, false); configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false); configure(MapperFeature.AUTO_DETECT_SETTERS, false); diff --git a/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java index 2b87d0a866d..c34b8e7e960 100644 --- a/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java +++ b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java @@ -21,6 +21,7 @@ package com.metamx.druid.utils; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; +import com.google.common.io.Files; import com.metamx.common.ISE; import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; @@ -29,6 +30,7 @@ import sun.misc.IOUtils; import java.io.BufferedInputStream; import java.io.File; import java.io.FileInputStream; +import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; @@ -36,6 +38,7 @@ import java.io.OutputStream; import java.util.zip.GZIPInputStream; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; +import java.util.zip.ZipOutputStream; /** */ @@ -43,10 +46,43 @@ public class CompressionUtils { private static final Logger log = new Logger(CompressionUtils.class); + public static long zip(File directory, File outputZipFile) throws IOException + { + if (!directory.isDirectory()) { + throw new IOException(String.format("directory[%s] is not a directory", directory)); + } + + if (!outputZipFile.getName().endsWith(".zip")) { + log.warn("No .zip suffix[%s], putting files from [%s] into it anyway.", outputZipFile, directory); + } + + long totalSize = 0; + ZipOutputStream zipOut = null; + try { + zipOut = new ZipOutputStream(new FileOutputStream(outputZipFile)); + File[] files = directory.listFiles(); + for (File file : files) { + log.info("Adding file[%s] with size[%,d]. Total size[%,d]", file, file.length(), totalSize); + if (file.length() >= Integer.MAX_VALUE) { + zipOut.close(); + outputZipFile.delete(); + throw new IOException(String.format("file[%s] too large [%,d]", file, file.length())); + } + zipOut.putNextEntry(new ZipEntry(file.getName())); + totalSize += ByteStreams.copy(Files.newInputStreamSupplier(file), zipOut); + } + } + finally { + Closeables.closeQuietly(zipOut); + } + + return totalSize; + } + public static void unzip(File pulledFile, File outDir) throws IOException { if (!(outDir.exists() && outDir.isDirectory())) { - throw new ISE("outDir[%s] must exist and be a directory"); + throw new ISE("outDir[%s] must exist and be a directory", outDir); } log.info("Unzipping file[%s] to [%s]", pulledFile, outDir); diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index ca5b9f64fd4..5f4d25cb95b 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -48,10 +48,12 @@ public class RealtimeStandaloneMain rn.setPhoneBook(dummyPhoneBook); MetadataUpdater dummyMetadataUpdater = - new MetadataUpdater(new DefaultObjectMapper(), + new MetadataUpdater( + new DefaultObjectMapper(), Config.createFactory(Initialization.loadProperties()).build(MetadataUpdaterConfig.class), dummyPhoneBook, - null) { + null + ) { @Override public void publishSegment(DataSegment segment) throws IOException { diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index d4ee1941396..9a72d997987 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -266,8 +266,7 @@ public class DeterminePartitionsJob implements Jobby Context context ) throws IOException, InterruptedException { - // Create group key - // TODO -- There are more efficient ways to do this + // Create group key, there are probably more efficient ways of doing this final Map> dims = Maps.newTreeMap(); for(final String dim : inputRow.getDimensions()) { final Set dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim)); diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index eed2339114b..979e2d989a4 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -662,8 +662,9 @@ public class HadoopDruidIndexerConfig return new Path( String.format( - "%s/%s_%s/%s/%s", + "%s/%s/%s_%s/%s/%s", getSegmentOutputDir(), + dataSource, bucketInterval.getStart().toString(), bucketInterval.getEnd().toString(), getVersion().toString(), diff --git a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index d8eba264c11..0620ba2bc85 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -379,7 +379,8 @@ public class IndexGeneratorJob implements Jobby ); } else if (outputFS instanceof LocalFileSystem) { loadSpec = ImmutableMap.of( - "type", "test" + "type", "local", + "path", indexOutURI.getPath() ); } else { throw new ISE("Unknown file system[%s]", outputFS.getClass()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index f77b08ce713..0ab0cf49be6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -25,6 +25,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.merger.common.task.Task; @@ -94,7 +95,14 @@ public class TaskToolbox final SingleSegmentLoader loader = new SingleSegmentLoader( new S3DataSegmentPuller(s3Client), new MMappedQueryableIndexFactory(), - new File(config.getTaskDir(task), "fetched_segments") + new SegmentLoaderConfig() + { + @Override + public File getCacheDirectory() + { + return new File(config.getTaskDir(task), "fetched_segments"); + } + } ); Map retVal = Maps.newLinkedHashMap(); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index 703dbe898f7..c26888c4485 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -45,11 +45,11 @@ import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.Sink; - - +import org.apache.commons.io.FileUtils; import org.joda.time.Interval; import java.io.File; +import java.io.IOException; import java.util.List; import java.util.Set; @@ -120,13 +120,13 @@ public class YeOldePlumberSchool implements PlumberSchool @Override public void finishJob() { + // The segment we will upload + File fileToUpload = null; + try { // User should have persisted everything by now. Preconditions.checkState(!theSink.swappable(), "All data must be persisted before fininshing the job!"); - // The segment we will upload - final File fileToUpload; - if(spilled.size() == 0) { throw new IllegalStateException("Nothing indexed?"); } else if(spilled.size() == 1) { @@ -160,6 +160,17 @@ public class YeOldePlumberSchool implements PlumberSchool log.warn(e, "Failed to merge and upload"); throw Throwables.propagate(e); } + finally { + try { + if (fileToUpload != null) { + log.info("Deleting Index File[%s]", fileToUpload); + FileUtils.deleteDirectory(fileToUpload); + } + } + catch (IOException e) { + log.warn(e, "Error deleting directory[%s]", fileToUpload); + } + } } private void spillIfSwappable() diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 5f37ad2853e..3e1bee62e5d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -22,6 +22,7 @@ package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; +import com.google.common.io.Files; import com.metamx.common.logger.Logger; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; @@ -37,7 +38,7 @@ import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.shard.NoneShardSpec; - +import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -102,6 +103,9 @@ public class DeleteTask extends AbstractTask segment.getVersion() ); + log.info("Deleting Uploaded Files[%s]", fileToUpload); + FileUtils.deleteDirectory(fileToUpload); + return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 2cfec4e5d2a..5d062b12892 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -46,7 +46,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import org.apache.commons.codec.digest.DigestUtils; - +import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -168,6 +168,9 @@ public abstract class MergeTask extends AbstractTask emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); + log.info("Deleting Uploaded Files[%s]", fileToUpload); + FileUtils.deleteDirectory(fileToUpload); + return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)); } catch (Exception e) { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index ec5404086d9..af29c7da2b2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -51,7 +51,7 @@ import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -405,7 +405,7 @@ public class IndexerCoordinatorNode extends RegisteringNode ); final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, - configFactory.build(S3SegmentPusherConfig.class), + configFactory.build(S3DataSegmentPusherConfig.class), jsonMapper ); taskToolbox = new TaskToolbox(config, emitter, s3Client, dataSegmentPusher, jsonMapper); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index a152f0f003f..e71cf7f3e98 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -37,6 +37,7 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.S3DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -45,7 +46,6 @@ import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; import com.metamx.druid.merger.worker.config.WorkerConfig; -import com.metamx.druid.loading.S3SegmentPusherConfig; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; @@ -292,7 +292,7 @@ public class WorkerNode extends RegisteringNode ); final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, - configFactory.build(S3SegmentPusherConfig.class), + configFactory.build(S3DataSegmentPusherConfig.class), jsonMapper ); taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, dataSegmentPusher, jsonMapper); diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java index d98997b5051..efc0c7a598f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java @@ -19,6 +19,7 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; @@ -31,6 +32,7 @@ public class FireDepartmentConfig private final int maxRowsInMemory; private final Period intermediatePersistPeriod; + @JsonCreator public FireDepartmentConfig( @JsonProperty("maxRowsInMemory") int maxRowsInMemory, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index 536074d6247..087e87d107f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -47,8 +47,10 @@ import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusherConfig; import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; @@ -258,20 +260,30 @@ public class RealtimeNode extends BaseServerNode { if (dataSegmentPusher == null) { final Properties props = getProps(); - final RestS3Service s3Client; - try { - s3Client = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) + if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) { + dataSegmentPusher = new LocalDataSegmentPusher( + getConfigFactory().build(LocalDataSegmentPusherConfig.class), getJsonMapper() ); } - catch (S3ServiceException e) { - throw Throwables.propagate(e); - } + else { - dataSegmentPusher = new S3DataSegmentPusher(s3Client, getConfigFactory().build(S3SegmentPusherConfig.class), getJsonMapper()); + final RestS3Service s3Client; + try { + s3Client = new RestS3Service( + new AWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ); + } + catch (S3ServiceException e) { + throw Throwables.propagate(e); + } + + dataSegmentPusher = new S3DataSegmentPusher( + s3Client, getConfigFactory().build(S3DataSegmentPusherConfig.class), getJsonMapper() + ); + } } } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java index 70c12eaaa45..775dc7d5305 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -307,7 +307,7 @@ public class RealtimePlumberSchool implements PlumberSchool } } - final File mergedFile; + File mergedFile = null; try { List indexes = Lists.newArrayList(); for (FireHydrant fireHydrant : sink) { @@ -337,6 +337,19 @@ public class RealtimePlumberSchool implements PlumberSchool .addData("interval", interval) .emit(); } + + + if (mergedFile != null) { + try { + if (mergedFile != null) { + log.info("Deleting Index File[%s]", mergedFile); + FileUtils.deleteDirectory(mergedFile); + } + } + catch (IOException e) { + log.warn(e, "Error deleting directory[%s]", mergedFile); + } + } } } ); diff --git a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java index 2e40c398bdc..007ea188b82 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java @@ -21,7 +21,7 @@ package com.metamx.druid.realtime; import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -35,7 +35,7 @@ public class S3SegmentPusher extends com.metamx.druid.loading.S3DataSegmentPushe { public S3SegmentPusher( RestS3Service s3Client, - S3SegmentPusherConfig config, + S3DataSegmentPusherConfig config, ObjectMapper jsonMapper ) { diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index 2230932d9a1..08b3eb93da3 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -39,7 +39,7 @@ import com.metamx.druid.coordination.ZkCoordinatorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.QueryableLoaderConfig; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.metrics.ServerMonitor; import com.metamx.druid.query.MetricsEmittingExecutorService; @@ -172,7 +172,7 @@ public class ComputeNode extends BaseServerNode ); setSegmentLoader( - ServerInit.makeDefaultQueryableLoader(s3Client, getConfigFactory().build(QueryableLoaderConfig.class)) + ServerInit.makeDefaultQueryableLoader(s3Client, getConfigFactory().build(SegmentLoaderConfig.class)) ); } catch (S3ServiceException e) { diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 82a5f263608..7cd6caf3c1b 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -26,15 +26,16 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.loading.DelegatingSegmentLoader; +import com.metamx.druid.loading.LocalDataSegmentPuller; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.query.group.GroupByQueryEngine; import com.metamx.druid.query.group.GroupByQueryEngineConfig; import com.metamx.druid.Query; import com.metamx.druid.collect.StupidPool; -import com.metamx.druid.loading.QueryableLoaderConfig; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; @@ -63,26 +64,23 @@ public class ServerInit public static SegmentLoader makeDefaultQueryableLoader( RestS3Service s3Client, - QueryableLoaderConfig config + SegmentLoaderConfig config ) { DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); final S3DataSegmentPuller segmentGetter = new S3DataSegmentPuller(s3Client); + final QueryableIndexFactory factory = new MMappedQueryableIndexFactory(); - final QueryableIndexFactory factory; - if ("mmap".equals(config.getQueryableFactoryType())) { - factory = new MMappedQueryableIndexFactory(); - } else { - throw new ISE("Unknown queryableFactoryType[%s]", config.getQueryableFactoryType()); - } + SingleSegmentLoader s3segmentLoader = new SingleSegmentLoader(segmentGetter, factory, config); + SingleSegmentLoader localSegmentLoader = new SingleSegmentLoader(new LocalDataSegmentPuller(), factory, config); - SingleSegmentLoader segmentLoader = new SingleSegmentLoader(segmentGetter, factory, config.getCacheDirectory()); delegateLoader.setLoaderTypes( ImmutableMap.builder() - .put("s3", segmentLoader) - .put("s3_zip", segmentLoader) - .build() + .put("s3", s3segmentLoader) + .put("s3_zip", s3segmentLoader) + .put("local", localSegmentLoader) + .build() ); return delegateLoader; diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java new file mode 100644 index 00000000000..e72bd787bb3 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java @@ -0,0 +1,44 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.loading; + +import com.google.common.base.Joiner; +import com.metamx.druid.client.DataSegment; + +/** + */ +public class DataSegmentPusherUtil +{ + private static final Joiner JOINER = Joiner.on("/").skipNulls(); + + public static String getStorageDir(DataSegment segment) + { + return JOINER.join( + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart(), + segment.getInterval().getEnd() + ), + segment.getVersion(), + segment.getShardSpec().getPartitionNum() + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java new file mode 100644 index 00000000000..8cdb8e0a7a2 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java @@ -0,0 +1,105 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.loading; + +import com.google.common.io.Files; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.utils.CompressionUtils; + +import java.io.File; +import java.io.IOException; +import java.util.Map; + +/** + */ +public class LocalDataSegmentPuller implements DataSegmentPuller +{ + private static final Logger log = new Logger(LocalDataSegmentPuller.class); + + @Override + public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException + { + final File path = getFile(segment); + + if (path.isDirectory()) { + if (path.equals(dir)) { + log.info("Asked to load [%s] into itself, done!", dir); + return; + } + + log.info("Copying files from [%s] to [%s]", path, dir); + File file = null; + try { + final File[] files = path.listFiles(); + for (int i = 0; i < files.length; ++i) { + file = files[i]; + Files.copy(file, new File(dir, file.getName())); + } + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Unable to copy file[%s].", file); + } + } else { + if (!path.getName().endsWith(".zip")) { + throw new SegmentLoadingException("File is not a zip file[%s]", path); + } + + log.info("Unzipping local file[%s] to [%s]", path, dir); + try { + CompressionUtils.unzip(path, dir); + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Unable to unzip file[%s]", path); + } + } + } + + @Override + public long getLastModified(DataSegment segment) throws SegmentLoadingException + { + final File file = getFile(segment); + + long lastModified = Long.MAX_VALUE; + if (file.isDirectory()) { + for (File childFile : file.listFiles()) { + lastModified = Math.min(childFile.lastModified(), lastModified); + } + } + else { + lastModified = file.lastModified(); + } + + return lastModified; + } + + private File getFile(DataSegment segment) throws SegmentLoadingException + { + final Map loadSpec = segment.getLoadSpec(); + final File path = new File(MapUtils.getString(loadSpec, "path")); + + if (!path.exists()) { + throw new SegmentLoadingException("Asked to load path[%s], but it doesn't exist.", path); + } + + return path; + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java new file mode 100644 index 00000000000..1493b162572 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java @@ -0,0 +1,96 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.loading; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.utils.CompressionUtils; + +import java.io.File; +import java.io.IOException; + +/** + */ +public class LocalDataSegmentPusher implements DataSegmentPusher +{ + private static final Logger log = new Logger(LocalDataSegmentPusher.class); + + private final LocalDataSegmentPusherConfig config; + private final ObjectMapper jsonMapper; + + public LocalDataSegmentPusher( + LocalDataSegmentPusherConfig config, + ObjectMapper jsonMapper + ) + { + this.config = config; + this.jsonMapper = jsonMapper; + } + + @Override + public DataSegment push(File dataSegmentFile, DataSegment segment) throws IOException + { + File outDir = new File(config.getStorageDirectory(), DataSegmentPusherUtil.getStorageDir(segment)); + + if (dataSegmentFile.equals(outDir)) { + long size = 0; + for (File file : dataSegmentFile.listFiles()) { + size += file.length(); + } + + return createDescriptorFile( + segment.withLoadSpec(makeLoadSpec(outDir)) + .withSize(size) + .withBinaryVersion(IndexIO.getVersionFromDir(dataSegmentFile)), + outDir + ); + } + + outDir.mkdirs(); + File outFile = new File(outDir, "index.zip"); + log.info("Compressing files from[%s] to [%s]", dataSegmentFile, outFile); + long size = CompressionUtils.zip(dataSegmentFile, outFile); + + return createDescriptorFile( + segment.withLoadSpec(makeLoadSpec(outFile)) + .withSize(size) + .withBinaryVersion(IndexIO.getVersionFromDir(dataSegmentFile)), + outDir + ); + } + + private DataSegment createDescriptorFile(DataSegment segment, File outDir) throws IOException + { + File descriptorFile = new File(outDir, "descriptor.json"); + log.info("Creating descriptor file at[%s]", descriptorFile); + Files.copy(ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(segment)), descriptorFile); + return segment; + } + + private ImmutableMap makeLoadSpec(File outFile) + { + return ImmutableMap.of("type", "local", "path", outFile.toString()); + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java similarity index 86% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java rename to server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java index c2a4c7f6308..d33a9a5130b 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java @@ -25,8 +25,8 @@ import java.io.File; /** */ -public abstract class S3SegmentGetterConfig +public abstract class LocalDataSegmentPusherConfig { - @Config("druid.paths.indexCache") - public abstract File getCacheDirectory(); + @Config("druid.pusher.local.storageDirectory") + public abstract File getStorageDirectory(); } diff --git a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java index 9f8594a30d2..9896c3f800b 100644 --- a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java @@ -66,7 +66,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory catch (IOException e2) { log.error(e, "Problem deleting parentDir[%s]", parentDir); } - throw new SegmentLoadingException(e, e.getMessage()); + throw new SegmentLoadingException(e, "%s", e.getMessage()); } } } diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index 89a15b056ec..273a07d36f3 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -22,24 +22,20 @@ package com.metamx.druid.loading; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; -import com.google.common.io.Closeables; -import com.metamx.common.ISE; -import com.metamx.common.StreamUtils; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.utils.CompressionUtils; import com.metamx.emitter.EmittingLogger; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; - import org.jets3t.service.S3ServiceException; import org.jets3t.service.acl.gs.GSAccessControlList; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Object; -import java.io.*; +import java.io.File; +import java.io.IOException; import java.security.NoSuchAlgorithmException; -import java.util.zip.ZipEntry; -import java.util.zip.ZipOutputStream; public class S3DataSegmentPusher implements DataSegmentPusher { @@ -47,12 +43,12 @@ public class S3DataSegmentPusher implements DataSegmentPusher private static final Joiner JOINER = Joiner.on("/").skipNulls(); private final RestS3Service s3Client; - private final S3SegmentPusherConfig config; + private final S3DataSegmentPusherConfig config; private final ObjectMapper jsonMapper; public S3DataSegmentPusher( RestS3Service s3Client, - S3SegmentPusherConfig config, + S3DataSegmentPusherConfig config, ObjectMapper jsonMapper ) { @@ -67,35 +63,11 @@ public class S3DataSegmentPusher implements DataSegmentPusher log.info("Uploading [%s] to S3", indexFilesDir); String outputKey = JOINER.join( config.getBaseKey().isEmpty() ? null : config.getBaseKey(), - segment.getDataSource(), - String.format( - "%s_%s", - segment.getInterval().getStart(), - segment.getInterval().getEnd() - ), - segment.getVersion(), - segment.getShardSpec().getPartitionNum() + DataSegmentPusherUtil.getStorageDir(segment) ); - long indexSize = 0; final File zipOutFile = File.createTempFile("druid", "index.zip"); - ZipOutputStream zipOut = null; - try { - zipOut = new ZipOutputStream(new FileOutputStream(zipOutFile)); - File[] indexFiles = indexFilesDir.listFiles(); - for (File indexFile : indexFiles) { - log.info("Adding indexFile[%s] with size[%,d]. Total size[%,d]", indexFile, indexFile.length(), indexSize); - if (indexFile.length() >= Integer.MAX_VALUE) { - throw new ISE("indexFile[%s] too large [%,d]", indexFile, indexFile.length()); - } - zipOut.putNextEntry(new ZipEntry(indexFile.getName())); - IOUtils.copy(new FileInputStream(indexFile), zipOut); - indexSize += indexFile.length(); - } - } - finally { - Closeables.closeQuietly(zipOut); - } + long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile); try { S3Object toPush = new S3Object(zipOutFile); @@ -119,7 +91,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher .withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir)); File descriptorFile = File.createTempFile("druid", "descriptor.json"); - StreamUtils.copyToFileAndClose(new ByteArrayInputStream(jsonMapper.writeValueAsBytes(segment)), descriptorFile); + Files.copy(ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(segment)), descriptorFile); S3Object descriptorObject = new S3Object(descriptorFile); descriptorObject.setBucketName(outputBucket); descriptorObject.setKey(outputKey + "/descriptor.json"); @@ -128,9 +100,6 @@ public class S3DataSegmentPusher implements DataSegmentPusher log.info("Pushing %s", descriptorObject); s3Client.putObject(outputBucket, descriptorObject); - log.info("Deleting Index File[%s]", indexFilesDir); - FileUtils.deleteDirectory(indexFilesDir); - log.info("Deleting zipped index File[%s]", zipOutFile); zipOutFile.delete(); diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java similarity index 95% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java rename to server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java index 0bd66a1a913..a2cada422fb 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java @@ -24,7 +24,7 @@ import org.skife.config.Default; /** */ -public abstract class S3SegmentPusherConfig +public abstract class S3DataSegmentPusherConfig { @Config("druid.pusher.s3.bucket") public abstract String getBucket(); diff --git a/server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java similarity index 76% rename from server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java rename to server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java index c6e2a72c931..294c91b9a38 100644 --- a/server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java @@ -21,13 +21,18 @@ package com.metamx.druid.loading; import org.skife.config.Config; +import java.io.File; + /** */ -public abstract class QueryableLoaderConfig extends S3SegmentGetterConfig +public abstract class SegmentLoaderConfig { - @Config("druid.queryable.factory") - public String getQueryableFactoryType() + @Config({"druid.paths.indexCache", "druid.segmentCache.path"}) + public abstract File getCacheDirectory(); + + @Config("druid.segmentCache.deleteOnRemove") + public boolean deleteOnRemove() { - return "mmap"; + return true; } } diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index 9aebfbd35f3..7e62f57fbf4 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -20,6 +20,7 @@ package com.metamx.druid.loading; import com.google.common.base.Joiner; +import com.google.common.base.Throwables; import com.google.inject.Inject; import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; @@ -39,19 +40,19 @@ public class SingleSegmentLoader implements SegmentLoader private final DataSegmentPuller dataSegmentPuller; private final QueryableIndexFactory factory; - private File cacheDirectory; + private final SegmentLoaderConfig config; private static final Joiner JOINER = Joiner.on("/").skipNulls(); @Inject public SingleSegmentLoader( DataSegmentPuller dataSegmentPuller, QueryableIndexFactory factory, - File cacheDirectory + SegmentLoaderConfig config ) { this.dataSegmentPuller = dataSegmentPuller; this.factory = factory; - this.cacheDirectory = cacheDirectory; + this.config = config; } @Override @@ -65,34 +66,37 @@ public class SingleSegmentLoader implements SegmentLoader public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException { - File cacheFile = getCacheFile(segment); - if (cacheFile.exists()) { - long localLastModified = cacheFile.lastModified(); + File localStorageDir = new File(config.getCacheDirectory(), DataSegmentPusherUtil.getStorageDir(segment)); + if (localStorageDir.exists()) { + long localLastModified = localStorageDir.lastModified(); long remoteLastModified = dataSegmentPuller.getLastModified(segment); if (remoteLastModified > 0 && localLastModified >= remoteLastModified) { log.info( - "Found cacheFile[%s] with modified[%s], which is same or after remote[%s]. Using.", - cacheFile, - localLastModified, - remoteLastModified + "Found localStorageDir[%s] with modified[%s], which is same or after remote[%s]. Using.", + localStorageDir, localLastModified, remoteLastModified ); - return cacheFile; + return localStorageDir; } } - dataSegmentPuller.getSegmentFiles(segment, cacheFile); - - if (!cacheFile.getParentFile().mkdirs()) { - log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); + if (localStorageDir.exists()) { + try { + FileUtils.deleteDirectory(localStorageDir); + } + catch (IOException e) { + log.warn(e, "Exception deleting previously existing local dir[%s]", localStorageDir); + } } - if (cacheFile.exists()) { - cacheFile.delete(); + if (!localStorageDir.mkdirs()) { + log.info("Unable to make parent file[%s]", localStorageDir); } - return cacheFile; + dataSegmentPuller.getSegmentFiles(segment, localStorageDir); + + return localStorageDir; } - private File getCacheFile(DataSegment segment) + private File getLocalStorageDir(DataSegment segment) { String outputKey = JOINER.join( segment.getDataSource(), @@ -105,7 +109,7 @@ public class SingleSegmentLoader implements SegmentLoader segment.getShardSpec().getPartitionNum() ); - return new File(cacheDirectory, outputKey); + return new File(config.getCacheDirectory(), outputKey); } private void moveToCache(File pulledFile, File cacheFile) throws SegmentLoadingException @@ -134,7 +138,7 @@ public class SingleSegmentLoader implements SegmentLoader @Override public void cleanup(DataSegment segment) throws SegmentLoadingException { - File cacheFile = getCacheFile(segment).getParentFile(); + File cacheFile = getLocalStorageDir(segment).getParentFile(); try { log.info("Deleting directory[%s]", cacheFile); From 606a2e4b01a60e2c0639d33a84565b839874edc8 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 15:49:55 -0600 Subject: [PATCH 067/121] [maven-release-plugin] prepare release druid-0.3.1 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 59cc716195c..22460ee95c5 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/common/pom.xml b/common/pom.xml index 710c88d7274..2642875bc5f 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 01078c607f4..1e5f30e4806 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.1-SNAPSHOT + 0.3.1 com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/examples/pom.xml b/examples/pom.xml index 4220ed243bc..75894e58a1a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index ae5e1e767e9..b39ec8fdd15 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index c557c46800b..4d714682f71 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/index-common/pom.xml b/index-common/pom.xml index c67359cb14e..f9b00d18c1c 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/indexer/pom.xml b/indexer/pom.xml index 0a32246ce06..deee2da0598 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/merger/pom.xml b/merger/pom.xml index e1c33c4b13b..cc73763841f 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/pom.xml b/pom.xml index b9b34479851..757862f4ba8 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.1-SNAPSHOT + 0.3.1 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 19553828205..c65b9f7e4db 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/server/pom.xml b/server/pom.xml index 09c44aea4b0..c10ff22b4d2 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 From 699aadd2b19f028cc4280b6807e39a3d2c45a7b2 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 15:50:01 -0600 Subject: [PATCH 068/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 22460ee95c5..453a44717a5 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 2642875bc5f..f813c06dca7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 1e5f30e4806..a2aa0136f94 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.1 + 0.3.2-SNAPSHOT com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 75894e58a1a..c2282b32a38 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index b39ec8fdd15..1b501a6239b 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 4d714682f71..a79dccedb98 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index f9b00d18c1c..2140c83512f 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index deee2da0598..0823e26e699 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index cc73763841f..e64b60808b1 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/pom.xml b/pom.xml index 757862f4ba8..ac1185a4a33 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.1 + 0.3.2-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index c65b9f7e4db..f336b4edf12 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index c10ff22b4d2..2bcf37d7054 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT From 5e123988a0c2b13ee1ae6dbde720c13c3e4b8398 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 20:37:51 -0600 Subject: [PATCH 069/121] 1) Specify old Jackson version to resolve version conflicts between AWS SDK and curator --- pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/pom.xml b/pom.xml index ac1185a4a33..02001be5ce0 100644 --- a/pom.xml +++ b/pom.xml @@ -192,6 +192,16 @@ jackson-jaxrs-json-provider 2.1.3 + + org.codehaus.jackson + jackson-core-asl + 1.9.11 + + + org.codehaus.jackson + jackson-mapper-asl + 1.9.11 + javax.inject javax.inject From 846bc0e4f0ee58997ef93f992b7197d20a1454a6 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 20:39:45 -0600 Subject: [PATCH 070/121] [maven-release-plugin] prepare release druid-0.3.2 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 453a44717a5..01fa2065885 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/common/pom.xml b/common/pom.xml index f813c06dca7..94c032684e4 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index a2aa0136f94..ac544f8a9eb 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.2-SNAPSHOT + 0.3.2 com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/examples/pom.xml b/examples/pom.xml index c2282b32a38..09bc6e84f53 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 1b501a6239b..ed856c5926b 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a79dccedb98..b151a08c5ad 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/index-common/pom.xml b/index-common/pom.xml index 2140c83512f..1ff0704164f 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/indexer/pom.xml b/indexer/pom.xml index 0823e26e699..ffca56c7de8 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/merger/pom.xml b/merger/pom.xml index e64b60808b1..b4c22f1944a 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/pom.xml b/pom.xml index 02001be5ce0..19f95b109b3 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.2-SNAPSHOT + 0.3.2 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index f336b4edf12..da2ed45cbdb 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/server/pom.xml b/server/pom.xml index 2bcf37d7054..62600955d21 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 From e0f6df1a5cc078d19d4ab3668bdfc0ab503c49cc Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 20:39:50 -0600 Subject: [PATCH 071/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 01fa2065885..7109bc96949 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 94c032684e4..10dbd719368 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index ac544f8a9eb..4022fbca433 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.2 + 0.3.3-SNAPSHOT com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 09bc6e84f53..ca22ebde9b9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index ed856c5926b..4cc3bc1514d 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index b151a08c5ad..15d7ec6b245 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 1ff0704164f..55353f21508 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index ffca56c7de8..9652c64ed78 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index b4c22f1944a..08f98b6389b 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/pom.xml b/pom.xml index 19f95b109b3..d24722c64cc 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.2 + 0.3.3-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index da2ed45cbdb..c9ed42132cf 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 62600955d21..5e73eab0396 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT From 8513a5ab2a59a925fca4763ccbc256bab7cfc0f2 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Sun, 24 Feb 2013 22:18:44 -0800 Subject: [PATCH 072/121] 1) Fix SimpleColumn to not produce NPEs when one of its parts is null. --- .../com/metamx/druid/index/column/SimpleColumn.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java index 93825a8e9f0..8179cd623e4 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java @@ -72,30 +72,30 @@ class SimpleColumn implements Column @Override public DictionaryEncodedColumn getDictionaryEncoding() { - return dictionaryEncodedColumn.get(); + return dictionaryEncodedColumn == null ? null : dictionaryEncodedColumn.get(); } @Override public RunLengthColumn getRunLengthColumn() { - return runLengthColumn.get(); + return runLengthColumn == null ? null : runLengthColumn.get(); } @Override public GenericColumn getGenericColumn() { - return genericColumn.get(); + return genericColumn == null ? null : genericColumn.get(); } @Override public ComplexColumn getComplexColumn() { - return complexColumn.get(); + return complexColumn == null ? null : complexColumn.get(); } @Override public BitmapIndex getBitmapIndex() { - return bitmapIndex.get(); + return bitmapIndex == null ? null : bitmapIndex.get(); } } From 7d7ce2b7feb98a8ec0f688e6312b0c88e6652192 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 10:57:26 -0800 Subject: [PATCH 073/121] Fix DeterminePartitionsJob ISE for dimensions not present in all rows --- .../druid/indexer/DeterminePartitionsJob.java | 30 ++++++++++++++----- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index 9a72d997987..f34ff2988f2 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -393,6 +393,9 @@ public class DeterminePartitionsJob implements Jobby final Interval interval = maybeInterval.get(); final byte[] groupKey = interval.getStart().toString().getBytes(Charsets.UTF_8); + // Emit row-counter value. + write(context, groupKey, new DimValueCount("", "", 1)); + for(final Map.Entry> dimAndValues : dims.entrySet()) { final String dim = dimAndValues.getKey(); @@ -509,9 +512,23 @@ public class DeterminePartitionsJob implements Jobby Context context, SortableBytes keyBytes, Iterable combinedIterable ) throws IOException, InterruptedException { - PeekingIterator iterator = Iterators.peekingIterator(combinedIterable.iterator()); + final DateTime bucket = new DateTime(new String(keyBytes.getGroupKey(), Charsets.UTF_8)); + final PeekingIterator iterator = Iterators.peekingIterator(combinedIterable.iterator()); - // "iterator" will take us over many candidate dimensions + log.info( + "Determining partitions for interval: %s", + config.getGranularitySpec().bucketInterval(bucket).orNull() + ); + + // First DVC should be the total row count indicator + final DimValueCount firstDvc = iterator.next(); + final int totalRows = firstDvc.numRows; + + if(!firstDvc.dim.equals("") || !firstDvc.value.equals("")) { + throw new IllegalStateException("WTF?! Expected total row indicator on first k/v pair!"); + } + + // "iterator" will now take us over many candidate dimensions DimPartitions currentDimPartitions = null; DimPartition currentDimPartition = null; String currentDimPartitionStart = null; @@ -635,8 +652,6 @@ public class DeterminePartitionsJob implements Jobby throw new ISE("No suitable partitioning dimension found!"); } - final int totalRows = dimPartitionss.values().iterator().next().getRows(); - int maxCardinality = Integer.MIN_VALUE; long minVariance = Long.MAX_VALUE; DimPartitions minVariancePartitions = null; @@ -644,12 +659,14 @@ public class DeterminePartitionsJob implements Jobby for(final DimPartitions dimPartitions : dimPartitionss.values()) { if(dimPartitions.getRows() != totalRows) { - throw new ISE( - "WTF?! Dimension[%s] row count %,d != expected row count %,d", + log.info( + "Dimension[%s] is not present in all rows (row count %,d != expected row count %,d)", dimPartitions.dim, dimPartitions.getRows(), totalRows ); + + continue; } // Make sure none of these shards are oversized @@ -683,7 +700,6 @@ public class DeterminePartitionsJob implements Jobby throw new ISE("No suitable partitioning dimension found!"); } - final DateTime bucket = new DateTime(new String(keyBytes.getGroupKey(), Charsets.UTF_8)); final OutputStream out = Utils.makePathAndOutputStream( context, config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)), config.isOverwriteFiles() ); From 99c4f9446c399b9e64bbcbb51158dd3617623b2a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 12:24:15 -0800 Subject: [PATCH 074/121] [maven-release-plugin] prepare release druid-0.3.3 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 7109bc96949..0774bd0c214 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/common/pom.xml b/common/pom.xml index 10dbd719368..17b376a2ee5 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 4022fbca433..0902c3e8064 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.3-SNAPSHOT + 0.3.3 com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/examples/pom.xml b/examples/pom.xml index ca22ebde9b9..976379c6678 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 4cc3bc1514d..179061524db 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 15d7ec6b245..344f5718bc0 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/index-common/pom.xml b/index-common/pom.xml index 55353f21508..d191045860b 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/indexer/pom.xml b/indexer/pom.xml index 9652c64ed78..2fd9c17894b 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/merger/pom.xml b/merger/pom.xml index 08f98b6389b..55fb1c76d82 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/pom.xml b/pom.xml index d24722c64cc..a1f3ed99638 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.3-SNAPSHOT + 0.3.3 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index c9ed42132cf..8ca33faab93 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/server/pom.xml b/server/pom.xml index 5e73eab0396..99f8ebcca4c 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 From 9964e7dfe446f6995a7fe8020084e5f4d07d277e Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 12:24:24 -0800 Subject: [PATCH 075/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 0774bd0c214..3af4d9b489a 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 17b376a2ee5..ef9c33b67e1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 0902c3e8064..b5b2a7e460c 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.3 + 0.3.4-SNAPSHOT com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 976379c6678..ae597675a62 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 179061524db..b9ff5296fde 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 344f5718bc0..88f4beae7da 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index d191045860b..34264feb745 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 2fd9c17894b..778f682a0f2 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 55fb1c76d82..e39359574a5 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/pom.xml b/pom.xml index a1f3ed99638..16d86ae8776 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.3 + 0.3.4-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 8ca33faab93..0e48bc168c0 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 99f8ebcca4c..e373da72f3d 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT From 14cf506c43d87e7c5ecb0d282a2478f7f049d5c4 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 17:05:01 -0800 Subject: [PATCH 076/121] DefaultObjectMapper: Add GuavaModule --- common/pom.xml | 8 ++++++-- .../com/metamx/druid/jackson/DefaultObjectMapper.java | 2 ++ pom.xml | 5 +++++ 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/common/pom.xml b/common/pom.xml index ef9c33b67e1..03483c15caa 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -80,8 +80,12 @@ jackson-databind - com.fasterxml.jackson.datatype - jackson-datatype-joda + com.fasterxml.jackson.datatype + jackson-datatype-guava + + + com.fasterxml.jackson.datatype + jackson-datatype-joda org.jdbi diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index 293f80d900d..12079a77959 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -36,6 +36,7 @@ import com.fasterxml.jackson.databind.SerializationFeature; import com.fasterxml.jackson.databind.SerializerProvider; import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.fasterxml.jackson.datatype.guava.GuavaModule; import com.fasterxml.jackson.datatype.joda.JodaModule; import com.google.common.base.Throwables; import com.metamx.common.Granularity; @@ -171,6 +172,7 @@ public class DefaultObjectMapper extends ObjectMapper } ); registerModule(serializerModule); + registerModule(new GuavaModule()); configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); configure(MapperFeature.AUTO_DETECT_GETTERS, false); diff --git a/pom.xml b/pom.xml index 16d86ae8776..1f1afee50e3 100644 --- a/pom.xml +++ b/pom.xml @@ -177,6 +177,11 @@ jackson-databind 2.1.4-mmx-2 + + com.fasterxml.jackson.datatype + jackson-datatype-guava + 2.1.2 + com.fasterxml.jackson.datatype jackson-datatype-joda From 2427e81874c117624fe9fc91af88e6e816d9368d Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 17:05:30 -0800 Subject: [PATCH 077/121] Merger: Feedback from code review --- .../common/actions/SegmentInsertAction.java | 23 +-------- .../common/actions/SegmentNukeAction.java | 23 +-------- .../common/actions/TaskActionToolbox.java | 48 +++++++++++++++++++ .../merger/common/config/TaskConfig.java | 2 +- .../merger/common/task/AbstractTask.java | 9 +--- .../task/IndexDeterminePartitionsTask.java | 12 ++++- .../common/task/IndexGeneratorTask.java | 14 +++++- .../druid/merger/common/task/IndexTask.java | 32 +++++++++---- .../druid/merger/common/task/MergeTask.java | 9 +++- .../merger/common/task/MergeTaskTest.java | 8 ++-- .../merger/common/task/TaskSerdeTest.java | 7 ++- .../coordinator/RemoteTaskRunnerTest.java | 2 +- .../merger/coordinator/TaskLifecycleTest.java | 8 ++-- 13 files changed, 123 insertions(+), 74 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java index 436c16e26fa..75ad4a9161f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java @@ -51,27 +51,8 @@ public class SegmentInsertAction implements TaskAction @Override public Void perform(TaskActionToolbox toolbox) { - // Verify that each of these segments-to-insert falls under some lock - // TODO: Should this be done while holding the giant lock on TaskLockbox? (To prevent anyone from grabbing - // TODO: these locks out from under us while the operation is ongoing.) Probably not necessary. - final List taskLocks = toolbox.getTaskLockbox().findLocksForTask(task); - for(final DataSegment segment : segments) { - final boolean ok = Iterables.any( - taskLocks, new Predicate() - { - @Override - public boolean apply(TaskLock taskLock) - { - return taskLock.getVersion().equals(segment.getVersion()) - && taskLock.getDataSource().equals(segment.getDataSource()) - && taskLock.getInterval().contains(segment.getInterval()); - } - } - ); - - if(!ok) { - throw new ISE("No currently-held lock covers segment: %s", segment); - } + if(!toolbox.taskLockCoversSegments(task, segments, false)) { + throw new ISE("Segments not covered by locks for task: %s", task.getId()); } try { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java index ca0d9f3a9bd..f1b61c58d9f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java @@ -51,27 +51,8 @@ public class SegmentNukeAction implements TaskAction @Override public Void perform(TaskActionToolbox toolbox) { - // Verify that each of these segments-to-nuke falls under some lock - // TODO: Should this be done while holding the giant lock on TaskLockbox? (To prevent anyone from grabbing - // TODO: these locks out from under us while the operation is ongoing.) Probably not necessary. - final List taskLocks = toolbox.getTaskLockbox().findLocksForTask(task); - for(final DataSegment segment : segments) { - final boolean ok = Iterables.any( - taskLocks, new Predicate() - { - @Override - public boolean apply(TaskLock taskLock) - { - return taskLock.getVersion().compareTo(segment.getVersion()) >= 0 - && taskLock.getDataSource().equals(segment.getDataSource()) - && taskLock.getInterval().contains(segment.getInterval()); - } - } - ); - - if(!ok) { - throw new ISE("No currently-held lock covers segment: %s", segment); - } + if(!toolbox.taskLockCoversSegments(task, segments, true)) { + throw new ISE("Segments not covered by locks for task: %s", task.getId()); } try { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java index 30f87a1f6c3..af8f6dcd40c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java @@ -1,10 +1,19 @@ package com.metamx.druid.merger.common.actions; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.metamx.common.ISE; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskLock; +import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.MergerDBCoordinator; import com.metamx.druid.merger.coordinator.TaskLockbox; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.emitter.service.ServiceEmitter; +import java.util.List; +import java.util.Set; + public class TaskActionToolbox { private final TaskQueue taskQueue; @@ -44,4 +53,43 @@ public class TaskActionToolbox { return emitter; } + + public boolean taskLockCoversSegments( + final Task task, + final Set segments, + final boolean allowOlderVersions + ) + { + // Verify that each of these segments falls under some lock + + // NOTE: It is possible for our lock to be revoked (if the task has failed and given up its locks) after we check + // NOTE: it and before we perform the segment insert, but, that should be OK since the worst that happens is we + // NOTE: insert some segments from the task but not others. + + final List taskLocks = getTaskLockbox().findLocksForTask(task); + for(final DataSegment segment : segments) { + final boolean ok = Iterables.any( + taskLocks, new Predicate() + { + @Override + public boolean apply(TaskLock taskLock) + { + final boolean versionOk = allowOlderVersions + ? taskLock.getVersion().compareTo(segment.getVersion()) >= 0 + : taskLock.getVersion().equals(segment.getVersion()); + + return versionOk + && taskLock.getDataSource().equals(segment.getDataSource()) + && taskLock.getInterval().contains(segment.getInterval()); + } + } + ); + + if (!ok) { + return false; + } + } + + return true; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java index c66009cd8ac..5b7609bd042 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java @@ -13,7 +13,7 @@ public abstract class TaskConfig @Config("druid.merger.rowFlushBoundary") @Default("500000") - public abstract long getRowFlushBoundary(); + public abstract int getDefaultRowFlushBoundary(); public File getTaskDir(final Task task) { return new File(getBaseTaskDir(), task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index 53653aa6595..bb65225a6a5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -69,20 +69,13 @@ public abstract class AbstractTask implements Task return dataSource; } + @JsonProperty("interval") @Override public Optional getFixedInterval() { return interval; } - // Awesome hack to get around lack of serde for Optional - // TODO Look into jackson-datatype-guava - @JsonProperty("interval") - private Interval getNullableIntervalForJackson() - { - return interval.orNull(); - } - @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index b94625e7ffa..569aa3e8a29 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -50,11 +50,16 @@ public class IndexDeterminePartitionsTask extends AbstractTask { @JsonProperty private final FirehoseFactory firehoseFactory; + @JsonProperty private final Schema schema; + @JsonProperty private final long targetPartitionSize; + @JsonProperty + private final int rowFlushBoundary; + private static final Logger log = new Logger(IndexTask.class); @JsonCreator @@ -63,7 +68,8 @@ public class IndexDeterminePartitionsTask extends AbstractTask @JsonProperty("interval") Interval interval, @JsonProperty("firehose") FirehoseFactory firehoseFactory, @JsonProperty("schema") Schema schema, - @JsonProperty("targetPartitionSize") long targetPartitionSize + @JsonProperty("targetPartitionSize") long targetPartitionSize, + @JsonProperty("rowFlushBoundary") int rowFlushBoundary ) { super( @@ -81,6 +87,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask this.firehoseFactory = firehoseFactory; this.schema = schema; this.targetPartitionSize = targetPartitionSize; + this.rowFlushBoundary = rowFlushBoundary; } @Override @@ -244,7 +251,8 @@ public class IndexDeterminePartitionsTask extends AbstractTask schema.getAggregators(), schema.getIndexGranularity(), shardSpec - ) + ), + rowFlushBoundary ); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index 514f6cb7461..f4daec2d90a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -58,6 +58,9 @@ public class IndexGeneratorTask extends AbstractTask @JsonProperty private final Schema schema; + @JsonProperty + private final int rowFlushBoundary; + private static final Logger log = new Logger(IndexTask.class); @JsonCreator @@ -65,7 +68,8 @@ public class IndexGeneratorTask extends AbstractTask @JsonProperty("groupId") String groupId, @JsonProperty("interval") Interval interval, @JsonProperty("firehose") FirehoseFactory firehoseFactory, - @JsonProperty("schema") Schema schema + @JsonProperty("schema") Schema schema, + @JsonProperty("rowFlushBoundary") int rowFlushBoundary ) { super( @@ -83,6 +87,7 @@ public class IndexGeneratorTask extends AbstractTask this.firehoseFactory = firehoseFactory; this.schema = schema; + this.rowFlushBoundary = rowFlushBoundary; } @Override @@ -139,6 +144,11 @@ public class IndexGeneratorTask extends AbstractTask tmpDir ).findPlumber(schema, metrics); + // rowFlushBoundary for this job + final int myRowFlushBoundary = this.rowFlushBoundary > 0 + ? rowFlushBoundary + : toolbox.getConfig().getDefaultRowFlushBoundary(); + try { while(firehose.hasMore()) { final InputRow inputRow = firehose.nextRow(); @@ -157,7 +167,7 @@ public class IndexGeneratorTask extends AbstractTask int numRows = sink.add(inputRow); metrics.incrementProcessed(); - if(numRows >= toolbox.getConfig().getRowFlushBoundary()) { + if(numRows >= myRowFlushBoundary) { plumber.persist(firehose.commit()); } } else { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 8cfaa15d300..246389470ef 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -42,11 +42,23 @@ import java.util.List; public class IndexTask extends AbstractTask { - @JsonProperty private final GranularitySpec granularitySpec; - @JsonProperty private final AggregatorFactory[] aggregators; - @JsonProperty private final QueryGranularity indexGranularity; - @JsonProperty private final long targetPartitionSize; - @JsonProperty private final FirehoseFactory firehoseFactory; + @JsonProperty + private final GranularitySpec granularitySpec; + + @JsonProperty + private final AggregatorFactory[] aggregators; + + @JsonProperty + private final QueryGranularity indexGranularity; + + @JsonProperty + private final long targetPartitionSize; + + @JsonProperty + private final FirehoseFactory firehoseFactory; + + @JsonProperty + private final int rowFlushBoundary; private static final Logger log = new Logger(IndexTask.class); @@ -57,7 +69,8 @@ public class IndexTask extends AbstractTask @JsonProperty("aggregators") AggregatorFactory[] aggregators, @JsonProperty("indexGranularity") QueryGranularity indexGranularity, @JsonProperty("targetPartitionSize") long targetPartitionSize, - @JsonProperty("firehose") FirehoseFactory firehoseFactory + @JsonProperty("firehose") FirehoseFactory firehoseFactory, + @JsonProperty("rowFlushBoundary") int rowFlushBoundary ) { super( @@ -75,6 +88,7 @@ public class IndexTask extends AbstractTask this.indexGranularity = indexGranularity; this.targetPartitionSize = targetPartitionSize; this.firehoseFactory = firehoseFactory; + this.rowFlushBoundary = rowFlushBoundary; } public List toSubtasks() @@ -95,7 +109,8 @@ public class IndexTask extends AbstractTask indexGranularity, new NoneShardSpec() ), - targetPartitionSize + targetPartitionSize, + rowFlushBoundary ) ); } else { @@ -110,7 +125,8 @@ public class IndexTask extends AbstractTask aggregators, indexGranularity, new NoneShardSpec() - ) + ), + rowFlushBoundary ) ); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 164c883d18b..f317efc7040 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -22,6 +22,7 @@ package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Objects; @@ -33,6 +34,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; +import com.google.common.hash.Hashing; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; @@ -46,7 +48,6 @@ import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import org.apache.commons.codec.digest.DigestUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -281,7 +282,11 @@ public abstract class MergeTask extends AbstractTask ) ); - return String.format("%s_%s", dataSource, DigestUtils.sha1Hex(segmentIDs).toLowerCase()); + return String.format( + "%s_%s", + dataSource, + Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase() + ); } private static Interval computeMergedInterval(final List segments) diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java index 5267b8c7983..0f1a7a66964 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java @@ -19,9 +19,10 @@ package com.metamx.druid.merger.common.task; +import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; +import com.google.common.hash.Hashing; import com.metamx.druid.client.DataSegment; -import org.apache.commons.codec.digest.DigestUtils; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -72,11 +73,12 @@ public class MergeTaskTest @Test public void testID() { - final String desiredPrefix = "merge_foo_" + DigestUtils.sha1Hex( + final String desiredPrefix = "merge_foo_" + Hashing.sha1().hashString( "2012-01-03T00:00:00.000Z_2012-01-05T00:00:00.000Z_V1_0" + "_2012-01-04T00:00:00.000Z_2012-01-06T00:00:00.000Z_V1_0" + "_2012-01-05T00:00:00.000Z_2012-01-07T00:00:00.000Z_V1_0" - ) + "_"; + , Charsets.UTF_8 + ).toString().toLowerCase() + "_"; Assert.assertEquals( desiredPrefix, testMergeTask.getId().substring(0, desiredPrefix.length()) diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java index 88b43a71667..51310bb2ef0 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java @@ -26,7 +26,8 @@ public class TaskSerdeTest new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, QueryGranularity.NONE, 10000, - null + null, + -1 ); final ObjectMapper jsonMapper = new DefaultObjectMapper(); @@ -52,7 +53,8 @@ public class TaskSerdeTest new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, QueryGranularity.NONE, new NoneShardSpec() - ) + ), + -1 ); final ObjectMapper jsonMapper = new DefaultObjectMapper(); @@ -97,6 +99,7 @@ public class TaskSerdeTest final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); + System.out.println(json); final Task task2 = jsonMapper.readValue(json, Task.class); Assert.assertEquals(task.getId(), task2.getId()); diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index b37ef435a41..5981bd8ed22 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -291,7 +291,7 @@ public class RemoteTaskRunnerTest } @Override - public long getRowFlushBoundary() + public int getDefaultRowFlushBoundary() { return 0; } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java index 3f469da6bb3..69a364e1900 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java @@ -106,7 +106,7 @@ public class TaskLifecycleTest } @Override - public long getRowFlushBoundary() + public int getDefaultRowFlushBoundary() { return 50000; } @@ -182,7 +182,8 @@ public class TaskLifecycleTest IR("2010-01-02T01", "a", "b", 2), IR("2010-01-02T01", "a", "c", 1) ) - ) + ), + -1 ); final TaskStatus mergedStatus = runTask(indexTask); @@ -216,7 +217,8 @@ public class TaskLifecycleTest new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, QueryGranularity.NONE, 10000, - newMockExceptionalFirehoseFactory() + newMockExceptionalFirehoseFactory(), + -1 ); final TaskStatus mergedStatus = runTask(indexTask); From d8fbddb9d418f71c316c9006a4b068d78788caaf Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 26 Feb 2013 11:36:07 -0800 Subject: [PATCH 078/121] Merger: Service discovery for worker -> master communication --- .../actions/RemoteTaskActionClient.java | 40 ++++++++++--- .../http/IndexerCoordinatorResource.java | 6 +- .../merger/worker/config/WorkerConfig.java | 3 + .../druid/merger/worker/http/WorkerNode.java | 58 ++++++++++++++++--- 4 files changed, 90 insertions(+), 17 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java index 2dc86ab3115..26900e29942 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java @@ -1,25 +1,31 @@ package com.metamx.druid.merger.common.actions; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.ToStringResponseHandler; import com.fasterxml.jackson.databind.ObjectMapper; +import com.netflix.curator.x.discovery.ServiceInstance; +import com.netflix.curator.x.discovery.ServiceProvider; import java.net.URI; import java.net.URISyntaxException; +import java.util.Map; public class RemoteTaskActionClient implements TaskActionClient { private final HttpClient httpClient; + private final ServiceProvider serviceProvider; private final ObjectMapper jsonMapper; private static final Logger log = new Logger(RemoteTaskActionClient.class); - public RemoteTaskActionClient(HttpClient httpClient, ObjectMapper jsonMapper) + public RemoteTaskActionClient(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) { this.httpClient = httpClient; + this.serviceProvider = serviceProvider; this.jsonMapper = jsonMapper; } @@ -34,20 +40,36 @@ public class RemoteTaskActionClient implements TaskActionClient .go(new ToStringResponseHandler(Charsets.UTF_8)) .get(); - // TODO Figure out how to check HTTP status code - if(response.equals("")) { - return null; - } else { - return jsonMapper.readValue(response, taskAction.getReturnTypeReference()); - } + final Map responseDict = jsonMapper.readValue( + response, + new TypeReference>() {} + ); + + return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference()); } catch (Exception e) { throw Throwables.propagate(e); } } - public URI getServiceUri() throws URISyntaxException + private URI getServiceUri() throws Exception { - return new URI("http://localhost:8087/mmx/merger/v1/action"); + final ServiceInstance instance = serviceProvider.getInstance(); + final String scheme; + final String host; + final int port; + final String path = "/mmx/merger/v1/action"; + + host = instance.getAddress(); + + if (instance.getSslPort() != null && instance.getSslPort() > 0) { + scheme = "https"; + port = instance.getSslPort(); + } else { + scheme = "http"; + port = instance.getPort(); + } + + return new URI(scheme, null, host, port, path, null, null); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index b01e27a9316..fd3e02bcad3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -21,6 +21,7 @@ package com.metamx.druid.merger.coordinator.http; import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; @@ -176,6 +177,9 @@ public class IndexerCoordinatorResource public Response doAction(final TaskAction action) { final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action); - return Response.ok().entity(ret).build(); + final Map retMap = Maps.newHashMap(); + retMap.put("result", ret); + + return Response.ok().entity(retMap).build(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java index 5b5f3a0a6e7..51a11546d24 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java @@ -39,6 +39,9 @@ public abstract class WorkerConfig @Config("druid.worker.version") public abstract String getVersion(); + @Config("druid.worker.masterService") + public abstract String getMasterService(); + public int getCapacity() { return Runtime.getRuntime().availableProcessors() - 1; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 06ba0f73a32..d02ffa5d9e3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -35,6 +35,7 @@ import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; +import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3DataSegmentPusher; @@ -64,6 +65,8 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; +import com.netflix.curator.x.discovery.ServiceDiscovery; +import com.netflix.curator.x.discovery.ServiceProvider; import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; @@ -105,6 +108,8 @@ public class WorkerNode extends RegisteringNode private WorkerConfig workerConfig = null; private TaskToolbox taskToolbox = null; private CuratorFramework curatorFramework = null; + private ServiceDiscovery serviceDiscovery = null; + private ServiceProvider coordinatorServiceProvider = null; private WorkerCuratorCoordinator workerCuratorCoordinator = null; private TaskMonitor taskMonitor = null; private Server server = null; @@ -156,6 +161,18 @@ public class WorkerNode extends RegisteringNode return this; } + public WorkerNode setCoordinatorServiceProvider(ServiceProvider coordinatorServiceProvider) + { + this.coordinatorServiceProvider = coordinatorServiceProvider; + return this; + } + + public WorkerNode setServiceDiscovery(ServiceDiscovery serviceDiscovery) + { + this.serviceDiscovery = serviceDiscovery; + return this; + } + public WorkerNode setWorkerCuratorCoordinator(WorkerCuratorCoordinator workerCuratorCoordinator) { this.workerCuratorCoordinator = workerCuratorCoordinator; @@ -175,10 +192,12 @@ public class WorkerNode extends RegisteringNode initializeS3Service(); initializeMonitors(); initializeMergerConfig(); + initializeCuratorFramework(); + initializeServiceDiscovery(); + initializeCoordinatorServiceProvider(); initializeTaskToolbox(); initializeJacksonInjections(); initializeJacksonSubtypes(); - initializeCuratorFramework(); initializeCuratorCoordinator(); initializeTaskMonitor(); initializeServer(); @@ -328,7 +347,7 @@ public class WorkerNode extends RegisteringNode ); taskToolbox = new TaskToolbox( taskConfig, - new RemoteTaskActionClient(httpClient, jsonMapper), + new RemoteTaskActionClient(httpClient, coordinatorServiceProvider, jsonMapper), emitter, s3Service, dataSegmentPusher, @@ -340,11 +359,36 @@ public class WorkerNode extends RegisteringNode public void initializeCuratorFramework() throws IOException { - final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class); - curatorFramework = Initialization.makeCuratorFrameworkClient( - curatorConfig, - lifecycle - ); + if (curatorFramework == null) { + final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class); + curatorFramework = Initialization.makeCuratorFrameworkClient( + curatorConfig, + lifecycle + ); + } + } + + public void initializeServiceDiscovery() throws Exception + { + if (serviceDiscovery == null) { + final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class); + this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( + curatorFramework, + config, + lifecycle + ); + } + } + + public void initializeCoordinatorServiceProvider() + { + if (coordinatorServiceProvider == null) { + this.coordinatorServiceProvider = Initialization.makeServiceProvider( + workerConfig.getMasterService(), + serviceDiscovery, + lifecycle + ); + } } public void initializeCuratorCoordinator() From ab456cf3b135ab83ee708863c5ad70e3b3c25c9b Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Tue, 26 Feb 2013 15:15:36 -0800 Subject: [PATCH 079/121] [maven-release-plugin] prepare release druid-0.3.4 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 3af4d9b489a..ccc386744f4 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4-SNAPSHOT + 0.3.4 diff --git a/common/pom.xml b/common/pom.xml index 03483c15caa..eca3ef61f84 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4-SNAPSHOT + 0.3.4 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index b5b2a7e460c..02a81b032ce 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.4-SNAPSHOT + 0.3.4 com.metamx druid - 0.3.4-SNAPSHOT + 0.3.4 diff --git a/examples/pom.xml b/examples/pom.xml index ae597675a62..3912eb4ceeb 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4-SNAPSHOT + 0.3.4 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index b9ff5296fde..8595f551d36 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.4-SNAPSHOT + 0.3.4 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 88f4beae7da..2bb5bc0965a 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.4-SNAPSHOT + 0.3.4 diff --git a/index-common/pom.xml b/index-common/pom.xml index 34264feb745..fca7c0f5c21 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4-SNAPSHOT + 0.3.4 diff --git a/indexer/pom.xml b/indexer/pom.xml index 778f682a0f2..81ec2ff1da6 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4-SNAPSHOT + 0.3.4 diff --git a/merger/pom.xml b/merger/pom.xml index e39359574a5..7cb71ae7e6a 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4-SNAPSHOT + 0.3.4 diff --git a/pom.xml b/pom.xml index 1f1afee50e3..f9a0e4bf8a7 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.4-SNAPSHOT + 0.3.4 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 0e48bc168c0..fd2d0c30262 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4-SNAPSHOT + 0.3.4 diff --git a/server/pom.xml b/server/pom.xml index e373da72f3d..650660525af 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4-SNAPSHOT + 0.3.4 From 2e1e1753d5a6c04e637dac8ccf13d4008966e5c3 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Tue, 26 Feb 2013 15:15:54 -0800 Subject: [PATCH 080/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index ccc386744f4..37333e4d612 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4 + 0.3.5-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index eca3ef61f84..2b3887bb50a 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4 + 0.3.5-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 02a81b032ce..ce20f26d1fc 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.4 + 0.3.5-SNAPSHOT com.metamx druid - 0.3.4 + 0.3.5-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 3912eb4ceeb..def4e6b7cd1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4 + 0.3.5-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 8595f551d36..3efcb0defbd 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.4 + 0.3.5-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 2bb5bc0965a..6cc864b9b5c 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.4 + 0.3.5-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index fca7c0f5c21..d0b6ac71a89 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4 + 0.3.5-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 81ec2ff1da6..7088500860e 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4 + 0.3.5-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 7cb71ae7e6a..85528bef2fa 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4 + 0.3.5-SNAPSHOT diff --git a/pom.xml b/pom.xml index f9a0e4bf8a7..5ddbfd74c0d 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.4 + 0.3.5-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index fd2d0c30262..ccde2d08307 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4 + 0.3.5-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 650660525af..bff22bca2a5 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.4 + 0.3.5-SNAPSHOT From d1634fe5efd0a8125061834a87c7b8de85f136f7 Mon Sep 17 00:00:00 2001 From: Nelson Ray Date: Tue, 26 Feb 2013 18:58:16 -0800 Subject: [PATCH 081/121] fix cost balancing so assignment/balancing doesn't get stuck --- .../metamx/druid/master/BalancerCostAnalyzer.java | 15 +++++++++++---- .../metamx/druid/master/DruidMasterBalancer.java | 6 ++++-- .../com/metamx/druid/master/rules/LoadRule.java | 2 +- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 6e7ca4d20a4..799bccca581 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -188,19 +188,26 @@ public class BalancerCostAnalyzer * A DataSegment that we are proposing to move. * @param serverHolders * An iterable of ServerHolders for a particular tier. + * @param assign + * A boolean that is true if used in assignment else false in balancing. * @return A ServerHolder with the new home for a segment. */ - public ServerHolder findNewSegmentHome(final DataSegment proposalSegment, final Iterable serverHolders) + public ServerHolder findNewSegmentHome( + final DataSegment proposalSegment, + final Iterable serverHolders, + final boolean assign + ) { final long proposalSegmentSize = proposalSegment.getSize(); double minCost = Double.MAX_VALUE; ServerHolder toServer = null; for (ServerHolder server : serverHolders) { - /** Don't calculate cost if the server doesn't have enough space or is serving/loading the segment. */ + /** Don't calculate cost if the server doesn't have enough space or is loading the segment */ if (proposalSegmentSize > server.getAvailableSize() - || server.isServingSegment(proposalSegment) - || server.isLoadingSegment(proposalSegment)) { + || server.isLoadingSegment(proposalSegment) + /** or if the ask is assignment and the server is serving the segment. */ + || (assign && server.isServingSegment(proposalSegment)) ) { continue; } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 809c1106a3c..3d66a21f9d7 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -111,7 +111,7 @@ public class DruidMasterBalancer implements DruidMasterHelper while (iter < maxSegmentsToMove) { iter++; final BalancerSegmentHolder segmentToMove = analyzer.pickSegmentToMove(serverHolderList, numSegments); - final ServerHolder holder = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList); + final ServerHolder holder = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList, false); if (holder == null) { continue; } @@ -128,11 +128,13 @@ public class DruidMasterBalancer implements DruidMasterHelper stats.addToTieredStat("initialCost", tier, (long) initialTotalCost); stats.addToTieredStat("normalization", tier, (long) normalization); + stats.addToTieredStat("normalizedInitialCostTimesOneThousand", tier, (long) (normalizedInitialCost * 1000)); stats.addToTieredStat("movedCount", tier, currentlyMovingSegments.get(tier).size()); log.info( - "Initial Total Cost: [%f], Initial Normalized Cost: [%f], Segments Moved: [%d]", + "Initial Total Cost: [%f], Normalization: [%f], Initial Normalized Cost: [%f], Segments Moved: [%d]", initialTotalCost, + normalization, normalizedInitialCost, currentlyMovingSegments.get(tier).size() ); diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index 4cd6acae640..a59d9c795b5 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -91,7 +91,7 @@ public abstract class LoadRule implements Rule final MasterStats stats = new MasterStats(); while (totalReplicants < expectedReplicants) { - final ServerHolder holder = analyzer.findNewSegmentHome(segment, serverHolderList); + final ServerHolder holder = analyzer.findNewSegmentHome(segment, serverHolderList, true); if (holder == null) { log.warn( From e40fba4de2bb32eb05a477708e1ec6abb4872912 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 27 Feb 2013 14:50:15 -0800 Subject: [PATCH 082/121] HadoopIndexTask: Jackson fixes and general overriding of storage-specific stuff --- .../indexer/HadoopDruidIndexerConfig.java | 4 +- .../merger/common/config/TaskConfig.java | 3 + .../merger/common/task/HadoopIndexTask.java | 43 ++++++++----- .../metamx/druid/merger/common/task/Task.java | 3 +- .../merger/common/task/TaskSerdeTest.java | 63 +++++++++++++++++-- .../coordinator/RemoteTaskRunnerTest.java | 6 ++ .../merger/coordinator/TaskLifecycleTest.java | 6 ++ 7 files changed, 105 insertions(+), 23 deletions(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index 41e812bba30..5dc45d19484 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -198,7 +198,7 @@ public class HadoopDruidIndexerConfig final @JsonProperty("pathSpec") PathSpec pathSpec, final @JsonProperty("workingPath") String jobOutputDir, final @JsonProperty("segmentOutputPath") String segmentOutputDir, - final @JsonProperty("version") DateTime version, + final @JsonProperty("version") String version, final @JsonProperty("partitionDimension") String partitionDimension, final @JsonProperty("targetPartitionSize") Long targetPartitionSize, final @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec, @@ -220,7 +220,7 @@ public class HadoopDruidIndexerConfig this.pathSpec = pathSpec; this.jobOutputDir = jobOutputDir; this.segmentOutputDir = segmentOutputDir; - this.version = version == null ? new DateTime().toString() : version.toString(); + this.version = version == null ? new DateTime().toString() : version; this.partitionsSpec = partitionsSpec; this.leaveIntermediate = leaveIntermediate; this.cleanupOnFailure = cleanupOnFailure; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java index 5b7609bd042..5918f0627c6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java @@ -15,6 +15,9 @@ public abstract class TaskConfig @Default("500000") public abstract int getDefaultRowFlushBoundary(); + @Config("druid.merger.hadoopWorkingPath") + public abstract String getHadoopWorkingPath(); + public File getTaskDir(final Task task) { return new File(getBaseTaskDir(), task.getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java index db25239b50c..29c0c517f17 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java @@ -1,6 +1,8 @@ package com.metamx.druid.merger.common.task; -import com.google.common.collect.ImmutableList; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.metamx.common.logger.Logger; @@ -14,20 +16,17 @@ import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.actions.LockListAction; import com.metamx.druid.merger.common.actions.SegmentInsertAction; import com.metamx.druid.utils.JodaUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; import org.joda.time.DateTime; import java.util.List; public class HadoopIndexTask extends AbstractTask { - @JsonProperty - private static final Logger log = new Logger(HadoopIndexTask.class); - @JsonProperty private final HadoopDruidIndexerConfig config; + private static final Logger log = new Logger(HadoopIndexTask.class); + /** * @param config is used by the HadoopDruidIndexerJob to set up the appropriate parameters * for creating Druid index segments. It may be modified. @@ -44,16 +43,16 @@ public class HadoopIndexTask extends AbstractTask ) { super( - String.format("index_hadoop_%s_interval_%s", config.getDataSource(), new DateTime()), + String.format("index_hadoop_%s_%s", config.getDataSource(), new DateTime()), config.getDataSource(), JodaUtils.umbrellaInterval(config.getIntervals()) ); - if (config.isUpdaterJobSpecSet()) { - throw new IllegalArgumentException( - "The UpdaterJobSpec field of the Hadoop Druid indexer config must be set to null" - ); - } + // Some HadoopDruidIndexerConfig stuff doesn't make sense in the context of the indexing service + Preconditions.checkArgument(config.getSegmentOutputDir() == null, "segmentOutputPath must be absent"); + Preconditions.checkArgument(config.getJobOutputDir() == null, "workingPath must be absent"); + Preconditions.checkArgument(!config.isUpdaterJobSpecSet(), "updaterJobSpec must be absent"); + this.config = config; } @@ -66,10 +65,21 @@ public class HadoopIndexTask extends AbstractTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { + // Copy config so we don't needlessly modify our provided one + // Also necessary to make constructor validations work upon serde-after-run + final HadoopDruidIndexerConfig configCopy = toolbox.getObjectMapper() + .readValue( + toolbox.getObjectMapper().writeValueAsBytes(config), + HadoopDruidIndexerConfig.class + ); + // We should have a lock from before we started running final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); log.info("Setting version to: %s", myLock.getVersion()); - config.setVersion(myLock.getVersion()); + configCopy.setVersion(myLock.getVersion()); + + // Set workingPath to some reasonable default + configCopy.setJobOutputDir(toolbox.getConfig().getHadoopWorkingPath()); if (toolbox.getSegmentPusher() instanceof S3DataSegmentPusher) { // Hack alert! Bypassing DataSegmentPusher... @@ -82,14 +92,15 @@ public class HadoopIndexTask extends AbstractTask ); log.info("Setting segment output path to: %s", s3Path); - config.setSegmentOutputDir(s3Path); + configCopy.setSegmentOutputDir(s3Path); } else { throw new IllegalStateException("Sorry, we only work with S3DataSegmentPushers! Bummer!"); } - HadoopDruidIndexerJob job = new HadoopDruidIndexerJob(config); - log.debug("Starting a hadoop index generator job..."); + HadoopDruidIndexerJob job = new HadoopDruidIndexerJob(configCopy); + configCopy.verify(); + log.info("Starting a hadoop index generator job..."); if (job.run()) { List publishedSegments = job.getPublishedSegments(); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 5ac4dcf71df..60a265564da 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -37,7 +37,8 @@ import org.joda.time.Interval; @JsonSubTypes.Type(name = "kill", value = KillTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), @JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class), - @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class) + @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class), + @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class) }) public interface Task { diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java index 51310bb2ef0..213bec34421 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java @@ -1,12 +1,17 @@ package com.metamx.druid.merger.common.task; +import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.metamx.common.Granularity; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.indexer.HadoopDruidIndexerConfig; +import com.metamx.druid.indexer.data.JSONDataSpec; import com.metamx.druid.indexer.granularity.UniformGranularitySpec; +import com.metamx.druid.indexer.path.StaticPathSpec; +import com.metamx.druid.indexer.rollup.DataRollupSpec; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; @@ -32,13 +37,17 @@ public class TaskSerdeTest final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); + + Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change final Task task2 = jsonMapper.readValue(json, Task.class); + Assert.assertEquals("foo", task.getDataSource()); + Assert.assertEquals(Optional.of(new Interval("2010-01-01/P2D")), task.getFixedInterval()); + Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); - Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); } @Test @@ -61,11 +70,13 @@ public class TaskSerdeTest final String json = jsonMapper.writeValueAsString(task); final Task task2 = jsonMapper.readValue(json, Task.class); + Assert.assertEquals("foo", task.getDataSource()); + Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getFixedInterval()); + Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); - Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); } @Test @@ -82,11 +93,13 @@ public class TaskSerdeTest final String json = jsonMapper.writeValueAsString(task); final Task task2 = jsonMapper.readValue(json, Task.class); + Assert.assertEquals("foo", task.getDataSource()); + Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getFixedInterval()); + Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); - Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); } @Test @@ -99,7 +112,6 @@ public class TaskSerdeTest final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); - System.out.println(json); final Task task2 = jsonMapper.readValue(json, Task.class); Assert.assertEquals(task.getId(), task2.getId()); @@ -108,4 +120,47 @@ public class TaskSerdeTest Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); } + + @Test + public void testHadoopIndexTaskSerde() throws Exception + { + final HadoopIndexTask task = new HadoopIndexTask( + new HadoopDruidIndexerConfig( + null, + "foo", + "timestamp", + "auto", + new JSONDataSpec(ImmutableList.of("foo")), + null, + new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P1D"))), + new StaticPathSpec("bar"), + null, + null, + null, + null, + null, + null, + false, + true, + null, + false, + new DataRollupSpec(ImmutableList.of(), QueryGranularity.NONE), + null, + false, + ImmutableList.of() + ) + ); + + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final String json = jsonMapper.writeValueAsString(task); + final Task task2 = jsonMapper.readValue(json, Task.class); + + Assert.assertEquals("foo", task.getDataSource()); + Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getFixedInterval()); + + Assert.assertEquals(task.getId(), task2.getId()); + Assert.assertEquals(task.getGroupId(), task2.getGroupId()); + Assert.assertEquals(task.getDataSource(), task2.getDataSource()); + Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + } } 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 60309f1e368..6d48ce17df9 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 @@ -300,6 +300,12 @@ public class RemoteTaskRunnerTest { return 0; } + + @Override + public String getHadoopWorkingPath() + { + return null; + } }, null, null, null, null, null, jsonMapper ), Executors.newSingleThreadExecutor() diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java index 69a364e1900..ae5b46fdfc1 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java @@ -110,6 +110,12 @@ public class TaskLifecycleTest { return 50000; } + + @Override + public String getHadoopWorkingPath() + { + return null; + } }, new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())), newMockEmitter(), From 26dca5630f97e44b50a29064b5a8430563a1e5da Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 1 Mar 2013 06:59:23 -0800 Subject: [PATCH 083/121] QueryServlet: Send alerts with remote address as "peer" instead of "host" "host" is used by the emitter. Both end up being emitted, which can be confusing. --- client/src/main/java/com/metamx/druid/http/QueryServlet.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/http/QueryServlet.java b/client/src/main/java/com/metamx/druid/http/QueryServlet.java index 9a13c50f45c..bed59a8b525 100644 --- a/client/src/main/java/com/metamx/druid/http/QueryServlet.java +++ b/client/src/main/java/com/metamx/druid/http/QueryServlet.java @@ -150,7 +150,7 @@ public class QueryServlet extends HttpServlet ImmutableMap.builder() .put("exception", e.toString()) .put("query", queryString) - .put("host", req.getRemoteAddr()) + .put("peer", req.getRemoteAddr()) .build() ) ); From fb874580564ad980c9a4febba3efee011d3fcf9d Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 11:41:28 -0800 Subject: [PATCH 084/121] cleaner code for dimension/metric exploration on broker --- .../metamx/druid/http/ClientInfoResource.java | 110 ++++++------------ 1 file changed, 34 insertions(+), 76 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/http/ClientInfoResource.java b/client/src/main/java/com/metamx/druid/http/ClientInfoResource.java index 50a0475de4b..23a587d351e 100644 --- a/client/src/main/java/com/metamx/druid/http/ClientInfoResource.java +++ b/client/src/main/java/com/metamx/druid/http/ClientInfoResource.java @@ -24,11 +24,11 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.inject.Inject; -import com.metamx.common.IAE; import com.metamx.druid.client.ClientInventoryManager; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; +import org.joda.time.DateTime; import org.joda.time.Interval; import javax.ws.rs.GET; @@ -36,7 +36,7 @@ import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; -import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; @@ -57,22 +57,26 @@ public class ClientInfoResource this.clientInventoryManager = clientInventoryManager; } - private Map updateDataSources() + private Map> getSegmentsForDatasources() { - final Map dataSources = Maps.newHashMap(); + final Map> dataSourceMap = Maps.newHashMap(); for (DruidServer server : clientInventoryManager.getInventory()) { for (DruidDataSource dataSource : server.getDataSources()) { - dataSources.put(dataSource.getName(), dataSource); + if (!dataSourceMap.containsKey(dataSource.getName())) { + dataSourceMap.put(dataSource.getName(), Lists.newArrayList()); + } + List segments = dataSourceMap.get(dataSource.getName()); + segments.addAll(dataSource.getSegments()); } } - return dataSources; + return dataSourceMap; } @GET @Produces("application/json") public Iterable getDataSources() { - return updateDataSources().keySet(); + return getSegmentsForDatasources().keySet(); } @GET @@ -80,7 +84,7 @@ public class ClientInfoResource @Produces("application/json") public Map getDatasource( @PathParam("dataSourceName") String dataSourceName, - @QueryParam("interval") String interval + @QueryParam("interval") Interval interval ) { return ImmutableMap.of( @@ -94,47 +98,24 @@ public class ClientInfoResource @Produces("application/json") public Iterable getDatasourceDimensions( @PathParam("dataSourceName") String dataSourceName, - @QueryParam("interval") String interval + @QueryParam("interval") Interval interval ) { - DruidDataSource dataSource = updateDataSources().get(dataSourceName); + List segments = getSegmentsForDatasources().get(dataSourceName); - Set retVal = Sets.newHashSet(); + if (interval == null) { + DateTime now = new DateTime(); + interval = new Interval(now.minusMillis(SEGMENT_HISTORY_MILLIS), now); + } - Interval dimInterval; - if (interval == null || interval.isEmpty()) { - Iterator iter = Lists.reverse(Lists.newArrayList(dataSource.getSegments())).iterator(); - DataSegment segment = iter.next(); - retVal.addAll(segment.getDimensions()); - - dimInterval = new Interval( - segment.getInterval().getEnd().minus(SEGMENT_HISTORY_MILLIS), - segment.getInterval().getEnd() - ); - - while (iter.hasNext() && dimInterval.contains(segment.getInterval())) { - retVal.addAll(segment.getDimensions()); - segment = iter.next(); - } - } else { - try { - dimInterval = new Interval(interval); - } - catch (Exception e) { - throw new IAE("Interval is not in a parseable format!"); - } - - Iterator iter = dataSource.getSegments().iterator(); - - while (iter.hasNext()) { - DataSegment segment = iter.next(); - if (dimInterval.contains(segment.getInterval())) { - retVal.addAll(segment.getDimensions()); - } + Set dims = Sets.newHashSet(); + for (DataSegment segment : segments) { + if (interval.overlaps(segment.getInterval())) { + dims.addAll(segment.getDimensions()); } } - return retVal; + return dims; } @GET @@ -142,46 +123,23 @@ public class ClientInfoResource @Produces("application/json") public Iterable getDatasourceMetrics( @PathParam("dataSourceName") String dataSourceName, - @QueryParam("interval") String interval + @QueryParam("interval") Interval interval ) { - DruidDataSource dataSource = updateDataSources().get(dataSourceName); + List segments = getSegmentsForDatasources().get(dataSourceName); - Set retVal = Sets.newHashSet(); + if (interval == null) { + DateTime now = new DateTime(); + interval = new Interval(now.minusMillis(SEGMENT_HISTORY_MILLIS), now); + } - Interval dimInterval; - if (interval == null || interval.isEmpty()) { - Iterator iter = Lists.reverse(Lists.newArrayList(dataSource.getSegments())).iterator(); - DataSegment segment = iter.next(); - retVal.addAll(segment.getMetrics()); - - dimInterval = new Interval( - segment.getInterval().getEnd().minus(SEGMENT_HISTORY_MILLIS), - segment.getInterval().getEnd() - ); - - while (iter.hasNext() && dimInterval.contains(segment.getInterval())) { - retVal.addAll(segment.getMetrics()); - segment = iter.next(); - } - } else { - try { - dimInterval = new Interval(interval); - } - catch (Exception e) { - throw new IAE("Interval is not in a parseable format!"); - } - - Iterator iter = dataSource.getSegments().iterator(); - - while (iter.hasNext()) { - DataSegment segment = iter.next(); - if (dimInterval.contains(segment.getInterval())) { - retVal.addAll(segment.getMetrics()); - } + Set metrics = Sets.newHashSet(); + for (DataSegment segment : segments) { + if (interval.overlaps(segment.getInterval())) { + metrics.addAll(segment.getMetrics()); } } - return retVal; + return metrics; } } From 334fffdef52dd0fc467bacc4f0648fe88038660b Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 11:43:16 -0800 Subject: [PATCH 085/121] [maven-release-plugin] prepare release druid-0.3.5 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 37333e4d612..04fc3c4f54e 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5-SNAPSHOT + 0.3.5 diff --git a/common/pom.xml b/common/pom.xml index 2b3887bb50a..461c1677459 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5-SNAPSHOT + 0.3.5 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index ce20f26d1fc..ed2acfb3979 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.5-SNAPSHOT + 0.3.5 com.metamx druid - 0.3.5-SNAPSHOT + 0.3.5 diff --git a/examples/pom.xml b/examples/pom.xml index def4e6b7cd1..e208e7d4e82 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5-SNAPSHOT + 0.3.5 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 3efcb0defbd..ade76c98472 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.5-SNAPSHOT + 0.3.5 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 6cc864b9b5c..3ef9d898293 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.5-SNAPSHOT + 0.3.5 diff --git a/index-common/pom.xml b/index-common/pom.xml index d0b6ac71a89..8cf733054f2 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5-SNAPSHOT + 0.3.5 diff --git a/indexer/pom.xml b/indexer/pom.xml index 7088500860e..5cd6eab9f41 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5-SNAPSHOT + 0.3.5 diff --git a/merger/pom.xml b/merger/pom.xml index 85528bef2fa..d4c0e5ab9d3 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5-SNAPSHOT + 0.3.5 diff --git a/pom.xml b/pom.xml index 5ddbfd74c0d..afe3e5fae79 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.5-SNAPSHOT + 0.3.5 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index ccde2d08307..0071bcf3b76 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5-SNAPSHOT + 0.3.5 diff --git a/server/pom.xml b/server/pom.xml index bff22bca2a5..16ae176ec71 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5-SNAPSHOT + 0.3.5 From 3264a689f1867b6fe2ed384b43c4a76a66c48831 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 11:43:23 -0800 Subject: [PATCH 086/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 04fc3c4f54e..bd163fbe855 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5 + 0.3.6-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 461c1677459..608d9059979 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5 + 0.3.6-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index ed2acfb3979..741b009454c 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.5 + 0.3.6-SNAPSHOT com.metamx druid - 0.3.5 + 0.3.6-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index e208e7d4e82..e3b21061cdd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5 + 0.3.6-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index ade76c98472..8cef0cb47be 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.5 + 0.3.6-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 3ef9d898293..06da7487fa4 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.5 + 0.3.6-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 8cf733054f2..f62c20d9a22 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5 + 0.3.6-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 5cd6eab9f41..8c351d28b14 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5 + 0.3.6-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index d4c0e5ab9d3..148782ee03f 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5 + 0.3.6-SNAPSHOT diff --git a/pom.xml b/pom.xml index afe3e5fae79..7794be8274b 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.5 + 0.3.6-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 0071bcf3b76..7b801ede463 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5 + 0.3.6-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 16ae176ec71..979976ff4e1 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.5 + 0.3.6-SNAPSHOT From 7f32629429021f8c693fd77a132a1aaadbfbe850 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 13:53:27 -0800 Subject: [PATCH 087/121] fix a bug with client info resource with guice --- .../metamx/druid/http/ClientInfoResource.java | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/http/ClientInfoResource.java b/client/src/main/java/com/metamx/druid/http/ClientInfoResource.java index 23a587d351e..8bf344e88bb 100644 --- a/client/src/main/java/com/metamx/druid/http/ClientInfoResource.java +++ b/client/src/main/java/com/metamx/druid/http/ClientInfoResource.java @@ -84,7 +84,7 @@ public class ClientInfoResource @Produces("application/json") public Map getDatasource( @PathParam("dataSourceName") String dataSourceName, - @QueryParam("interval") Interval interval + @QueryParam("interval") String interval ) { return ImmutableMap.of( @@ -98,19 +98,22 @@ public class ClientInfoResource @Produces("application/json") public Iterable getDatasourceDimensions( @PathParam("dataSourceName") String dataSourceName, - @QueryParam("interval") Interval interval + @QueryParam("interval") String interval ) { List segments = getSegmentsForDatasources().get(dataSourceName); - if (interval == null) { + Interval theInterval; + if (interval == null || interval.isEmpty()) { DateTime now = new DateTime(); - interval = new Interval(now.minusMillis(SEGMENT_HISTORY_MILLIS), now); + theInterval = new Interval(now.minusMillis(SEGMENT_HISTORY_MILLIS), now); + } else { + theInterval = new Interval(interval); } Set dims = Sets.newHashSet(); for (DataSegment segment : segments) { - if (interval.overlaps(segment.getInterval())) { + if (theInterval.overlaps(segment.getInterval())) { dims.addAll(segment.getDimensions()); } } @@ -123,19 +126,22 @@ public class ClientInfoResource @Produces("application/json") public Iterable getDatasourceMetrics( @PathParam("dataSourceName") String dataSourceName, - @QueryParam("interval") Interval interval + @QueryParam("interval") String interval ) { List segments = getSegmentsForDatasources().get(dataSourceName); - if (interval == null) { + Interval theInterval; + if (interval == null || interval.isEmpty()) { DateTime now = new DateTime(); - interval = new Interval(now.minusMillis(SEGMENT_HISTORY_MILLIS), now); + theInterval = new Interval(now.minusMillis(SEGMENT_HISTORY_MILLIS), now); + } else { + theInterval = new Interval(interval); } Set metrics = Sets.newHashSet(); for (DataSegment segment : segments) { - if (interval.overlaps(segment.getInterval())) { + if (theInterval.overlaps(segment.getInterval())) { metrics.addAll(segment.getMetrics()); } } From a43dc44ebb40daf6f73f94c11e7f181d0c7366f3 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 13:55:01 -0800 Subject: [PATCH 088/121] [maven-release-plugin] prepare release druid-0.3.6 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index bd163fbe855..e3f001315f8 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6-SNAPSHOT + 0.3.6 diff --git a/common/pom.xml b/common/pom.xml index 608d9059979..97e377a3617 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6-SNAPSHOT + 0.3.6 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 741b009454c..7ef359fa68b 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.6-SNAPSHOT + 0.3.6 com.metamx druid - 0.3.6-SNAPSHOT + 0.3.6 diff --git a/examples/pom.xml b/examples/pom.xml index e3b21061cdd..eeb8968b01d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6-SNAPSHOT + 0.3.6 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 8cef0cb47be..ab3822f780a 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.6-SNAPSHOT + 0.3.6 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 06da7487fa4..37265b2fde2 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.6-SNAPSHOT + 0.3.6 diff --git a/index-common/pom.xml b/index-common/pom.xml index f62c20d9a22..1e1a4050113 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6-SNAPSHOT + 0.3.6 diff --git a/indexer/pom.xml b/indexer/pom.xml index 8c351d28b14..866a6cba205 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6-SNAPSHOT + 0.3.6 diff --git a/merger/pom.xml b/merger/pom.xml index 148782ee03f..576b412d3d1 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6-SNAPSHOT + 0.3.6 diff --git a/pom.xml b/pom.xml index 7794be8274b..db83c96fa65 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.6-SNAPSHOT + 0.3.6 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 7b801ede463..116b6d10dcc 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6-SNAPSHOT + 0.3.6 diff --git a/server/pom.xml b/server/pom.xml index 979976ff4e1..9b122c64a80 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6-SNAPSHOT + 0.3.6 From 360c6a1df12039be4a84ab80a624c69b5e757608 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 13:55:07 -0800 Subject: [PATCH 089/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index e3f001315f8..990c822e16b 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6 + 0.3.7-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 97e377a3617..e5216314758 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6 + 0.3.7-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 7ef359fa68b..6d82feaa726 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.6 + 0.3.7-SNAPSHOT com.metamx druid - 0.3.6 + 0.3.7-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index eeb8968b01d..11ac68e42c0 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6 + 0.3.7-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index ab3822f780a..139b456a801 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.6 + 0.3.7-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 37265b2fde2..c99887267f7 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.6 + 0.3.7-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 1e1a4050113..133b9566afc 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6 + 0.3.7-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 866a6cba205..c477854706a 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6 + 0.3.7-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 576b412d3d1..8931ae8b7c5 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6 + 0.3.7-SNAPSHOT diff --git a/pom.xml b/pom.xml index db83c96fa65..ee0e0a85493 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.6 + 0.3.7-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 116b6d10dcc..af6c347b93e 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6 + 0.3.7-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 9b122c64a80..b7a666a4955 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.6 + 0.3.7-SNAPSHOT From 6b4f6bffcee57b2e4b8154b6618e08c60195351d Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 1 Mar 2013 14:40:29 -0800 Subject: [PATCH 090/121] DbTaskStorage: Fix getAuditLogs --- .../java/com/metamx/druid/merger/coordinator/DbTaskStorage.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java index 4bbf0f37a32..5f39efae0bb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java @@ -379,7 +379,7 @@ public class DbTaskStorage implements TaskStorage public TaskAction apply(Map row) { try { - return jsonMapper.readValue(row.get("payload").toString(), TaskAction.class); + return jsonMapper.readValue(row.get("log_payload").toString(), TaskAction.class); } catch(Exception e) { throw Throwables.propagate(e); } From 89d5ddf2f79e6f8b54db0af761928e63e08be34e Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 1 Mar 2013 14:43:13 -0800 Subject: [PATCH 091/121] [maven-release-plugin] prepare release druid-0.3.7 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 990c822e16b..0a8d4a2d4b8 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7-SNAPSHOT + 0.3.7 diff --git a/common/pom.xml b/common/pom.xml index e5216314758..c484ed0a889 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7-SNAPSHOT + 0.3.7 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 6d82feaa726..bbec593a184 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.7-SNAPSHOT + 0.3.7 com.metamx druid - 0.3.7-SNAPSHOT + 0.3.7 diff --git a/examples/pom.xml b/examples/pom.xml index 11ac68e42c0..781dd8f8324 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7-SNAPSHOT + 0.3.7 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 139b456a801..2cb1b616662 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.7-SNAPSHOT + 0.3.7 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index c99887267f7..e6740fdddc7 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.7-SNAPSHOT + 0.3.7 diff --git a/index-common/pom.xml b/index-common/pom.xml index 133b9566afc..feda7500613 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7-SNAPSHOT + 0.3.7 diff --git a/indexer/pom.xml b/indexer/pom.xml index c477854706a..5e054f4ee9e 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7-SNAPSHOT + 0.3.7 diff --git a/merger/pom.xml b/merger/pom.xml index 8931ae8b7c5..ad4bd4975a1 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7-SNAPSHOT + 0.3.7 diff --git a/pom.xml b/pom.xml index ee0e0a85493..b57eccf9d2f 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.7-SNAPSHOT + 0.3.7 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index af6c347b93e..f341745cc23 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7-SNAPSHOT + 0.3.7 diff --git a/server/pom.xml b/server/pom.xml index b7a666a4955..d9c2c811022 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7-SNAPSHOT + 0.3.7 From 998ed0a8dea05a401291d1080863b89ba802cb04 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 1 Mar 2013 14:43:20 -0800 Subject: [PATCH 092/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 0a8d4a2d4b8..5992b20d9a0 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7 + 0.3.8-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index c484ed0a889..6ac75e63215 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7 + 0.3.8-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index bbec593a184..628edb70eda 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.7 + 0.3.8-SNAPSHOT com.metamx druid - 0.3.7 + 0.3.8-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 781dd8f8324..09a29085107 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7 + 0.3.8-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 2cb1b616662..e5b18f83e9e 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.7 + 0.3.8-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index e6740fdddc7..870fc5ad16c 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.7 + 0.3.8-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index feda7500613..e2cf86856bb 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7 + 0.3.8-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 5e054f4ee9e..1ae4ad2e323 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7 + 0.3.8-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index ad4bd4975a1..6107d649f3c 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7 + 0.3.8-SNAPSHOT diff --git a/pom.xml b/pom.xml index b57eccf9d2f..f5877ca1f97 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.7 + 0.3.8-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index f341745cc23..34cc1525fd3 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7 + 0.3.8-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index d9c2c811022..2b3f645103a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.7 + 0.3.8-SNAPSHOT From 53021c2e35b02415cb23d844df796013e3b59657 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 16:53:49 -0800 Subject: [PATCH 093/121] more logging --- .../metamx/druid/merger/coordinator/RemoteTaskRunner.java | 3 ++- .../druid/merger/coordinator/TaskMasterLifecycle.java | 1 + .../coordinator/scaling/ResourceManagementScheduler.java | 4 ++++ pom.xml | 8 ++++---- 4 files changed, 11 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 06dd63730cc..12a07979489 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 @@ -414,8 +414,9 @@ public class RemoteTaskRunner implements TaskRunner @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { - try { + log.info("Event[%s]: %s", event.getType(), event.getData().getPath()); + if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java index b9fefecf013..0eb3a335bc4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java @@ -100,6 +100,7 @@ public class TaskMasterLifecycle leaderLifecycle.addManagedInstance(taskRunner); Initialization.makeServiceDiscoveryClient(curator, serviceDiscoveryConfig, leaderLifecycle); leaderLifecycle.addManagedInstance(taskConsumer); + leaderLifecycle.addManagedInstance(scheduler); leading = true; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java index 45c0060fda6..2473e9d6e41 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java @@ -71,6 +71,8 @@ public class ResourceManagementScheduler return; } + log.info("Started Resource Management Scheduler"); + ScheduledExecutors.scheduleAtFixedRate( exec, config.getProvisionResourcesDuration(), @@ -79,6 +81,7 @@ public class ResourceManagementScheduler @Override public void run() { + log.info("Running resource management strategy"); resourceManagementStrategy.doProvision( taskRunner.getPendingTasks(), taskRunner.getWorkers() @@ -123,6 +126,7 @@ public class ResourceManagementScheduler if (!started) { return; } + log.info("Stopping Resource Management Scheduler"); exec.shutdown(); started = false; } diff --git a/pom.xml b/pom.xml index ee0e0a85493..d14760ba5c1 100644 --- a/pom.xml +++ b/pom.xml @@ -115,22 +115,22 @@ com.netflix.curator curator-client - 1.2.2 + 1.3.2 com.netflix.curator curator-framework - 1.2.2 + 1.3.2 com.netflix.curator curator-recipes - 1.2.2 + 1.3.2 com.netflix.curator curator-x-discovery - 1.2.2 + 1.3.2 com.netflix.curator From 03ff72b7e25c1230a9bf3c8d3a42bdc1503b4ca5 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 1 Mar 2013 16:57:38 -0800 Subject: [PATCH 094/121] [maven-release-plugin] prepare release druid-0.3.8 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 5992b20d9a0..4b466a35001 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8-SNAPSHOT + 0.3.8 diff --git a/common/pom.xml b/common/pom.xml index 6ac75e63215..e937f001e7f 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8-SNAPSHOT + 0.3.8 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 628edb70eda..eccbc6edbb0 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.8-SNAPSHOT + 0.3.8 com.metamx druid - 0.3.8-SNAPSHOT + 0.3.8 diff --git a/examples/pom.xml b/examples/pom.xml index 09a29085107..90f5a795068 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8-SNAPSHOT + 0.3.8 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index e5b18f83e9e..aa38cbb03e6 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.8-SNAPSHOT + 0.3.8 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 870fc5ad16c..174daf92429 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.8-SNAPSHOT + 0.3.8 diff --git a/index-common/pom.xml b/index-common/pom.xml index e2cf86856bb..26526450329 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8-SNAPSHOT + 0.3.8 diff --git a/indexer/pom.xml b/indexer/pom.xml index 1ae4ad2e323..cff2c7524c3 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8-SNAPSHOT + 0.3.8 diff --git a/merger/pom.xml b/merger/pom.xml index 6107d649f3c..9e7a7fefa79 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8-SNAPSHOT + 0.3.8 diff --git a/pom.xml b/pom.xml index a99dd1da497..ac2c3b5920f 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.8-SNAPSHOT + 0.3.8 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 34cc1525fd3..f99f3c4f420 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8-SNAPSHOT + 0.3.8 diff --git a/server/pom.xml b/server/pom.xml index 2b3f645103a..79368e91900 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8-SNAPSHOT + 0.3.8 From 0092ad7efb8ff5f64149fb897a86b418b66bd6f7 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 1 Mar 2013 16:57:47 -0800 Subject: [PATCH 095/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 4b466a35001..4166757987c 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8 + 0.3.9-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index e937f001e7f..85aaf813044 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8 + 0.3.9-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index eccbc6edbb0..27eff05c7f6 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.8 + 0.3.9-SNAPSHOT com.metamx druid - 0.3.8 + 0.3.9-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 90f5a795068..e4108e9a825 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8 + 0.3.9-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index aa38cbb03e6..ac8531a5869 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.8 + 0.3.9-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 174daf92429..68568df0359 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.8 + 0.3.9-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 26526450329..4f400d6fa6f 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8 + 0.3.9-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index cff2c7524c3..bf35674189d 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8 + 0.3.9-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 9e7a7fefa79..e0576a28b25 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8 + 0.3.9-SNAPSHOT diff --git a/pom.xml b/pom.xml index ac2c3b5920f..b608e25a0f3 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.8 + 0.3.9-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index f99f3c4f420..18b4c242a49 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8 + 0.3.9-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 79368e91900..207d9f87ec6 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.8 + 0.3.9-SNAPSHOT From 375e342d5fd1786a138afa14ca52232d98ea09da Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 18:00:02 -0800 Subject: [PATCH 096/121] remove deadlock problem in pending tasks --- .../merger/coordinator/RemoteTaskRunner.java | 29 +++++++------------ 1 file changed, 11 insertions(+), 18 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 12a07979489..8bd5f33bd48 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 @@ -241,35 +241,28 @@ public class RemoteTaskRunner implements TaskRunner */ private void runPendingTasks() { - Future future = runPendingTasksExec.submit( + runPendingTasksExec.submit( new Callable() { @Override public Void call() throws Exception { - // make a copy of the pending tasks because assignTask may delete tasks from pending and move them - // into running status - List copy = Lists.newArrayList(pendingTasks.values()); - for (TaskRunnerWorkItem taskWrapper : copy) { - assignTask(taskWrapper); + try { + // make a copy of the pending tasks because assignTask may delete tasks from pending and move them + // into running status + List copy = Lists.newArrayList(pendingTasks.values()); + for (TaskRunnerWorkItem taskWrapper : copy) { + assignTask(taskWrapper); + } + } + catch (Exception e) { + log.makeAlert(e, "Exception in running pending tasks").emit(); } return null; } } ); - - try { - future.get(); - } - catch (InterruptedException e) { - log.error(e, "InterruptedException in runPendingTasks()"); - throw Throwables.propagate(e); - } - catch (ExecutionException e) { - log.error(e, "ExecutionException in runPendingTasks()"); - throw Throwables.propagate(e.getCause()); - } } /** From a574da55ed0dc1892be00d12bb5e4424e2499756 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 18:22:30 -0800 Subject: [PATCH 097/121] fix broken UT --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 +- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- .../merger/coordinator/RemoteTaskRunner.java | 4 +- .../coordinator/RemoteTaskRunnerTest.java | 57 +++++++++---------- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 14 files changed, 43 insertions(+), 44 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 4166757987c..5c2b2d88e76 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 85aaf813044..4e0e2894664 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 27eff05c7f6..7c2bf45a3c0 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT com.metamx druid - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index e4108e9a825..47110d1ff03 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index ac8531a5869..c7e80542775 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 68568df0359..5e3157fb62f 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 4f400d6fa6f..3761f082e5d 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index bf35674189d..f8305ae4502 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index e0576a28b25..14f681a7c90 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT 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 8bd5f33bd48..21b43d7ae2b 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 @@ -408,10 +408,10 @@ public class RemoteTaskRunner implements TaskRunner public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { try { - log.info("Event[%s]: %s", event.getType(), event.getData().getPath()); - if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { + log.info("Event[%s]: %s", event.getType(), event.getData().getPath()); + final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); final TaskStatus taskStatus; 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 6d48ce17df9..9242e19f355 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 @@ -21,6 +21,8 @@ import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.service.ServiceEmitter; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.CuratorFrameworkFactory; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; @@ -158,35 +160,32 @@ public class RemoteTaskRunnerTest @Test public void testRunTooMuchZKData() throws Exception { - boolean exceptionOccurred = false; - try { - remoteTaskRunner.run( - new TestTask( - new String(new char[5000]), - "dummyDs", - Lists.newArrayList( - new DataSegment( - "dummyDs", - new Interval(new DateTime(), new DateTime()), - new DateTime().toString(), - null, - null, - null, - null, - 0, - 0 - ) - ), - Lists.newArrayList(), - TaskStatus.success("foo") - ), - null - ); - } - catch (IllegalStateException e) { - exceptionOccurred = true; - } - Assert.assertTrue(exceptionOccurred); + ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class); + EmittingLogger.registerEmitter(emitter); + EasyMock.replay(emitter); + remoteTaskRunner.run( + new TestTask( + new String(new char[5000]), + "dummyDs", + Lists.newArrayList( + new DataSegment( + "dummyDs", + new Interval(new DateTime(), new DateTime()), + new DateTime().toString(), + null, + null, + null, + null, + 0, + 0 + ) + ), + Lists.newArrayList(), + TaskStatus.success("foo") + ), + null + ); + EasyMock.verify(emitter); } @Test diff --git a/pom.xml b/pom.xml index b608e25a0f3..54633c7421e 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 18b4c242a49..dbdfd35634b 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 207d9f87ec6..8daf4abee25 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.9-SNAPSHOT + 0.3.10-SNAPSHOT From 023199702e71d5209d735335d3ab0640ada75736 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 18:24:03 -0800 Subject: [PATCH 098/121] [maven-release-plugin] prepare release druid-0.3.10 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 5c2b2d88e76..d82b354a1d6 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10-SNAPSHOT + 0.3.10 diff --git a/common/pom.xml b/common/pom.xml index 4e0e2894664..80b934f3452 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10-SNAPSHOT + 0.3.10 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 7c2bf45a3c0..9cc128e4fc8 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.10-SNAPSHOT + 0.3.10 com.metamx druid - 0.3.10-SNAPSHOT + 0.3.10 diff --git a/examples/pom.xml b/examples/pom.xml index 47110d1ff03..1dd679ca796 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10-SNAPSHOT + 0.3.10 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index c7e80542775..1a62ca55b6b 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.10-SNAPSHOT + 0.3.10 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 5e3157fb62f..7842592d637 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.10-SNAPSHOT + 0.3.10 diff --git a/index-common/pom.xml b/index-common/pom.xml index 3761f082e5d..ae2d8080e39 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10-SNAPSHOT + 0.3.10 diff --git a/indexer/pom.xml b/indexer/pom.xml index f8305ae4502..8a302332930 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10-SNAPSHOT + 0.3.10 diff --git a/merger/pom.xml b/merger/pom.xml index 14f681a7c90..3193d7fad7e 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10-SNAPSHOT + 0.3.10 diff --git a/pom.xml b/pom.xml index 54633c7421e..f2b59d663ef 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.10-SNAPSHOT + 0.3.10 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index dbdfd35634b..320e10734b5 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10-SNAPSHOT + 0.3.10 diff --git a/server/pom.xml b/server/pom.xml index 8daf4abee25..db624011f11 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10-SNAPSHOT + 0.3.10 From e2b4163e3141fca33c4cb53389f6bc9f77c9c362 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 18:24:09 -0800 Subject: [PATCH 099/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index d82b354a1d6..29b501b8525 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10 + 0.3.11-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 80b934f3452..bab9e7399f2 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10 + 0.3.11-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 9cc128e4fc8..9fb4e23fed1 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.10 + 0.3.11-SNAPSHOT com.metamx druid - 0.3.10 + 0.3.11-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 1dd679ca796..96fae605ee9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10 + 0.3.11-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 1a62ca55b6b..167cd8f46b3 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.10 + 0.3.11-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 7842592d637..f17cbe442fd 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.10 + 0.3.11-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index ae2d8080e39..c10af038347 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10 + 0.3.11-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 8a302332930..55921dc261e 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10 + 0.3.11-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 3193d7fad7e..c51383f49f7 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10 + 0.3.11-SNAPSHOT diff --git a/pom.xml b/pom.xml index f2b59d663ef..893bb33d140 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.10 + 0.3.11-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 320e10734b5..672b0f42805 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10 + 0.3.11-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index db624011f11..775756c4cbe 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.10 + 0.3.11-SNAPSHOT From 0b054cdd3d712e895e01e015aeb61026e29b0ccc Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 18:43:40 -0800 Subject: [PATCH 100/121] better logging for indexing service --- .../com/metamx/druid/merger/coordinator/ZkWorker.java | 2 -- .../scaling/ResourceManagementScheduler.java | 1 - .../java/com/metamx/druid/merger/worker/Worker.java | 11 +++++++++++ 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java index ff5ad95bf04..d500fe7c0a0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java @@ -112,8 +112,6 @@ public class ZkWorker implements Closeable { return "ZkWorker{" + "worker=" + worker + - ", statusCache=" + statusCache + - ", cacheConverter=" + cacheConverter + ", lastCompletedTaskTime=" + lastCompletedTaskTime + '}'; } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java index 2473e9d6e41..b3ea474cc0c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java @@ -81,7 +81,6 @@ public class ResourceManagementScheduler @Override public void run() { - log.info("Running resource management strategy"); resourceManagementStrategy.doProvision( taskRunner.getPendingTasks(), taskRunner.getWorkers() diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java b/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java index 6b349de38f7..e8eaac824b3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java @@ -84,4 +84,15 @@ public class Worker { return version; } + + @Override + public String toString() + { + return "Worker{" + + "host='" + host + '\'' + + ", ip='" + ip + '\'' + + ", capacity=" + capacity + + ", version='" + version + '\'' + + '}'; + } } From 665d1723c0a7a45e045a6a28ec1664dca67c747e Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 1 Mar 2013 18:48:34 -0800 Subject: [PATCH 101/121] Add TaskLockPosse#toString --- .../metamx/druid/merger/coordinator/TaskLockbox.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java index b62bca604d4..5e972b18805 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java @@ -1,6 +1,7 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Function; +import com.google.common.base.Objects; import com.google.common.base.Optional; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; @@ -401,5 +402,14 @@ public class TaskLockbox { return taskIds; } + + @Override + public String toString() + { + return Objects.toStringHelper(this) + .add("taskLock", taskLock) + .add("taskIds", taskIds) + .toString(); + } } } From df532f021a04c374fb972e1e82e0ee45c73437ef Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:03:16 -0800 Subject: [PATCH 102/121] more logging --- .../com/metamx/druid/merger/coordinator/RemoteTaskRunner.java | 1 + 1 file changed, 1 insertion(+) 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 21b43d7ae2b..9c36eecf6e9 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 @@ -472,6 +472,7 @@ public class RemoteTaskRunner implements TaskRunner runPendingTasks(); } } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { + log.info("Event[%s]: %s", event.getType(), event.getData().getPath()); final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); if (runningTasks.containsKey(taskId)) { log.info("Task %s just disappeared!", taskId); From 7e316e350888e57ef3785436ced8f9cfac2db0a5 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:05:39 -0800 Subject: [PATCH 103/121] [maven-release-plugin] prepare release druid-0.3.11 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 29b501b8525..5c3dc83b101 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11-SNAPSHOT + 0.3.11 diff --git a/common/pom.xml b/common/pom.xml index bab9e7399f2..a128c364d29 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11-SNAPSHOT + 0.3.11 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 9fb4e23fed1..99d9bfee95c 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.11-SNAPSHOT + 0.3.11 com.metamx druid - 0.3.11-SNAPSHOT + 0.3.11 diff --git a/examples/pom.xml b/examples/pom.xml index 96fae605ee9..612a5729b4d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11-SNAPSHOT + 0.3.11 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 167cd8f46b3..f5db26ed0e8 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.11-SNAPSHOT + 0.3.11 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index f17cbe442fd..b7276f928a7 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.11-SNAPSHOT + 0.3.11 diff --git a/index-common/pom.xml b/index-common/pom.xml index c10af038347..396a8bd81e5 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11-SNAPSHOT + 0.3.11 diff --git a/indexer/pom.xml b/indexer/pom.xml index 55921dc261e..c50b7fde6d1 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11-SNAPSHOT + 0.3.11 diff --git a/merger/pom.xml b/merger/pom.xml index c51383f49f7..b0ee76df4ed 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11-SNAPSHOT + 0.3.11 diff --git a/pom.xml b/pom.xml index 893bb33d140..eae813fcb71 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.11-SNAPSHOT + 0.3.11 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 672b0f42805..ccb633057f1 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11-SNAPSHOT + 0.3.11 diff --git a/server/pom.xml b/server/pom.xml index 775756c4cbe..246d9ef450b 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11-SNAPSHOT + 0.3.11 From 49e61d00fde5c092adfa11e50aeaf66404d2e053 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:05:46 -0800 Subject: [PATCH 104/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 5c3dc83b101..3d57c6e9a4d 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11 + 0.3.12-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index a128c364d29..ed2d92f854e 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11 + 0.3.12-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 99d9bfee95c..548a92e32a0 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.11 + 0.3.12-SNAPSHOT com.metamx druid - 0.3.11 + 0.3.12-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 612a5729b4d..939106d18f6 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11 + 0.3.12-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index f5db26ed0e8..a24f08843bc 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.11 + 0.3.12-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index b7276f928a7..146363f446f 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.11 + 0.3.12-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 396a8bd81e5..68656fd4976 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11 + 0.3.12-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index c50b7fde6d1..3d5bd9a3102 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11 + 0.3.12-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index b0ee76df4ed..42ad66cc0c4 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11 + 0.3.12-SNAPSHOT diff --git a/pom.xml b/pom.xml index eae813fcb71..eaca6d3ca93 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.11 + 0.3.12-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index ccb633057f1..d22b3a0dd0a 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11 + 0.3.12-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 246d9ef450b..1b666ae0f5e 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.11 + 0.3.12-SNAPSHOT From cbe0555d00736752e4a7c8a650d22d450baad5da Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:25:25 -0800 Subject: [PATCH 105/121] better logging --- .../druid/merger/coordinator/RemoteTaskRunner.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 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 9c36eecf6e9..8c5661856c4 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 @@ -408,9 +408,12 @@ public class RemoteTaskRunner implements TaskRunner public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { try { + if (event.getData() != null) { + log.info("Event[%s]: %s", event.getType(), event.getData().getPath()); + } + if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { - log.info("Event[%s]: %s", event.getType(), event.getData().getPath()); final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); final TaskStatus taskStatus; @@ -471,8 +474,8 @@ public class RemoteTaskRunner implements TaskRunner cleanup(worker.getHost(), taskId); runPendingTasks(); } - } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { - log.info("Event[%s]: %s", event.getType(), event.getData().getPath()); + } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED) + || event.getType().equals(PathChildrenCacheEvent.Type.CONNECTION_LOST)) { final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); if (runningTasks.containsKey(taskId)) { log.info("Task %s just disappeared!", taskId); From 38c8ffb552665c9d3f2d1fd59e5f9292e1baa23f Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:28:28 -0800 Subject: [PATCH 106/121] [maven-release-plugin] prepare release druid-0.3.12 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 3d57c6e9a4d..552440b39c8 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12-SNAPSHOT + 0.3.12 diff --git a/common/pom.xml b/common/pom.xml index ed2d92f854e..e4babd7a8da 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12-SNAPSHOT + 0.3.12 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 548a92e32a0..77121b81339 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.12-SNAPSHOT + 0.3.12 com.metamx druid - 0.3.12-SNAPSHOT + 0.3.12 diff --git a/examples/pom.xml b/examples/pom.xml index 939106d18f6..b80aadc6ef9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12-SNAPSHOT + 0.3.12 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index a24f08843bc..fa95aeb36f9 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.12-SNAPSHOT + 0.3.12 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 146363f446f..72d484ea50a 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.12-SNAPSHOT + 0.3.12 diff --git a/index-common/pom.xml b/index-common/pom.xml index 68656fd4976..074c22ca00a 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12-SNAPSHOT + 0.3.12 diff --git a/indexer/pom.xml b/indexer/pom.xml index 3d5bd9a3102..861c70bbb95 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12-SNAPSHOT + 0.3.12 diff --git a/merger/pom.xml b/merger/pom.xml index 42ad66cc0c4..7fcea9b159f 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12-SNAPSHOT + 0.3.12 diff --git a/pom.xml b/pom.xml index eaca6d3ca93..8e8e25902e7 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.12-SNAPSHOT + 0.3.12 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index d22b3a0dd0a..e5c60f53827 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12-SNAPSHOT + 0.3.12 diff --git a/server/pom.xml b/server/pom.xml index 1b666ae0f5e..cdadbf3c8c9 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12-SNAPSHOT + 0.3.12 From e0700bb1614e29907794ec2530dd5a403561002d Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:28:34 -0800 Subject: [PATCH 107/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 552440b39c8..de2215545bb 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12 + 0.3.13-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index e4babd7a8da..18a5fba7c9c 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12 + 0.3.13-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 77121b81339..e55e67bd77d 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.12 + 0.3.13-SNAPSHOT com.metamx druid - 0.3.12 + 0.3.13-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index b80aadc6ef9..4903c42c225 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12 + 0.3.13-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index fa95aeb36f9..c1cbf28f9a0 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.12 + 0.3.13-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 72d484ea50a..96ea0a683dd 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.12 + 0.3.13-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 074c22ca00a..742be15ae9d 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12 + 0.3.13-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 861c70bbb95..acb68954cbd 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12 + 0.3.13-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 7fcea9b159f..e04f36a5289 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12 + 0.3.13-SNAPSHOT diff --git a/pom.xml b/pom.xml index 8e8e25902e7..4f73b0d40a0 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.12 + 0.3.13-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index e5c60f53827..27e033a4607 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12 + 0.3.13-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index cdadbf3c8c9..e88dfdd7ba3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.12 + 0.3.13-SNAPSHOT From 1995039fb99648e2b695878ea137b898b7b0f328 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:44:14 -0800 Subject: [PATCH 108/121] additional logs --- .../merger/coordinator/RemoteTaskRunner.java | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 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 8c5661856c4..5d3805f0ccd 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 @@ -53,10 +53,8 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -407,11 +405,11 @@ public class RemoteTaskRunner implements TaskRunner @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { + log.info("Event: %s", event.getType()); + if (event.getData() != null) { + log.info("Data: %s", event.getData().getPath()); + } try { - if (event.getData() != null) { - log.info("Event[%s]: %s", event.getType(), event.getData().getPath()); - } - if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { @@ -474,12 +472,13 @@ public class RemoteTaskRunner implements TaskRunner cleanup(worker.getHost(), taskId); runPendingTasks(); } - } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED) - || event.getType().equals(PathChildrenCacheEvent.Type.CONNECTION_LOST)) { + } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); if (runningTasks.containsKey(taskId)) { log.info("Task %s just disappeared!", taskId); retryTask(runningTasks.get(taskId), worker.getHost()); + } else { + log.info("Lost a task I didn't know about: %s", taskId); } } } From 06a2a3a793b56f53f8895d9631a2e41311c6868e Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:45:50 -0800 Subject: [PATCH 109/121] [maven-release-plugin] prepare release druid-0.3.13 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index de2215545bb..b2ed1a402b2 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13-SNAPSHOT + 0.3.13 diff --git a/common/pom.xml b/common/pom.xml index 18a5fba7c9c..db0769b0b44 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13-SNAPSHOT + 0.3.13 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index e55e67bd77d..83963689ce5 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.13-SNAPSHOT + 0.3.13 com.metamx druid - 0.3.13-SNAPSHOT + 0.3.13 diff --git a/examples/pom.xml b/examples/pom.xml index 4903c42c225..6c20539694b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13-SNAPSHOT + 0.3.13 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index c1cbf28f9a0..4fa36803fa3 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.13-SNAPSHOT + 0.3.13 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 96ea0a683dd..5f90f84313b 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.13-SNAPSHOT + 0.3.13 diff --git a/index-common/pom.xml b/index-common/pom.xml index 742be15ae9d..a0b5db2d3f2 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13-SNAPSHOT + 0.3.13 diff --git a/indexer/pom.xml b/indexer/pom.xml index acb68954cbd..81f0d875f79 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13-SNAPSHOT + 0.3.13 diff --git a/merger/pom.xml b/merger/pom.xml index e04f36a5289..61c221b072a 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13-SNAPSHOT + 0.3.13 diff --git a/pom.xml b/pom.xml index 4f73b0d40a0..22a4c23b9ce 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.13-SNAPSHOT + 0.3.13 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 27e033a4607..991b8613507 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13-SNAPSHOT + 0.3.13 diff --git a/server/pom.xml b/server/pom.xml index e88dfdd7ba3..89611a7c0ad 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13-SNAPSHOT + 0.3.13 From 7f6747194d3ca522f2e2e0d2d96eec7e1efa5750 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:45:59 -0800 Subject: [PATCH 110/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index b2ed1a402b2..9b514b80ce0 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13 + 0.3.14-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index db0769b0b44..3605e1c608e 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13 + 0.3.14-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 83963689ce5..0dd3b8bf14c 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.13 + 0.3.14-SNAPSHOT com.metamx druid - 0.3.13 + 0.3.14-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 6c20539694b..faa941b7e44 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13 + 0.3.14-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 4fa36803fa3..480304f02d6 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.13 + 0.3.14-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 5f90f84313b..e046e308ffa 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.13 + 0.3.14-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index a0b5db2d3f2..47587d62e60 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13 + 0.3.14-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 81f0d875f79..dcc5b9b8ac1 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13 + 0.3.14-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 61c221b072a..dcc2d3c5050 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13 + 0.3.14-SNAPSHOT diff --git a/pom.xml b/pom.xml index 22a4c23b9ce..6f5cff69380 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.13 + 0.3.14-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 991b8613507..784b786c578 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13 + 0.3.14-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 89611a7c0ad..cc88e9f622f 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.13 + 0.3.14-SNAPSHOT From bb83ddb564710742f4fa9efe626d45dd20f05f1e Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:51:33 -0800 Subject: [PATCH 111/121] fix bug with worker disappearance --- .../merger/coordinator/RemoteTaskRunner.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 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 5d3805f0ccd..42e70abef08 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 @@ -478,7 +478,7 @@ public class RemoteTaskRunner implements TaskRunner log.info("Task %s just disappeared!", taskId); retryTask(runningTasks.get(taskId), worker.getHost()); } else { - log.info("Lost a task I didn't know about: %s", taskId); + log.info("A task disappeared I didn't know about: %s", taskId); } } } @@ -513,19 +513,23 @@ public class RemoteTaskRunner implements TaskRunner ZkWorker zkWorker = zkWorkers.get(worker.getHost()); if (zkWorker != null) { try { - Set tasksPending = Sets.newHashSet( + Set tasksToRetry = Sets.newHashSet( cf.getChildren() .forPath(JOINER.join(config.getTaskPath(), worker.getHost())) ); - log.info("%s had %d tasks pending", worker.getHost(), tasksPending.size()); + tasksToRetry.addAll( + cf.getChildren() + .forPath(JOINER.join(config.getStatusPath(), worker.getHost())) + ); + log.info("%s has %d tasks to retry", worker.getHost(), tasksToRetry.size()); - for (String taskId : tasksPending) { - TaskRunnerWorkItem taskRunnerWorkItem = pendingTasks.get(taskId); + for (String taskId : tasksToRetry) { + TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); if (taskRunnerWorkItem != null) { cf.delete().guaranteed().forPath(JOINER.join(config.getTaskPath(), worker.getHost(), taskId)); retryTask(taskRunnerWorkItem, worker.getHost()); } else { - log.warn("RemoteTaskRunner has no knowledge of pending task %s", taskId); + log.warn("RemoteTaskRunner has no knowledge of task %s", taskId); } } From e3ba6efb80d53163b8848e124401d6b5db462525 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:53:07 -0800 Subject: [PATCH 112/121] [maven-release-plugin] prepare release druid-0.3.14 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 9b514b80ce0..20d826b8cc2 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14-SNAPSHOT + 0.3.14 diff --git a/common/pom.xml b/common/pom.xml index 3605e1c608e..cba00541db0 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14-SNAPSHOT + 0.3.14 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 0dd3b8bf14c..57faf6162bc 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.14-SNAPSHOT + 0.3.14 com.metamx druid - 0.3.14-SNAPSHOT + 0.3.14 diff --git a/examples/pom.xml b/examples/pom.xml index faa941b7e44..9ed5659cebd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14-SNAPSHOT + 0.3.14 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 480304f02d6..2e398cad5cb 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.14-SNAPSHOT + 0.3.14 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index e046e308ffa..18cea73552c 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.14-SNAPSHOT + 0.3.14 diff --git a/index-common/pom.xml b/index-common/pom.xml index 47587d62e60..b5fee9fe6b3 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14-SNAPSHOT + 0.3.14 diff --git a/indexer/pom.xml b/indexer/pom.xml index dcc5b9b8ac1..cda71c34230 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14-SNAPSHOT + 0.3.14 diff --git a/merger/pom.xml b/merger/pom.xml index dcc2d3c5050..58953190155 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14-SNAPSHOT + 0.3.14 diff --git a/pom.xml b/pom.xml index 6f5cff69380..b8aa1155c2c 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.14-SNAPSHOT + 0.3.14 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 784b786c578..5241d0a125c 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14-SNAPSHOT + 0.3.14 diff --git a/server/pom.xml b/server/pom.xml index cc88e9f622f..3f2ca141cb5 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14-SNAPSHOT + 0.3.14 From 879b2475d450c79b6bf4659c9f86e9785ce02bdd Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 1 Mar 2013 19:53:13 -0800 Subject: [PATCH 113/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 20d826b8cc2..501a251cd9d 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14 + 0.3.15-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index cba00541db0..2e3c8c2786b 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14 + 0.3.15-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 57faf6162bc..381403c539b 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.14 + 0.3.15-SNAPSHOT com.metamx druid - 0.3.14 + 0.3.15-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 9ed5659cebd..f81a1ae517f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14 + 0.3.15-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 2e398cad5cb..2eb91abf4d4 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.14 + 0.3.15-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 18cea73552c..475b739a731 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.14 + 0.3.15-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index b5fee9fe6b3..4d8da67b852 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14 + 0.3.15-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index cda71c34230..ca05306535a 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14 + 0.3.15-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 58953190155..f9072841596 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14 + 0.3.15-SNAPSHOT diff --git a/pom.xml b/pom.xml index b8aa1155c2c..e5b48ee66c9 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.14 + 0.3.15-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 5241d0a125c..74035433d87 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14 + 0.3.15-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 3f2ca141cb5..fd8cb9680a2 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.14 + 0.3.15-SNAPSHOT From e5cc6d241dff76ed3712248251a0561d0510c695 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Sat, 2 Mar 2013 08:58:22 -0800 Subject: [PATCH 114/121] cleaning up some code for RTR --- .../merger/coordinator/RemoteTaskRunner.java | 34 +++++-------------- 1 file changed, 9 insertions(+), 25 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 42e70abef08..b344700a51a 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 @@ -405,37 +405,18 @@ public class RemoteTaskRunner implements TaskRunner @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { - log.info("Event: %s", event.getType()); - if (event.getData() != null) { - log.info("Data: %s", event.getData().getPath()); - } try { if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { - final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); - final TaskStatus taskStatus; + final TaskStatus taskStatus = jsonMapper.readValue( + event.getData().getData(), TaskStatus.class + ); // This can fail if a worker writes a bogus status. Retry if so. - try { - taskStatus = jsonMapper.readValue( - event.getData().getData(), TaskStatus.class - ); - - if (!taskStatus.getId().equals(taskId)) { - // Sanity check - throw new ISE( - "Worker[%s] status id does not match payload id: %s != %s", - worker.getHost(), - taskId, - taskStatus.getId() - ); - } - } - catch (Exception e) { - log.warn(e, "Worker[%s] wrote bogus status for task: %s", worker.getHost(), taskId); + if (!taskStatus.getId().equals(taskId)) { retryTask(runningTasks.get(taskId), worker.getHost()); - throw Throwables.propagate(e); + return; } log.info( @@ -526,7 +507,10 @@ public class RemoteTaskRunner implements TaskRunner for (String taskId : tasksToRetry) { TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); if (taskRunnerWorkItem != null) { - cf.delete().guaranteed().forPath(JOINER.join(config.getTaskPath(), worker.getHost(), taskId)); + String taskPath = JOINER.join(config.getTaskPath(), worker.getHost(), taskId); + if (cf.checkExists().forPath(taskPath) != null) { + cf.delete().guaranteed().forPath(taskPath); + } retryTask(taskRunnerWorkItem, worker.getHost()); } else { log.warn("RemoteTaskRunner has no knowledge of task %s", taskId); From aa63e0b25b2576ad6489a1ff57543b42400649ad Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Sat, 2 Mar 2013 09:25:44 -0800 Subject: [PATCH 115/121] account for duplicate events --- .../merger/coordinator/RemoteTaskRunner.java | 114 +++++++++--------- 1 file changed, 57 insertions(+), 57 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 b344700a51a..739a910473f 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 @@ -405,69 +405,69 @@ public class RemoteTaskRunner implements TaskRunner @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { - try { - if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || - event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { - final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); - final TaskStatus taskStatus = jsonMapper.readValue( - event.getData().getData(), TaskStatus.class - ); - - // This can fail if a worker writes a bogus status. Retry if so. - if (!taskStatus.getId().equals(taskId)) { - retryTask(runningTasks.get(taskId), worker.getHost()); - return; - } - - log.info( - "Worker[%s] wrote %s status for task: %s", - worker.getHost(), - taskStatus.getStatusCode(), - taskId - ); - - // Synchronizing state with ZK - synchronized (statusLock) { - statusLock.notify(); - } - - final TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); - if (taskRunnerWorkItem == null) { - log.warn( - "WTF?! Worker[%s] announcing a status for a task I didn't know about: %s", - worker.getHost(), - taskId + synchronized (statusLock) { + try { + if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || + event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { + final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); + final TaskStatus taskStatus = jsonMapper.readValue( + event.getData().getData(), TaskStatus.class ); - } - if (taskStatus.isComplete()) { - if (taskRunnerWorkItem != null) { - final TaskCallback callback = taskRunnerWorkItem.getCallback(); - if (callback != null) { - callback.notify(taskStatus); - } + // This can fail if a worker writes a bogus status. Retry if so. + if (!taskStatus.getId().equals(taskId)) { + retryTask(runningTasks.get(taskId), worker.getHost()); + return; } - // Worker is done with this task - zkWorker.setLastCompletedTaskTime(new DateTime()); - cleanup(worker.getHost(), taskId); - runPendingTasks(); - } - } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { - final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); - if (runningTasks.containsKey(taskId)) { - log.info("Task %s just disappeared!", taskId); - retryTask(runningTasks.get(taskId), worker.getHost()); - } else { - log.info("A task disappeared I didn't know about: %s", taskId); + log.info( + "Worker[%s] wrote %s status for task: %s", + worker.getHost(), + taskStatus.getStatusCode(), + taskId + ); + + // Synchronizing state with ZK + statusLock.notify(); + + final TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); + if (taskRunnerWorkItem == null) { + log.warn( + "WTF?! Worker[%s] announcing a status for a task I didn't know about: %s", + worker.getHost(), + taskId + ); + } + + if (taskStatus.isComplete()) { + if (taskRunnerWorkItem != null) { + final TaskCallback callback = taskRunnerWorkItem.getCallback(); + if (callback != null) { + callback.notify(taskStatus); + } + } + + // Worker is done with this task + zkWorker.setLastCompletedTaskTime(new DateTime()); + cleanup(worker.getHost(), taskId); + runPendingTasks(); + } + } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { + final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); + if (runningTasks.containsKey(taskId)) { + log.info("Task %s just disappeared!", taskId); + retryTask(runningTasks.get(taskId), worker.getHost()); + } else { + log.info("A task disappeared I didn't know about: %s", taskId); + } } } - } - catch (Exception e) { - log.makeAlert(e, "Failed to handle new worker status") - .addData("worker", worker.getHost()) - .addData("znode", event.getData().getPath()) - .emit(); + catch (Exception e) { + log.makeAlert(e, "Failed to handle new worker status") + .addData("worker", worker.getHost()) + .addData("znode", event.getData().getPath()) + .emit(); + } } } } From a90d204ba2567a80ee6e091f5a1b91bfec150f84 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Sat, 2 Mar 2013 09:28:51 -0800 Subject: [PATCH 116/121] [maven-release-plugin] prepare release druid-0.3.15 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 501a251cd9d..99af2c64333 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15-SNAPSHOT + 0.3.15 diff --git a/common/pom.xml b/common/pom.xml index 2e3c8c2786b..40423cebd86 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15-SNAPSHOT + 0.3.15 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 381403c539b..36d61824ed4 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.15-SNAPSHOT + 0.3.15 com.metamx druid - 0.3.15-SNAPSHOT + 0.3.15 diff --git a/examples/pom.xml b/examples/pom.xml index f81a1ae517f..a9678d85256 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15-SNAPSHOT + 0.3.15 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 2eb91abf4d4..55d31871468 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.15-SNAPSHOT + 0.3.15 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 475b739a731..98cc48ed1d1 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.15-SNAPSHOT + 0.3.15 diff --git a/index-common/pom.xml b/index-common/pom.xml index 4d8da67b852..70b5289740d 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15-SNAPSHOT + 0.3.15 diff --git a/indexer/pom.xml b/indexer/pom.xml index ca05306535a..b74f8d5a07a 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15-SNAPSHOT + 0.3.15 diff --git a/merger/pom.xml b/merger/pom.xml index f9072841596..e0d03582e11 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15-SNAPSHOT + 0.3.15 diff --git a/pom.xml b/pom.xml index e5b48ee66c9..944c3a79c47 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.15-SNAPSHOT + 0.3.15 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 74035433d87..1cd85f50e2b 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15-SNAPSHOT + 0.3.15 diff --git a/server/pom.xml b/server/pom.xml index fd8cb9680a2..1187fa1533d 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15-SNAPSHOT + 0.3.15 From ee38d096c72001955a34c17f414b0d6767860982 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Sat, 2 Mar 2013 09:29:04 -0800 Subject: [PATCH 117/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 99af2c64333..7735497764a 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15 + 0.3.16-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 40423cebd86..212e8f4c035 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15 + 0.3.16-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 36d61824ed4..ae429a40002 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.15 + 0.3.16-SNAPSHOT com.metamx druid - 0.3.15 + 0.3.16-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index a9678d85256..ce809546e0b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15 + 0.3.16-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 55d31871468..2f14eb10ceb 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.15 + 0.3.16-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 98cc48ed1d1..0f3d21ae1a0 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.15 + 0.3.16-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 70b5289740d..2553823ffab 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15 + 0.3.16-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index b74f8d5a07a..4338b90e3ce 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15 + 0.3.16-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index e0d03582e11..80e5667882c 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15 + 0.3.16-SNAPSHOT diff --git a/pom.xml b/pom.xml index 944c3a79c47..8e97764e169 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.15 + 0.3.16-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 1cd85f50e2b..5e012a4bdf5 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15 + 0.3.16-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 1187fa1533d..82d8754928a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.15 + 0.3.16-SNAPSHOT From 4e2107fbeed44ee83b26fdba7975b18bf83af36f Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 4 Mar 2013 11:23:36 -0800 Subject: [PATCH 118/121] Merger: Rename Task.getFixedInterval -> getImplicitLockInterval. Add more Task docs. --- .../merger/common/task/AbstractTask.java | 5 ++-- .../druid/merger/common/task/DeleteTask.java | 2 +- .../task/IndexDeterminePartitionsTask.java | 4 +-- .../common/task/IndexGeneratorTask.java | 4 +-- .../druid/merger/common/task/KillTask.java | 4 +-- .../druid/merger/common/task/MergeTask.java | 4 +-- .../metamx/druid/merger/common/task/Task.java | 28 +++++++++++++------ .../druid/merger/coordinator/TaskLockbox.java | 6 ++-- .../druid/merger/coordinator/TaskQueue.java | 8 +++--- .../merger/coordinator/exec/TaskConsumer.java | 6 ++-- .../merger/common/task/MergeTaskTest.java | 2 +- .../merger/common/task/TaskSerdeTest.java | 20 ++++++------- 12 files changed, 52 insertions(+), 41 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index bb65225a6a5..899e2066fb0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -19,7 +19,6 @@ package com.metamx.druid.merger.common.task; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Optional; @@ -71,7 +70,7 @@ public abstract class AbstractTask implements Task @JsonProperty("interval") @Override - public Optional getFixedInterval() + public Optional getImplicitLockInterval() { return interval; } @@ -89,7 +88,7 @@ public abstract class AbstractTask implements Task .add("id", id) .add("type", getType()) .add("dataSource", dataSource) - .add("interval", getFixedInterval()) + .add("interval", getImplicitLockInterval()) .toString(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index f9d0de34138..991d76228dd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -78,7 +78,7 @@ public class DeleteTask extends AbstractTask { // Strategy: Create an empty segment covering the interval to be deleted final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); - final Interval interval = this.getFixedInterval().get(); + final Interval interval = this.getImplicitLockInterval().get(); final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 569aa3e8a29..2886c8cc77a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -104,7 +104,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask // TODO: Replace/merge/whatever with hadoop determine-partitions code // We know this exists - final Interval interval = getFixedInterval().get(); + final Interval interval = getImplicitLockInterval().get(); // Blacklist dimensions that have multiple values per row final Set unusableDimensions = Sets.newHashSet(); @@ -244,7 +244,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask { return new IndexGeneratorTask( getGroupId(), - getFixedInterval().get(), + getImplicitLockInterval().get(), firehoseFactory, new Schema( schema.getDataSource(), diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index f4daec2d90a..7dc65797ea8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -103,7 +103,7 @@ public class IndexGeneratorTask extends AbstractTask final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); // We know this exists - final Interval interval = getFixedInterval().get(); + final Interval interval = getImplicitLockInterval().get(); // Set up temporary directory for indexing final File tmpDir = new File( @@ -205,7 +205,7 @@ public class IndexGeneratorTask extends AbstractTask * @return true or false */ private boolean shouldIndex(InputRow inputRow) { - if(!getFixedInterval().get().contains(inputRow.getTimestampFromEpoch())) { + if(!getImplicitLockInterval().get().contains(inputRow.getTimestampFromEpoch())) { return false; } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index 84deaee5e33..0c1602c759d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -59,8 +59,8 @@ public class KillTask extends AbstractTask throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); } - if(!myLock.getInterval().equals(getFixedInterval().get())) { - throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getFixedInterval().get()); + if(!myLock.getInterval().equals(getImplicitLockInterval().get())) { + throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getImplicitLockInterval().get()); } // List unused segments diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index f317efc7040..1791c2a097d 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 @@ -215,7 +215,7 @@ public abstract class MergeTask extends AbstractTask final Set current = ImmutableSet.copyOf( Iterables.transform( toolbox.getTaskActionClient() - .submit(new SegmentListUsedAction(this, getDataSource(), getFixedInterval().get())), + .submit(new SegmentListUsedAction(this, getDataSource(), getImplicitLockInterval().get())), toIdentifier ) ); @@ -256,7 +256,7 @@ public abstract class MergeTask extends AbstractTask return Objects.toStringHelper(this) .add("id", getId()) .add("dataSource", getDataSource()) - .add("interval", getFixedInterval()) + .add("interval", getImplicitLockInterval()) .add("segments", segments) .toString(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 60a265564da..8d93486ff6b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -27,7 +27,19 @@ import com.metamx.druid.merger.common.TaskToolbox; import org.joda.time.Interval; /** - * Represents a task that can run on a worker. Immutable. + * Represents a task that can run on a worker. The general contracts surrounding Tasks are: + *

    + *
  • Tasks must operate on a single datasource.
  • + *
  • Tasks should be immutable, since the task ID is used as a proxy for the task in many locations.
  • + *
  • Task IDs must be unique. This can be done by naming them using UUIDs or the current timestamp.
  • + *
  • Tasks are each part of a "task group", which is a set of tasks that can share interval locks. These are + * useful for producing sharded segments.
  • + *
  • Tasks can optionally have an "implicit lock interval". Tasks with this property are guaranteed to have + * a lock on that interval during their {@link #preflight(com.metamx.druid.merger.common.TaskToolbox)} and + * {@link #run(com.metamx.druid.merger.common.TaskToolbox)} methods.
  • + *
  • Tasks do not need to explicitly release locks; they are released upon task completion. Tasks may choose + * to release locks early if they desire.
  • + *
*/ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class) @JsonSubTypes(value = { @@ -64,15 +76,15 @@ public interface Task public String getDataSource(); /** - * Returns fixed interval for this task, if any. Tasks without fixed intervals are not granted locks when started - * and must explicitly request them. + * Returns implicit lock interval for this task, if any. Tasks without implicit lock intervals are not granted locks + * when started and must explicitly request them. */ - public Optional getFixedInterval(); + public Optional getImplicitLockInterval(); /** * Execute preflight checks for a task. This typically runs on the coordinator, and will be run while - * holding a lock on our dataSource and interval. If this method throws an exception, the task should be - * considered a failure. + * holding a lock on our dataSource and implicit lock interval (if any). If this method throws an exception, the + * task should be considered a failure. * * @param toolbox Toolbox for this task * @@ -85,8 +97,8 @@ public interface Task /** * Execute a task. This typically runs on a worker as determined by a TaskRunner, and will be run while - * holding a lock on our dataSource and interval. If this method throws an exception, the task should be - * considered a failure. + * holding a lock on our dataSource and implicit lock interval (if any). If this method throws an exception, the task + * should be considered a failure. * * @param toolbox Toolbox for this task * diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java index 5e972b18805..07abd0594ac 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java @@ -78,7 +78,7 @@ public class TaskLockbox try { - if(task.getFixedInterval().isPresent() && !task.getFixedInterval().get().equals(interval)) { + if(task.getImplicitLockInterval().isPresent() && !task.getImplicitLockInterval().get().equals(interval)) { // Task may only lock its fixed interval, if present throw new IAE("Task must lock its fixed interval: %s", task.getId()); } @@ -292,9 +292,9 @@ public class TaskLockbox try { final Iterable searchSpace; - if (task.getFixedInterval().isPresent()) { + if (task.getImplicitLockInterval().isPresent()) { // Narrow down search using findLockPossesForInterval - searchSpace = findLockPossesForInterval(task.getDataSource(), task.getFixedInterval().get()); + searchSpace = findLockPossesForInterval(task.getDataSource(), task.getImplicitLockInterval().get()); } else { // Scan through all locks for this datasource final NavigableMap dsRunning = running.get(task.getDataSource()); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java index 377004d473f..e16912b4c6e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java @@ -225,8 +225,8 @@ public class TaskQueue // Attempt to add this task to a running task group. Silently continue if this is not possible. // The main reason this is here is so when subtasks are added, they end up in the same task group // as their parent whenever possible. - if(task.getFixedInterval().isPresent()) { - taskLockbox.tryLock(task, task.getFixedInterval().get()); + if(task.getImplicitLockInterval().isPresent()) { + taskLockbox.tryLock(task, task.getImplicitLockInterval().get()); } return true; @@ -274,9 +274,9 @@ public class TaskQueue try { for (final Task task : queue) { - if(task.getFixedInterval().isPresent()) { + if(task.getImplicitLockInterval().isPresent()) { // If this task has a fixed interval, attempt to lock it right now. - final Optional maybeLock = taskLockbox.tryLock(task, task.getFixedInterval().get()); + final Optional maybeLock = taskLockbox.tryLock(task, task.getImplicitLockInterval().get()); if(maybeLock.isPresent()) { log.info("Task claimed with fixed interval lock: %s", task.getId()); queue.remove(task); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index 14135174e59..f1e31a9e5c1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -96,7 +96,7 @@ public class TaskConsumer implements Runnable .addData("task", task.getId()) .addData("type", task.getType()) .addData("dataSource", task.getDataSource()) - .addData("interval", task.getFixedInterval()) + .addData("interval", task.getImplicitLockInterval()) .emit(); // Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now. @@ -118,7 +118,7 @@ public class TaskConsumer implements Runnable final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() .setUser2(task.getDataSource()) .setUser4(task.getType()) - .setUser5(task.getFixedInterval().toString()); + .setUser5(task.getImplicitLockInterval().toString()); // Run preflight checks TaskStatus preflightStatus; @@ -166,7 +166,7 @@ public class TaskConsumer implements Runnable .addData("task", task.getId()) .addData("type", task.getType()) .addData("dataSource", task.getDataSource()) - .addData("interval", task.getFixedInterval()) + .addData("interval", task.getImplicitLockInterval()) .emit(); } diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java index 0f1a7a66964..7c779f6a74c 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java @@ -67,7 +67,7 @@ public class MergeTaskTest @Test public void testInterval() { - Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getFixedInterval().get()); + Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getImplicitLockInterval().get()); } @Test diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java index 213bec34421..701093209ea 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java @@ -42,12 +42,12 @@ public class TaskSerdeTest final Task task2 = jsonMapper.readValue(json, Task.class); Assert.assertEquals("foo", task.getDataSource()); - Assert.assertEquals(Optional.of(new Interval("2010-01-01/P2D")), task.getFixedInterval()); + Assert.assertEquals(Optional.of(new Interval("2010-01-01/P2D")), task.getImplicitLockInterval()); Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); - Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval()); } @Test @@ -71,12 +71,12 @@ public class TaskSerdeTest final Task task2 = jsonMapper.readValue(json, Task.class); Assert.assertEquals("foo", task.getDataSource()); - Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getFixedInterval()); + Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval()); Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); - Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval()); } @Test @@ -94,12 +94,12 @@ public class TaskSerdeTest final Task task2 = jsonMapper.readValue(json, Task.class); Assert.assertEquals("foo", task.getDataSource()); - Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getFixedInterval()); + Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval()); Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); - Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval()); } @Test @@ -117,8 +117,8 @@ public class TaskSerdeTest Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); - Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); - Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); + Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval()); + Assert.assertEquals(task.getImplicitLockInterval().get(), task2.getImplicitLockInterval().get()); } @Test @@ -156,11 +156,11 @@ public class TaskSerdeTest final Task task2 = jsonMapper.readValue(json, Task.class); Assert.assertEquals("foo", task.getDataSource()); - Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getFixedInterval()); + Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval()); Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); - Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval()); } } From 92ab51e70c7ebc7488dfc99d9652f49bf6642960 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 4 Mar 2013 13:03:15 -0800 Subject: [PATCH 119/121] fix bugs in balancing logic and removing auto-scaled node --- .../merger/coordinator/RemoteTaskRunner.java | 2 - .../SimpleResourceManagementStrategy.java | 2 +- .../druid/master/BalancerCostAnalyzer.java | 74 +++++++++---------- .../druid/master/DruidMasterBalancer.java | 38 ++++------ .../metamx/druid/master/rules/LoadRule.java | 2 +- 5 files changed, 54 insertions(+), 64 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 739a910473f..e4a13136152 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 @@ -457,8 +457,6 @@ public class RemoteTaskRunner implements TaskRunner if (runningTasks.containsKey(taskId)) { log.info("Task %s just disappeared!", taskId); retryTask(runningTasks.get(taskId), worker.getHost()); - } else { - log.info("A task disappeared I didn't know about: %s", taskId); } } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java index 1cd3f8e46e6..74d34d718d2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -167,7 +167,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat ) ); - if (thoseLazyWorkers.isEmpty()) { + if (thoseLazyWorkers.size() <= minNumWorkers) { return false; } diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 799bccca581..6519785e76b 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -52,11 +52,12 @@ public class BalancerCostAnalyzer /** * Calculates the cost normalization. This is such that the normalized cost is lower bounded * by 1 (e.g. when each segment gets its own compute node). - * @param serverHolders - * A list of ServerHolders for a particular tier. - * @return The normalization value (the sum of the diagonal entries in the - * pairwise cost matrix). This is the cost of a cluster if each - * segment were to get its own compute node. + * + * @param serverHolders A list of ServerHolders for a particular tier. + * + * @return The normalization value (the sum of the diagonal entries in the + * pairwise cost matrix). This is the cost of a cluster if each + * segment were to get its own compute node. */ public double calculateNormalization(final List serverHolders) { @@ -71,9 +72,10 @@ public class BalancerCostAnalyzer /** * Calculates the initial cost of the Druid segment configuration. - * @param serverHolders - * A list of ServerHolders for a particular tier. - * @return The initial cost of the Druid tier. + * + * @param serverHolders A list of ServerHolders for a particular tier. + * + * @return The initial cost of the Druid tier. */ public double calculateInitialTotalCost(final List serverHolders) { @@ -96,11 +98,11 @@ public class BalancerCostAnalyzer * dataSourcePenalty: if two segments belong to the same data source, they are more likely to be involved * in the same queries * gapPenalty: it is more likely that segments close together in time will be queried together - * @param segment1 - * The first DataSegment. - * @param segment2 - * The second DataSegment. - * @return The joint cost of placing the two DataSegments together on one node. + * + * @param segment1 The first DataSegment. + * @param segment2 The second DataSegment. + * + * @return The joint cost of placing the two DataSegments together on one node. */ public double computeJointSegmentCosts(final DataSegment segment1, final DataSegment segment2) { @@ -140,12 +142,12 @@ public class BalancerCostAnalyzer /** * Sample from each server with probability proportional to the number of segments on that server. - * @param serverHolders - * A list of ServerHolders for a particular tier. - * @param numSegments - - * @return A ServerHolder sampled with probability proportional to the - * number of segments on that server + * + * @param serverHolders A list of ServerHolders for a particular tier. + * @param numSegments + * + * @return A ServerHolder sampled with probability proportional to the + * number of segments on that server */ private ServerHolder sampleServer(final List serverHolders, final int numSegments) { @@ -163,11 +165,11 @@ public class BalancerCostAnalyzer /** * The balancing application requires us to pick a proposal segment. - * @param serverHolders - * A list of ServerHolders for a particular tier. - * @param numSegments - * The total number of segments on a particular tier. - * @return A BalancerSegmentHolder sampled uniformly at random. + * + * @param serverHolders A list of ServerHolders for a particular tier. + * @param numSegments The total number of segments on a particular tier. + * + * @return A BalancerSegmentHolder sampled uniformly at random. */ public BalancerSegmentHolder pickSegmentToMove(final List serverHolders, final int numSegments) { @@ -175,7 +177,7 @@ public class BalancerCostAnalyzer ServerHolder fromServerHolder = sampleServer(serverHolders, numSegments); /** and actually pick that segment uniformly at random w.p. 1 / numSegmentsOnServer - so that the probability of picking a segment is 1 / totalSegments. */ + so that the probability of picking a segment is 1 / totalSegments. */ List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); @@ -184,18 +186,15 @@ public class BalancerCostAnalyzer /** * The assignment application requires us to supply a proposal segment. - * @param proposalSegment - * A DataSegment that we are proposing to move. - * @param serverHolders - * An iterable of ServerHolders for a particular tier. - * @param assign - * A boolean that is true if used in assignment else false in balancing. - * @return A ServerHolder with the new home for a segment. + * + * @param proposalSegment A DataSegment that we are proposing to move. + * @param serverHolders An iterable of ServerHolders for a particular tier. + * + * @return A ServerHolder with the new home for a segment. */ public ServerHolder findNewSegmentHome( final DataSegment proposalSegment, - final Iterable serverHolders, - final boolean assign + final Iterable serverHolders ) { final long proposalSegmentSize = proposalSegment.getSize(); @@ -204,10 +203,9 @@ public class BalancerCostAnalyzer for (ServerHolder server : serverHolders) { /** Don't calculate cost if the server doesn't have enough space or is loading the segment */ - if (proposalSegmentSize > server.getAvailableSize() - || server.isLoadingSegment(proposalSegment) - /** or if the ask is assignment and the server is serving the segment. */ - || (assign && server.isServingSegment(proposalSegment)) ) { + if (proposalSegmentSize > server.getAvailableSize() || + server.isLoadingSegment(proposalSegment) || + server.isServingSegment(proposalSegment)) { continue; } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 3d66a21f9d7..7678e85624d 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -19,6 +19,7 @@ package com.metamx.druid.master; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.metamx.common.guava.Comparators; @@ -66,9 +67,9 @@ public class DruidMasterBalancer implements DruidMasterHelper holder.reduceLifetime(); if (holder.getLifetime() <= 0) { log.makeAlert("[%s]: Balancer move segments queue has a segment stuck", tier) - .addData("segment", holder.getSegment().getIdentifier()) - .addData("server", holder.getFromServer().getStringProps()) - .emit(); + .addData("segment", holder.getSegment().getIdentifier()) + .addData("server", holder.getFromServer().getStringProps()) + .emit(); } } } @@ -95,7 +96,13 @@ public class DruidMasterBalancer implements DruidMasterHelper continue; } - final List serverHolderList = new ArrayList(entry.getValue()); + final List serverHolderList = Lists.newArrayList(entry.getValue()); + + if (serverHolderList.size() <= 1) { + log.info("[%s]: One or fewer servers found. Cannot balance.", tier); + continue; + } + int numSegments = 0; for (ServerHolder server : serverHolderList) { numSegments += server.getServer().getSegments().size(); @@ -107,19 +114,14 @@ public class DruidMasterBalancer implements DruidMasterHelper } int iter = 0; - while (iter < maxSegmentsToMove) { iter++; final BalancerSegmentHolder segmentToMove = analyzer.pickSegmentToMove(serverHolderList, numSegments); - final ServerHolder holder = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList, false); + final ServerHolder holder = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList); if (holder == null) { continue; } - final DruidServer toServer = holder.getServer(); - - if (!currentlyMovingSegments.get(tier).containsKey(segmentToMove.getSegment())) { - moveSegment(segmentToMove, toServer, params); - } + moveSegment(segmentToMove, holder.getServer(), params); } final double initialTotalCost = analyzer.calculateInitialTotalCost(serverHolderList); @@ -132,21 +134,13 @@ public class DruidMasterBalancer implements DruidMasterHelper stats.addToTieredStat("movedCount", tier, currentlyMovingSegments.get(tier).size()); log.info( - "Initial Total Cost: [%f], Normalization: [%f], Initial Normalized Cost: [%f], Segments Moved: [%d]", + "[%s]: Initial Total Cost: [%f], Normalization: [%f], Initial Normalized Cost: [%f], Segments Moved: [%d]", + tier, initialTotalCost, normalization, normalizedInitialCost, currentlyMovingSegments.get(tier).size() ); - - if (serverHolderList.size() <= 1) { - log.info( - "[%s]: One or fewer servers found. Cannot balance.", - tier - ); - continue; - } - } return params.buildFromExisting() @@ -194,7 +188,7 @@ public class DruidMasterBalancer implements DruidMasterHelper log.makeAlert(e, String.format("[%s] : Moving exception", segmentName)).emit(); } } else { - currentlyMovingSegments.get(toServer.getTier()).remove(segment); + currentlyMovingSegments.get(toServer.getTier()).remove(segmentName); } } diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index a59d9c795b5..4cd6acae640 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -91,7 +91,7 @@ public abstract class LoadRule implements Rule final MasterStats stats = new MasterStats(); while (totalReplicants < expectedReplicants) { - final ServerHolder holder = analyzer.findNewSegmentHome(segment, serverHolderList, true); + final ServerHolder holder = analyzer.findNewSegmentHome(segment, serverHolderList); if (holder == null) { log.warn( From d4f4e41320e3553f8a0f9b68b6fb2061ad296a7a Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 4 Mar 2013 13:08:09 -0800 Subject: [PATCH 120/121] [maven-release-plugin] prepare release druid-0.3.16 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 7735497764a..7fcc0fff2ae 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/common/pom.xml b/common/pom.xml index 212e8f4c035..b7846385378 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index ae429a40002..098d00dd7da 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.16-SNAPSHOT + 0.3.16 com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/examples/pom.xml b/examples/pom.xml index ce809546e0b..ceafebe0f40 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 2f14eb10ceb..348270455eb 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 0f3d21ae1a0..04d1424cab1 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/index-common/pom.xml b/index-common/pom.xml index 2553823ffab..53be8c60664 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/indexer/pom.xml b/indexer/pom.xml index 4338b90e3ce..1a07d1896d2 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/merger/pom.xml b/merger/pom.xml index 80e5667882c..979ccb33315 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/pom.xml b/pom.xml index 8e97764e169..17596631b21 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.16-SNAPSHOT + 0.3.16 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 5e012a4bdf5..4cffee16f43 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/server/pom.xml b/server/pom.xml index 82d8754928a..4feb70e6646 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 From 6977fe03bdb76d5f962f755f3ce26a453b4ddac7 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 4 Mar 2013 13:08:26 -0800 Subject: [PATCH 121/121] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 7fcc0fff2ae..8ad2bea5ee2 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index b7846385378..c4d83b8f817 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 098d00dd7da..b71317d2352 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.16 + 0.3.17-SNAPSHOT com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index ceafebe0f40..8dfe0dca4d1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 348270455eb..4b1f8ab2a1d 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 04d1424cab1..60241675115 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 53be8c60664..d06dd7d561e 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 1a07d1896d2..006dea455d5 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 979ccb33315..3d7d4a05f30 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/pom.xml b/pom.xml index 17596631b21..ebdb8169216 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.16 + 0.3.17-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 4cffee16f43..37bc5d12c58 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 4feb70e6646..45584f572ba 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT