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(); } }
@Override public void addSegment(DataSegment segment) { try { log.info("Loading segment %s", segment.getIdentifier()); try { serverManager.loadSegment(segment); } catch (Exception e) { removeSegment(segment); throw new SegmentLoadingException( e, "Exception loading segment[%s]", segment.getIdentifier()); } File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.exists()) { try { jsonMapper.writeValue(segmentInfoCacheFile, segment); } catch (IOException e) { removeSegment(segment); throw new SegmentLoadingException( e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile); } } try { announcer.announceSegment(segment); } catch (IOException e) { throw new SegmentLoadingException( e, "Failed to announce segment[%s]", segment.getIdentifier()); } } catch (SegmentLoadingException e) { log.makeAlert(e, "Failed to load segment for dataSource").addData("segment", segment).emit(); } }
private void loadCache() { File baseDir = config.getInfoDir(); if (!baseDir.exists()) { return; } List<DataSegment> cachedSegments = Lists.newArrayList(); for (File file : baseDir.listFiles()) { log.info("Loading segment cache file [%s]", file); try { DataSegment segment = jsonMapper.readValue(file, DataSegment.class); if (serverManager.isSegmentCached(segment)) { cachedSegments.add(segment); } else { log.warn( "Unable to find cache file for %s. Deleting lookup entry", segment.getIdentifier()); File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.delete()) { log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); } } } catch (Exception e) { log.makeAlert(e, "Failed to load segment from segmentInfo file") .addData("file", file) .emit(); } } addSegments(cachedSegments); }
public void addSegments(Iterable<DataSegment> segments) { try { final List<String> segmentFailures = Lists.newArrayList(); final List<DataSegment> validSegments = Lists.newArrayList(); for (DataSegment segment : segments) { log.info("Loading segment %s", segment.getIdentifier()); try { serverManager.loadSegment(segment); } catch (Exception e) { log.error(e, "Exception loading segment[%s]", segment.getIdentifier()); removeSegment(segment); segmentFailures.add(segment.getIdentifier()); continue; } File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.exists()) { try { jsonMapper.writeValue(segmentInfoCacheFile, segment); } catch (IOException e) { log.error( e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile); removeSegment(segment); segmentFailures.add(segment.getIdentifier()); continue; } } validSegments.add(segment); } try { announcer.announceSegments(validSegments); } catch (IOException e) { throw new SegmentLoadingException(e, "Failed to announce segments[%s]", segments); } if (!segmentFailures.isEmpty()) { for (String segmentFailure : segmentFailures) { log.error("%s failed to load", segmentFailure); } throw new SegmentLoadingException( "%,d errors seen while loading segments", segmentFailures.size()); } } catch (SegmentLoadingException e) { log.makeAlert(e, "Failed to load segments for dataSource") .addData("segments", segments) .emit(); } }
@Override public void removeSegment(DataSegment segment) { try { serverManager.dropSegment(segment); File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.delete()) { log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); } announcer.unannounceSegment(segment); } catch (Exception e) { log.makeAlert(e, "Failed to remove segment").addData("segment", segment).emit(); } }
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(); } } }
@Override public TaskStatus call() { final long startTime = System.currentTimeMillis(); final File taskDir = toolbox.getTaskWorkDir(); TaskStatus status; try { log.info("Running task: %s", task.getId()); status = task.run(toolbox); } catch (InterruptedException e) { log.error(e, "Interrupted while running task[%s]", task); throw Throwables.propagate(e); } catch (Exception e) { log.error(e, "Exception while running task[%s]", task); status = TaskStatus.failure(task.getId()); } catch (Throwable t) { log.error(t, "Uncaught Throwable while running task[%s]", task); throw Throwables.propagate(t); } try { if (taskDir.exists()) { log.info("Removing task directory: %s", taskDir); FileUtils.deleteDirectory(taskDir); } } catch (Exception e) { log.makeAlert(e, "Failed to delete task directory") .addData("taskDir", taskDir.toString()) .addData("task", task.getId()) .emit(); } try { return status.withDuration(System.currentTimeMillis() - startTime); } catch (Exception e) { log.error(e, "Uncaught Exception during callback for task[%s]", task); throw Throwables.propagate(e); } }
private <T> QueryRunner<T> getQueryRunnerImpl(Query<T> query) { QueryRunner<T> queryRunner = null; final String queryDataSource = Iterables.getOnlyElement(query.getDataSource().getNames()); for (final ThreadPoolTaskRunnerWorkItem taskRunnerWorkItem : ImmutableList.copyOf(runningItems)) { final Task task = taskRunnerWorkItem.getTask(); if (task.getDataSource().equals(queryDataSource)) { final QueryRunner<T> taskQueryRunner = task.getQueryRunner(query); if (taskQueryRunner != null) { if (queryRunner == null) { queryRunner = taskQueryRunner; } else { log.makeAlert("Found too many query runners for datasource") .addData("dataSource", queryDataSource) .emit(); } } } } return queryRunner == null ? new NoopQueryRunner<T>() : queryRunner; }
public Set<DataSegment> getAvailableDataSegments() { Set<DataSegment> availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator())); Iterable<DataSegment> dataSegments = Iterables.concat( Iterables.transform( databaseSegmentManager.getInventory(), new Function<DruidDataSource, Iterable<DataSegment>>() { @Override public Iterable<DataSegment> apply(DruidDataSource input) { return input.getSegments(); } })); for (DataSegment dataSegment : dataSegments) { if (dataSegment.getSize() < 0) { log.makeAlert("No size on Segment, wtf?").addData("segment", dataSegment).emit(); } availableSegments.add(dataSegment); } return availableSegments; }
@Override public Sequence<T> run(final Query<T> query, final Map<String, Object> responseContext) { final QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query); final CacheStrategy<T, Object, Query<T>> strategy = toolChest.getCacheStrategy(query); final Map<DruidServer, List<SegmentDescriptor>> serverSegments = Maps.newTreeMap(); final List<Pair<Interval, byte[]>> cachedResults = Lists.newArrayList(); final Map<String, CachePopulator> cachePopulatorMap = Maps.newHashMap(); final boolean useCache = BaseQuery.getContextUseCache(query, true) && strategy != null && cacheConfig.isUseCache() && cacheConfig.isQueryCacheable(query); final boolean populateCache = BaseQuery.getContextPopulateCache(query, true) && strategy != null && cacheConfig.isPopulateCache() && cacheConfig.isQueryCacheable(query); final boolean isBySegment = BaseQuery.getContextBySegment(query, false); final ImmutableMap.Builder<String, Object> contextBuilder = new ImmutableMap.Builder<>(); final int priority = BaseQuery.getContextPriority(query, 0); contextBuilder.put("priority", priority); if (populateCache) { // prevent down-stream nodes from caching results as well if we are populating the cache contextBuilder.put(CacheConfig.POPULATE_CACHE, false); contextBuilder.put("bySegment", true); } TimelineLookup<String, ServerSelector> timeline = serverView.getTimeline(query.getDataSource()); if (timeline == null) { return Sequences.empty(); } // build set of segments to query Set<Pair<ServerSelector, SegmentDescriptor>> segments = Sets.newLinkedHashSet(); List<TimelineObjectHolder<String, ServerSelector>> serversLookup = Lists.newLinkedList(); // Note that enabling this leads to putting uncovered intervals information in the response // headers // and might blow up in some cases https://github.com/druid-io/druid/issues/2108 int uncoveredIntervalsLimit = BaseQuery.getContextUncoveredIntervalsLimit(query, 0); if (uncoveredIntervalsLimit > 0) { List<Interval> uncoveredIntervals = Lists.newArrayListWithCapacity(uncoveredIntervalsLimit); boolean uncoveredIntervalsOverflowed = false; for (Interval interval : query.getIntervals()) { Iterable<TimelineObjectHolder<String, ServerSelector>> lookup = timeline.lookup(interval); long startMillis = interval.getStartMillis(); long endMillis = interval.getEndMillis(); for (TimelineObjectHolder<String, ServerSelector> holder : lookup) { Interval holderInterval = holder.getInterval(); long intervalStart = holderInterval.getStartMillis(); if (!uncoveredIntervalsOverflowed && startMillis != intervalStart) { if (uncoveredIntervalsLimit > uncoveredIntervals.size()) { uncoveredIntervals.add(new Interval(startMillis, intervalStart)); } else { uncoveredIntervalsOverflowed = true; } } startMillis = holderInterval.getEndMillis(); serversLookup.add(holder); } if (!uncoveredIntervalsOverflowed && startMillis < endMillis) { if (uncoveredIntervalsLimit > uncoveredIntervals.size()) { uncoveredIntervals.add(new Interval(startMillis, endMillis)); } else { uncoveredIntervalsOverflowed = true; } } } if (!uncoveredIntervals.isEmpty()) { // This returns intervals for which NO segment is present. // Which is not necessarily an indication that the data doesn't exist or is // incomplete. The data could exist and just not be loaded yet. In either // case, though, this query will not include any data from the identified intervals. responseContext.put("uncoveredIntervals", uncoveredIntervals); responseContext.put("uncoveredIntervalsOverflowed", uncoveredIntervalsOverflowed); } } else { for (Interval interval : query.getIntervals()) { Iterables.addAll(serversLookup, timeline.lookup(interval)); } } // Let tool chest filter out unneeded segments final List<TimelineObjectHolder<String, ServerSelector>> filteredServersLookup = toolChest.filterSegments(query, serversLookup); Map<String, Optional<RangeSet<String>>> dimensionRangeCache = Maps.newHashMap(); // Filter unneeded chunks based on partition dimension for (TimelineObjectHolder<String, ServerSelector> holder : filteredServersLookup) { final Set<PartitionChunk<ServerSelector>> filteredChunks = DimFilterUtils.filterShards( query.getFilter(), holder.getObject(), new Function<PartitionChunk<ServerSelector>, ShardSpec>() { @Override public ShardSpec apply(PartitionChunk<ServerSelector> input) { return input.getObject().getSegment().getShardSpec(); } }, dimensionRangeCache); for (PartitionChunk<ServerSelector> chunk : filteredChunks) { 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 ((populateCache || useCache) // implies strategy != null && !isBySegment) // explicit bySegment queries are never cached { queryCacheKey = strategy.computeCacheKey(query); } else { queryCacheKey = null; } if (queryCacheKey != null) { // cachKeys map must preserve segment ordering, in order for shards to always be combined in // the same order Map<Pair<ServerSelector, SegmentDescriptor>, Cache.NamedKey> cacheKeys = Maps.newLinkedHashMap(); for (Pair<ServerSelector, SegmentDescriptor> segment : segments) { final Cache.NamedKey segmentCacheKey = CacheUtil.computeSegmentCacheKey( segment.lhs.getSegment().getIdentifier(), segment.rhs, queryCacheKey); cacheKeys.put(segment, segmentCacheKey); } // Pull cached segments from cache and remove from set of segments to query final Map<Cache.NamedKey, byte[]> cachedValues; if (useCache) { cachedValues = cache.getBulk( Iterables.limit(cacheKeys.values(), cacheConfig.getCacheBulkMergeLimit())); } else { cachedValues = ImmutableMap.of(); } for (Map.Entry<Pair<ServerSelector, SegmentDescriptor>, Cache.NamedKey> entry : cacheKeys.entrySet()) { Pair<ServerSelector, SegmentDescriptor> segment = entry.getKey(); Cache.NamedKey segmentCacheKey = entry.getValue(); final Interval segmentQueryInterval = segment.rhs.getInterval(); final byte[] cachedValue = cachedValues.get(segmentCacheKey); if (cachedValue != null) { // remove cached segment from set of segments to query segments.remove(segment); cachedResults.add(Pair.of(segmentQueryInterval, cachedValue)); } else if (populateCache) { // otherwise, if populating cache, add segment to list of segments to cache final String segmentIdentifier = segment.lhs.getSegment().getIdentifier(); cachePopulatorMap.put( String.format("%s_%s", segmentIdentifier, segmentQueryInterval), new CachePopulator(cache, objectMapper, segmentCacheKey)); } } } // Compile list of all segments not pulled from cache for (Pair<ServerSelector, SegmentDescriptor> segment : segments) { final QueryableDruidServer queryableDruidServer = segment.lhs.pick(); if (queryableDruidServer == null) { log.makeAlert( "No servers found for SegmentDescriptor[%s] for DataSource[%s]?! How can this be?!", segment.rhs, query.getDataSource()) .emit(); } else { final DruidServer server = queryableDruidServer.getServer(); List<SegmentDescriptor> descriptors = serverSegments.get(server); if (descriptors == null) { descriptors = Lists.newArrayList(); serverSegments.put(server, descriptors); } descriptors.add(segment.rhs); } } return new LazySequence<>( new Supplier<Sequence<T>>() { @Override public Sequence<T> get() { ArrayList<Sequence<T>> sequencesByInterval = Lists.newArrayList(); addSequencesFromCache(sequencesByInterval); addSequencesFromServer(sequencesByInterval); return mergeCachedAndUncachedSequences(query, sequencesByInterval); } private void addSequencesFromCache(ArrayList<Sequence<T>> listOfSequences) { if (strategy == null) { return; } final Function<Object, T> pullFromCacheFunction = strategy.pullFromCache(); final TypeReference<Object> cacheObjectClazz = strategy.getCacheObjectClazz(); for (Pair<Interval, byte[]> cachedResultPair : cachedResults) { final byte[] cachedResult = cachedResultPair.rhs; Sequence<Object> cachedSequence = new BaseSequence<>( new BaseSequence.IteratorMaker<Object, Iterator<Object>>() { @Override public Iterator<Object> make() { try { if (cachedResult.length == 0) { return Iterators.emptyIterator(); } return objectMapper.readValues( objectMapper.getFactory().createParser(cachedResult), cacheObjectClazz); } catch (IOException e) { throw Throwables.propagate(e); } } @Override public void cleanup(Iterator<Object> iterFromMake) {} }); listOfSequences.add(Sequences.map(cachedSequence, pullFromCacheFunction)); } } private void addSequencesFromServer(ArrayList<Sequence<T>> listOfSequences) { listOfSequences.ensureCapacity(listOfSequences.size() + serverSegments.size()); final Query<T> rewrittenQuery = query.withOverriddenContext(contextBuilder.build()); // Loop through each server, setting up the query and initiating it. // The data gets handled as a Future and parsed in the long Sequence chain in the // resultSeqToAdd setter. for (Map.Entry<DruidServer, List<SegmentDescriptor>> entry : serverSegments.entrySet()) { final DruidServer server = entry.getKey(); final List<SegmentDescriptor> descriptors = entry.getValue(); final QueryRunner clientQueryable = serverView.getQueryRunner(server); if (clientQueryable == null) { log.error("WTF!? server[%s] doesn't have a client Queryable?", server); continue; } final MultipleSpecificSegmentSpec segmentSpec = new MultipleSpecificSegmentSpec(descriptors); final Sequence<T> resultSeqToAdd; if (!server.isAssignable() || !populateCache || isBySegment) { // Direct server queryable if (!isBySegment) { resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec), responseContext); } else { // bySegment queries need to be de-serialized, see DirectDruidClient.run() @SuppressWarnings("unchecked") final Query<Result<BySegmentResultValueClass<T>>> bySegmentQuery = (Query<Result<BySegmentResultValueClass<T>>>) ((Query) query); @SuppressWarnings("unchecked") final Sequence<Result<BySegmentResultValueClass<T>>> resultSequence = clientQueryable.run( bySegmentQuery.withQuerySegmentSpec(segmentSpec), responseContext); resultSeqToAdd = (Sequence) Sequences.map( resultSequence, new Function< Result<BySegmentResultValueClass<T>>, Result<BySegmentResultValueClass<T>>>() { @Override public Result<BySegmentResultValueClass<T>> apply( Result<BySegmentResultValueClass<T>> input) { final BySegmentResultValueClass<T> bySegmentValue = input.getValue(); return new Result<>( input.getTimestamp(), new BySegmentResultValueClass<T>( Lists.transform( bySegmentValue.getResults(), toolChest.makePreComputeManipulatorFn( query, MetricManipulatorFns.deserializing())), bySegmentValue.getSegmentId(), bySegmentValue.getInterval())); } }); } } else { // Requires some manipulation on broker side @SuppressWarnings("unchecked") final Sequence<Result<BySegmentResultValueClass<T>>> runningSequence = clientQueryable.run( rewrittenQuery.withQuerySegmentSpec(segmentSpec), responseContext); resultSeqToAdd = new MergeSequence( query.getResultOrdering(), Sequences.<Result<BySegmentResultValueClass<T>>, Sequence<T>>map( runningSequence, new Function<Result<BySegmentResultValueClass<T>>, Sequence<T>>() { private final Function<T, Object> cacheFn = strategy.prepareForCache(); // Acctually do something with the results @Override public Sequence<T> apply(Result<BySegmentResultValueClass<T>> input) { final BySegmentResultValueClass<T> value = input.getValue(); final CachePopulator cachePopulator = cachePopulatorMap.get( String.format( "%s_%s", value.getSegmentId(), value.getInterval())); final Queue<ListenableFuture<Object>> cacheFutures = new ConcurrentLinkedQueue<>(); return Sequences.<T>withEffect( Sequences.<T, T>map( Sequences.<T, T>map( Sequences.<T>simple(value.getResults()), new Function<T, T>() { @Override public T apply(final T input) { if (cachePopulator != null) { // only compute cache data if populating cache cacheFutures.add( backgroundExecutorService.submit( new Callable<Object>() { @Override public Object call() { return cacheFn.apply(input); } })); } return input; } }), toolChest.makePreComputeManipulatorFn( // Ick... most makePreComputeManipulatorFn directly cast // to their ToolChest query type of choice // This casting is sub-optimal, but hasn't caused any // major problems yet... (Query) rewrittenQuery, MetricManipulatorFns.deserializing())), new Runnable() { @Override public void run() { if (cachePopulator != null) { Futures.addCallback( Futures.allAsList(cacheFutures), new FutureCallback<List<Object>>() { @Override public void onSuccess(List<Object> cacheData) { cachePopulator.populate(cacheData); // Help out GC by making sure all references are // gone cacheFutures.clear(); } @Override public void onFailure(Throwable throwable) { log.error(throwable, "Background caching failed"); } }, backgroundExecutorService); } } }, MoreExecutors.sameThreadExecutor()); // End withEffect } })); } listOfSequences.add(resultSeqToAdd); } } } // End of Supplier ); }
@Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { if (this.plumber != null) { throw new IllegalStateException("WTF?!? run with non-null plumber??!"); } // Shed any locks we might have (e.g. if we were uncleanly killed and restarted) since we'll // reacquire // them if we actually need them for (final TaskLock taskLock : getTaskLocks(toolbox)) { toolbox.getTaskActionClient().submit(new LockReleaseAction(taskLock.getInterval())); } boolean normalExit = true; // Set up firehose final Period intermediatePersistPeriod = fireDepartmentConfig.getIntermediatePersistPeriod(); final Firehose firehose = firehoseFactory.connect(); // It would be nice to get the PlumberSchool in the constructor. Although that will need // jackson injectables for // stuff like the ServerView, which seems kind of odd? Perhaps revisit this when Guice has been // introduced. final RealtimePlumberSchool realtimePlumberSchool = new RealtimePlumberSchool( windowPeriod, new File(toolbox.getTaskWorkDir(), "persist"), segmentGranularity); realtimePlumberSchool.setDefaultMaxPendingPersists(maxPendingPersists); final SegmentPublisher segmentPublisher = new TaskActionSegmentPublisher(this, toolbox); // NOTE: We talk to the coordinator in various places in the plumber and we could be more robust // to issues // with the coordinator. Right now, we'll block/throw in whatever thread triggered the // coordinator behavior, // which will typically be either the main data processing loop or the persist thread. // Wrap default DataSegmentAnnouncer such that we unlock intervals as we unannounce segments final DataSegmentAnnouncer lockingSegmentAnnouncer = new DataSegmentAnnouncer() { @Override public void announceSegment(final DataSegment segment) throws IOException { // Side effect: Calling announceSegment causes a lock to be acquired toolbox.getTaskActionClient().submit(new LockAcquireAction(segment.getInterval())); toolbox.getSegmentAnnouncer().announceSegment(segment); } @Override public void unannounceSegment(final DataSegment segment) throws IOException { try { toolbox.getSegmentAnnouncer().unannounceSegment(segment); } finally { toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval())); } } @Override public void announceSegments(Iterable<DataSegment> segments) throws IOException { // Side effect: Calling announceSegments causes locks to be acquired for (DataSegment segment : segments) { toolbox.getTaskActionClient().submit(new LockAcquireAction(segment.getInterval())); } toolbox.getSegmentAnnouncer().announceSegments(segments); } @Override public void unannounceSegments(Iterable<DataSegment> segments) throws IOException { try { toolbox.getSegmentAnnouncer().unannounceSegments(segments); } finally { for (DataSegment segment : segments) { toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval())); } } } }; // NOTE: getVersion will block if there is lock contention, which will block plumber.getSink // NOTE: (and thus the firehose) // Shouldn't usually happen, since we don't expect people to submit tasks that intersect with // the // realtime window, but if they do it can be problematic. If we decide to care, we can use more // threads in // the plumber such that waiting for the coordinator doesn't block data processing. final VersioningPolicy versioningPolicy = new VersioningPolicy() { @Override public String getVersion(final Interval interval) { try { // Side effect: Calling getVersion causes a lock to be acquired final TaskLock myLock = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval)); return myLock.getVersion(); } catch (IOException e) { throw Throwables.propagate(e); } } }; // NOTE: This pusher selects path based purely on global configuration and the DataSegment, // which means // NOTE: that redundant realtime tasks will upload to the same location. This can cause // index.zip and // NOTE: descriptor.json to mismatch, or it can cause historical nodes to load different // instances of the // NOTE: "same" segment. realtimePlumberSchool.setDataSegmentPusher(toolbox.getSegmentPusher()); realtimePlumberSchool.setConglomerate(toolbox.getQueryRunnerFactoryConglomerate()); realtimePlumberSchool.setQueryExecutorService(toolbox.getQueryExecutorService()); realtimePlumberSchool.setVersioningPolicy(versioningPolicy); realtimePlumberSchool.setSegmentAnnouncer(lockingSegmentAnnouncer); realtimePlumberSchool.setSegmentPublisher(segmentPublisher); realtimePlumberSchool.setServerView(toolbox.getNewSegmentServerView()); realtimePlumberSchool.setEmitter(toolbox.getEmitter()); if (this.rejectionPolicyFactory != null) { realtimePlumberSchool.setRejectionPolicyFactory(rejectionPolicyFactory); } final FireDepartment fireDepartment = new FireDepartment(schema, fireDepartmentConfig, null, null); final RealtimeMetricsMonitor metricsMonitor = new RealtimeMetricsMonitor(ImmutableList.of(fireDepartment)); this.queryRunnerFactoryConglomerate = toolbox.getQueryRunnerFactoryConglomerate(); this.plumber = realtimePlumberSchool.findPlumber(schema, fireDepartment.getMetrics()); try { plumber.startJob(); // Set up metrics emission toolbox.getMonitorScheduler().addMonitor(metricsMonitor); // Time to read data! long nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); while (firehose.hasMore()) { final InputRow inputRow; try { inputRow = firehose.nextRow(); if (inputRow == null) { continue; } final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch()); if (sink == null) { fireDepartment.getMetrics().incrementThrownAway(); log.debug("Throwing away event[%s]", inputRow); if (System.currentTimeMillis() > nextFlush) { plumber.persist(firehose.commit()); nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); } continue; } if (sink.isEmpty()) { log.info("Task %s: New sink: %s", getId(), sink); } int currCount = sink.add(inputRow); fireDepartment.getMetrics().incrementProcessed(); if (currCount >= fireDepartmentConfig.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) { plumber.persist(firehose.commit()); nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); } } catch (FormattedException e) { log.warn(e, "unparseable line"); fireDepartment.getMetrics().incrementUnparseable(); } } } catch (Throwable e) { normalExit = false; log.makeAlert(e, "Exception aborted realtime processing[%s]", schema.getDataSource()).emit(); throw e; } finally { if (normalExit) { try { plumber.persist(firehose.commit()); plumber.finishJob(); } catch (Exception e) { log.makeAlert(e, "Failed to finish realtime task").emit(); } finally { Closeables.closeQuietly(firehose); toolbox.getMonitorScheduler().removeMonitor(metricsMonitor); } } } return TaskStatus.success(getId()); }
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(); } }
@POST @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) @Consumes({ MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE, APPLICATION_SMILE }) public Response doPost( InputStream in, @QueryParam("pretty") String pretty, @Context final HttpServletRequest req // used only to get request content-type and remote address ) throws IOException { final long start = System.currentTimeMillis(); Query query = null; String queryId = null; final String reqContentType = req.getContentType(); final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(reqContentType) || APPLICATION_SMILE.equals(reqContentType); final String contentType = isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON; ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; final ObjectWriter jsonWriter = pretty != null ? objectMapper.writerWithDefaultPrettyPrinter() : objectMapper.writer(); final String currThreadName = Thread.currentThread().getName(); try { query = objectMapper.readValue(in, Query.class); queryId = query.getId(); if (queryId == null) { queryId = UUID.randomUUID().toString(); query = query.withId(queryId); } if (query.getContextValue(QueryContextKeys.TIMEOUT) == null) { query = query.withOverriddenContext( ImmutableMap.of( QueryContextKeys.TIMEOUT, config.getMaxIdleTime().toStandardDuration().getMillis())); } Thread.currentThread() .setName( String.format( "%s[%s_%s_%s]", currThreadName, query.getType(), query.getDataSource(), queryId)); if (log.isDebugEnabled()) { log.debug("Got query [%s]", query); } final Map<String, Object> responseContext = new MapMaker().makeMap(); final Sequence res = query.run(texasRanger, responseContext); final Sequence results; if (res == null) { results = Sequences.empty(); } else { results = res; } final Yielder yielder = results.toYielder( null, new YieldingAccumulator() { @Override public Object accumulate(Object accumulated, Object in) { yield(); return in; } }); try { final Query theQuery = query; Response.ResponseBuilder builder = Response.ok( new StreamingOutput() { @Override public void write(OutputStream outputStream) throws IOException, WebApplicationException { // json serializer will always close the yielder CountingOutputStream os = new CountingOutputStream(outputStream); jsonWriter.writeValue(os, yielder); os .flush(); // Some types of OutputStream suppress flush errors in the // .close() method. os.close(); final long queryTime = System.currentTimeMillis() - start; emitter.emit( DruidMetrics.makeQueryTimeMetric( jsonMapper, theQuery, req.getRemoteAddr()) .setDimension("success", "true") .build("query/time", queryTime)); emitter.emit( DruidMetrics.makeQueryTimeMetric( jsonMapper, theQuery, req.getRemoteAddr()) .build("query/bytes", os.getCount())); requestLogger.log( new RequestLogLine( new DateTime(start), req.getRemoteAddr(), theQuery, new QueryStats( ImmutableMap.<String, Object>of( "query/time", queryTime, "query/bytes", os.getCount(), "success", true)))); } }, contentType) .header("X-Druid-Query-Id", queryId); // Limit the response-context header, see https://github.com/druid-io/druid/issues/2331 // Note that Response.ResponseBuilder.header(String key,Object value).build() calls // value.toString() // and encodes the string using ASCII, so 1 char is = 1 byte String responseCtxString = jsonMapper.writeValueAsString(responseContext); if (responseCtxString.length() > RESPONSE_CTX_HEADER_LEN_LIMIT) { log.warn( "Response Context truncated for id [%s] . Full context is [%s].", queryId, responseCtxString); responseCtxString = responseCtxString.substring(0, RESPONSE_CTX_HEADER_LEN_LIMIT); } return builder.header("X-Druid-Response-Context", responseCtxString).build(); } catch (Exception e) { // make sure to close yielder if anything happened before starting to serialize the // response. yielder.close(); throw Throwables.propagate(e); } finally { // do not close yielder here, since we do not want to close the yielder prior to // StreamingOutput having iterated over all the results } } catch (QueryInterruptedException e) { try { log.info("%s [%s]", e.getMessage(), queryId); final long queryTime = System.currentTimeMillis() - start; emitter.emit( DruidMetrics.makeQueryTimeMetric(jsonMapper, query, req.getRemoteAddr()) .setDimension("success", "false") .build("query/time", queryTime)); requestLogger.log( new RequestLogLine( new DateTime(start), req.getRemoteAddr(), query, new QueryStats( ImmutableMap.<String, Object>of( "query/time", queryTime, "success", false, "interrupted", true, "reason", e.toString())))); } catch (Exception e2) { log.error(e2, "Unable to log query [%s]!", query); } return Response.serverError() .type(contentType) .entity( jsonWriter.writeValueAsBytes( ImmutableMap.of( "error", e.getMessage() == null ? "null exception" : e.getMessage()))) .build(); } catch (Exception e) { // Input stream has already been consumed by the json object mapper if query == null final String queryString = query == null ? "unparsable query" : query.toString(); log.warn(e, "Exception occurred on request [%s]", queryString); try { final long queryTime = System.currentTimeMillis() - start; emitter.emit( DruidMetrics.makeQueryTimeMetric(jsonMapper, query, req.getRemoteAddr()) .setDimension("success", "false") .build("query/time", queryTime)); requestLogger.log( new RequestLogLine( new DateTime(start), req.getRemoteAddr(), query, new QueryStats( ImmutableMap.<String, Object>of( "query/time", queryTime, "success", false, "exception", e.toString())))); } catch (Exception e2) { log.error(e2, "Unable to log query [%s]!", queryString); } log.makeAlert(e, "Exception handling request") .addData("exception", e.toString()) .addData("query", queryString) .addData("peer", req.getRemoteAddr()) .emit(); return Response.serverError() .type(contentType) .entity( jsonWriter.writeValueAsBytes( ImmutableMap.of( "error", e.getMessage() == null ? "null exception" : e.getMessage()))) .build(); } finally { Thread.currentThread().setName(currThreadName); } }
@Override public Sequence<Result<SearchResultValue>> run( final Query<Result<SearchResultValue>> input, Map<String, Object> responseContext) { if (!(input instanceof SearchQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SearchQuery.class); } final SearchQuery query = (SearchQuery) input; final Filter filter = Filters.convertDimensionFilters(query.getDimensionsFilter()); final List<DimensionSpec> dimensions = query.getDimensions(); final SearchQuerySpec searchQuerySpec = query.getQuery(); final int limit = query.getLimit(); final boolean descending = query.isDescending(); // Closing this will cause segfaults in unit tests. final QueryableIndex index = segment.asQueryableIndex(); if (index != null) { final TreeMap<SearchHit, MutableInt> retVal = Maps.newTreeMap(query.getSort().getComparator()); Iterable<DimensionSpec> dimsToSearch; if (dimensions == null || dimensions.isEmpty()) { dimsToSearch = Iterables.transform(index.getAvailableDimensions(), Druids.DIMENSION_IDENTITY); } else { dimsToSearch = dimensions; } final BitmapFactory bitmapFactory = index.getBitmapFactoryForDimensions(); final ImmutableBitmap baseFilter = filter == null ? null : filter.getBitmapIndex(new ColumnSelectorBitmapIndexSelector(bitmapFactory, index)); for (DimensionSpec dimension : dimsToSearch) { final Column column = index.getColumn(dimension.getDimension()); if (column == null) { continue; } final BitmapIndex bitmapIndex = column.getBitmapIndex(); ExtractionFn extractionFn = dimension.getExtractionFn(); if (extractionFn == null) { extractionFn = IdentityExtractionFn.getInstance(); } if (bitmapIndex != null) { for (int i = 0; i < bitmapIndex.getCardinality(); ++i) { String dimVal = Strings.nullToEmpty(extractionFn.apply(bitmapIndex.getValue(i))); if (!searchQuerySpec.accept(dimVal)) { continue; } ImmutableBitmap bitmap = bitmapIndex.getBitmap(i); if (baseFilter != null) { bitmap = bitmapFactory.intersection(Arrays.asList(baseFilter, bitmap)); } if (bitmap.size() > 0) { MutableInt counter = new MutableInt(bitmap.size()); MutableInt prev = retVal.put(new SearchHit(dimension.getOutputName(), dimVal), counter); if (prev != null) { counter.add(prev.intValue()); } if (retVal.size() >= limit) { return makeReturnResult(limit, retVal); } } } } } return makeReturnResult(limit, retVal); } final StorageAdapter adapter = segment.asStorageAdapter(); if (adapter == null) { log.makeAlert("WTF!? Unable to process search query on segment.") .addData("segment", segment.getIdentifier()) .addData("query", query) .emit(); throw new ISE( "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped."); } final Iterable<DimensionSpec> dimsToSearch; if (dimensions == null || dimensions.isEmpty()) { dimsToSearch = Iterables.transform(adapter.getAvailableDimensions(), Druids.DIMENSION_IDENTITY); } else { dimsToSearch = dimensions; } final Sequence<Cursor> cursors = adapter.makeCursors(filter, segment.getDataInterval(), QueryGranularity.ALL, descending); final TreeMap<SearchHit, MutableInt> retVal = cursors.accumulate( Maps.<SearchHit, SearchHit, MutableInt>newTreeMap(query.getSort().getComparator()), new Accumulator<TreeMap<SearchHit, MutableInt>, Cursor>() { @Override public TreeMap<SearchHit, MutableInt> accumulate( TreeMap<SearchHit, MutableInt> set, Cursor cursor) { if (set.size() >= limit) { return set; } Map<String, DimensionSelector> dimSelectors = Maps.newHashMap(); for (DimensionSpec dim : dimsToSearch) { dimSelectors.put(dim.getOutputName(), cursor.makeDimensionSelector(dim)); } while (!cursor.isDone()) { for (Map.Entry<String, DimensionSelector> entry : dimSelectors.entrySet()) { final DimensionSelector selector = entry.getValue(); if (selector != null) { final IndexedInts vals = selector.getRow(); for (int i = 0; i < vals.size(); ++i) { final String dimVal = selector.lookupName(vals.get(i)); if (searchQuerySpec.accept(dimVal)) { MutableInt counter = new MutableInt(1); MutableInt prev = set.put(new SearchHit(entry.getKey(), dimVal), counter); if (prev != null) { counter.add(prev.intValue()); } if (set.size() >= limit) { return set; } } } } } cursor.advance(); } return set; } }); return makeReturnResult(limit, retVal); }