public void dropSegment(String from, String segmentName, final LoadPeonCallback callback) { try { final DruidServer fromServer = serverInventoryView.getInventoryValue(from); if (fromServer == null) { throw new IAE("Unable to find server [%s]", from); } final DataSegment segment = fromServer.getSegment(segmentName); if (segment == null) { throw new IAE("Unable to find segment [%s] on server [%s]", segmentName, from); } final LoadQueuePeon dropPeon = loadManagementPeons.get(from); if (dropPeon == null) { throw new IAE("LoadQueuePeon hasn't been created yet for path [%s]", from); } if (!dropPeon.getSegmentsToDrop().contains(segment)) { dropPeon.dropSegment(segment, callback); } } catch (Exception e) { log.makeAlert(e, "Exception dropping segment %s", segmentName).emit(); callback.execute(); } }
public Map<String, Double> getLoadStatus() { // find available segments Map<String, Set<DataSegment>> availableSegments = Maps.newHashMap(); for (DataSegment dataSegment : getAvailableDataSegments()) { Set<DataSegment> segments = availableSegments.get(dataSegment.getDataSource()); if (segments == null) { segments = Sets.newHashSet(); availableSegments.put(dataSegment.getDataSource(), segments); } segments.add(dataSegment); } // find segments currently loaded Map<String, Set<DataSegment>> segmentsInCluster = Maps.newHashMap(); for (DruidServer druidServer : serverInventoryView.getInventory()) { for (DruidDataSource druidDataSource : druidServer.getDataSources()) { Set<DataSegment> segments = segmentsInCluster.get(druidDataSource.getName()); if (segments == null) { segments = Sets.newHashSet(); segmentsInCluster.put(druidDataSource.getName(), segments); } segments.addAll(druidDataSource.getSegments()); } } // compare available segments with currently loaded Map<String, Double> loadStatus = Maps.newHashMap(); for (Map.Entry<String, Set<DataSegment>> entry : availableSegments.entrySet()) { String dataSource = entry.getKey(); Set<DataSegment> segmentsAvailable = entry.getValue(); Set<DataSegment> loadedSegments = segmentsInCluster.get(dataSource); if (loadedSegments == null) { loadedSegments = Sets.newHashSet(); } Set<DataSegment> unloadedSegments = Sets.difference(segmentsAvailable, loadedSegments); loadStatus.put( dataSource, 100 * ((double) (segmentsAvailable.size() - unloadedSegments.size()) / (double) segmentsAvailable.size())); } return loadStatus; }
private void stopBeingLeader() { synchronized (lock) { try { log.info("I am no longer the leader..."); for (String server : loadManagementPeons.keySet()) { LoadQueuePeon peon = loadManagementPeons.remove(server); peon.stop(); } loadManagementPeons.clear(); serviceAnnouncer.unannounce(self); serverInventoryView.stop(); databaseRuleManager.stop(); databaseSegmentManager.stop(); leader = false; } catch (Exception e) { log.makeAlert(e, "Unable to stopBeingLeader").emit(); } } }
private void becomeLeader() { synchronized (lock) { if (!started) { return; } log.info("I am the leader of the coordinators, all must bow!"); try { leader = true; databaseSegmentManager.start(); databaseRuleManager.start(); serverInventoryView.start(); serviceAnnouncer.announce(self); final List<Pair<? extends CoordinatorRunnable, Duration>> coordinatorRunnables = Lists.newArrayList(); dynamicConfigs = configManager.watch( CoordinatorDynamicConfig.CONFIG_KEY, CoordinatorDynamicConfig.class, new CoordinatorDynamicConfig.Builder().build()); coordinatorRunnables.add( Pair.of(new CoordinatorHistoricalManagerRunnable(), config.getCoordinatorPeriod())); if (indexingServiceClient != null) { coordinatorRunnables.add( Pair.of( new CoordinatorIndexingServiceRunnable( makeIndexingServiceHelpers( configManager.watch( DatasourceWhitelist.CONFIG_KEY, DatasourceWhitelist.class))), config.getCoordinatorIndexingPeriod())); } for (final Pair<? extends CoordinatorRunnable, Duration> coordinatorRunnable : coordinatorRunnables) { ScheduledExecutors.scheduleWithFixedDelay( exec, config.getCoordinatorStartDelay(), coordinatorRunnable.rhs, new Callable<ScheduledExecutors.Signal>() { private final CoordinatorRunnable theRunnable = coordinatorRunnable.lhs; @Override public ScheduledExecutors.Signal call() { if (leader) { theRunnable.run(); } if (leader) { // (We might no longer be coordinator) return ScheduledExecutors.Signal.REPEAT; } else { return ScheduledExecutors.Signal.STOP; } } }); } } catch (Exception e) { log.makeAlert(e, "Unable to become leader").emit(); final LeaderLatch oldLatch = createNewLeaderLatch(); Closeables.closeQuietly(oldLatch); try { leaderLatch.get().start(); } catch (Exception e1) { // If an exception gets thrown out here, then the coordinator will zombie out 'cause it // won't be looking for // the latch anymore. I don't believe it's actually possible for an Exception to throw // out here, but // Curator likes to have "throws Exception" on methods so it might happen... log.makeAlert(e1, "I am a zombie").emit(); } } } }
public void moveSegment( String from, String to, String segmentName, final LoadPeonCallback callback) { try { final DruidServer fromServer = serverInventoryView.getInventoryValue(from); if (fromServer == null) { throw new IAE("Unable to find server [%s]", from); } final DruidServer toServer = serverInventoryView.getInventoryValue(to); if (toServer == null) { throw new IAE("Unable to find server [%s]", to); } if (to.equalsIgnoreCase(from)) { throw new IAE( "Redundant command to move segment [%s] from [%s] to [%s]", segmentName, from, to); } final DataSegment segment = fromServer.getSegment(segmentName); if (segment == null) { throw new IAE("Unable to find segment [%s] on server [%s]", segmentName, from); } final LoadQueuePeon loadPeon = loadManagementPeons.get(to); if (loadPeon == null) { throw new IAE("LoadQueuePeon hasn't been created yet for path [%s]", to); } final LoadQueuePeon dropPeon = loadManagementPeons.get(from); if (dropPeon == null) { throw new IAE("LoadQueuePeon hasn't been created yet for path [%s]", from); } final ServerHolder toHolder = new ServerHolder(toServer, loadPeon); if (toHolder.getAvailableSize() < segment.getSize()) { throw new IAE( "Not enough capacity on server [%s] for segment [%s]. Required: %,d, available: %,d.", to, segment, segment.getSize(), toHolder.getAvailableSize()); } final String toLoadQueueSegPath = ZKPaths.makePath(ZKPaths.makePath(zkPaths.getLoadQueuePath(), to), segmentName); final String toServedSegPath = ZKPaths.makePath( ZKPaths.makePath( serverInventoryView.getInventoryManagerConfig().getInventoryPath(), to), segmentName); loadPeon.loadSegment( segment, new LoadPeonCallback() { @Override protected void execute() { try { if (curator.checkExists().forPath(toServedSegPath) != null && curator.checkExists().forPath(toLoadQueueSegPath) == null && !dropPeon.getSegmentsToDrop().contains(segment)) { dropPeon.dropSegment(segment, callback); } else if (callback != null) { callback.execute(); } } catch (Exception e) { throw Throwables.propagate(e); } } }); } catch (Exception e) { log.makeAlert(e, "Exception moving segment %s", segmentName).emit(); callback.execute(); } }