@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); }
@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); } }
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 DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { DatasourceWhitelist whitelist = whitelistRef.get(); for (DataSegment dataSegment : params.getAvailableSegments()) { if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) { final Integer binaryVersion = dataSegment.getBinaryVersion(); if (binaryVersion == null || binaryVersion < IndexIO.CURRENT_VERSION_ID) { log.info("Upgrading version on segment[%s]", dataSegment.getIdentifier()); indexingServiceClient.upgradeSegment(dataSegment); } } } return params; }
@LifecycleStop public void stop() { log.info("Stopping ZkCoordinator for [%s]", me); synchronized (lock) { if (!started) { return; } try { loadQueueCache.close(); } catch (Exception e) { throw Throwables.propagate(e); } finally { loadQueueCache = null; started = false; } } }
protected void addSingleInventory(final DruidServer container, final DataSegment inventory) { log.info("Server[%s] added segment[%s]", container.getName(), inventory.getIdentifier()); if (container.getSegment(inventory.getIdentifier()) != null) { log.warn( "Not adding or running callbacks for existing segment[%s] on server[%s]", inventory.getIdentifier(), container.getName()); return; } container.addDataSegment(inventory.getIdentifier(), inventory); runSegmentCallbacks( new Function<SegmentCallback, CallbackAction>() { @Override public CallbackAction apply(SegmentCallback input) { return input.segmentAdded(container.getMetadata(), inventory); } }); }
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(); } } }
protected void removeSingleInventory(final DruidServer container, String inventoryKey) { log.info("Server[%s] removed segment[%s]", container.getName(), inventoryKey); final DataSegment segment = container.getSegment(inventoryKey); if (segment == null) { log.warn( "Not running cleanup or callbacks for non-existing segment[%s] on server[%s]", inventoryKey, container.getName()); return; } container.removeDataSegment(inventoryKey); runSegmentCallbacks( new Function<SegmentCallback, CallbackAction>() { @Override public CallbackAction apply(SegmentCallback input) { return input.segmentRemoved(container.getMetadata(), segment); } }); }
@Override public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics) { verifyState(); final RejectionPolicy rejectionPolicy = rejectionPolicyFactory.create(windowPeriod); log.info("Creating plumber using rejectionPolicy[%s]", rejectionPolicy); return new RealtimePlumber( windowPeriod, basePersistDirectory, segmentGranularity, schema, metrics, rejectionPolicy, emitter, conglomerate, segmentAnnouncer, queryExecutorService, versioningPolicy, dataSegmentPusher, segmentPublisher, serverView, maxPendingPersists); }
@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 removeSegment(DataSegment segment) { log.info("Removing Segment[%s]", segment); databaseSegmentManager.removeSegment(segment.getDataSource(), segment.getIdentifier()); }
@LifecycleStart public void start() throws IOException { synchronized (lock) { if (started) { return; } log.info("Starting zkCoordinator for server[%s]", me.getName()); final String loadQueueLocation = ZKPaths.makePath(zkPaths.getLoadQueuePath(), me.getName()); final String servedSegmentsLocation = ZKPaths.makePath(zkPaths.getServedSegmentsPath(), me.getName()); final String liveSegmentsLocation = ZKPaths.makePath(zkPaths.getLiveSegmentsPath(), me.getName()); loadQueueCache = new PathChildrenCache(curator, loadQueueLocation, true, true, loadingExec); try { curator.newNamespaceAwareEnsurePath(loadQueueLocation).ensure(curator.getZookeeperClient()); curator .newNamespaceAwareEnsurePath(servedSegmentsLocation) .ensure(curator.getZookeeperClient()); curator .newNamespaceAwareEnsurePath(liveSegmentsLocation) .ensure(curator.getZookeeperClient()); loadLocalCache(); loadQueueCache .getListenable() .addListener( new PathChildrenCacheListener() { @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { final ChildData child = event.getData(); switch (event.getType()) { case CHILD_ADDED: final String path = child.getPath(); final DataSegmentChangeRequest request = jsonMapper.readValue(child.getData(), DataSegmentChangeRequest.class); log.info("New request[%s] with zNode[%s].", request.asString(), path); try { request.go( getDataSegmentChangeHandler(), new DataSegmentChangeCallback() { boolean hasRun = false; @Override public void execute() { try { if (!hasRun) { curator.delete().guaranteed().forPath(path); log.info("Completed request [%s]", request.asString()); hasRun = true; } } catch (Exception e) { try { curator.delete().guaranteed().forPath(path); } catch (Exception e1) { log.error( e1, "Failed to delete zNode[%s], but ignoring exception.", path); } log.error(e, "Exception while removing zNode[%s]", path); throw Throwables.propagate(e); } } }); } catch (Exception e) { try { curator.delete().guaranteed().forPath(path); } catch (Exception e1) { log.error( e1, "Failed to delete zNode[%s], but ignoring exception.", path); } log.makeAlert(e, "Segment load/unload: uncaught exception.") .addData("node", path) .addData("nodeProperties", request) .emit(); } break; case CHILD_REMOVED: log.info("zNode[%s] was removed", event.getData().getPath()); break; default: log.info("Ignoring event[%s]", event); } } }); loadQueueCache.start(); } catch (Exception e) { Throwables.propagateIfPossible(e, IOException.class); throw Throwables.propagate(e); } started = true; } }
@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); } }
@LifecycleStart public void start() throws IOException { log.info("Starting zkCoordinator for server[%s]", me); synchronized (lock) { if (started) { return; } final String loadQueueLocation = ZKPaths.makePath(zkPaths.getLoadQueuePath(), me.getName()); final String servedSegmentsLocation = ZKPaths.makePath(zkPaths.getServedSegmentsPath(), me.getName()); final String liveSegmentsLocation = ZKPaths.makePath(zkPaths.getLiveSegmentsPath(), me.getName()); loadQueueCache = new PathChildrenCache( curator, loadQueueLocation, true, true, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ZkCoordinator-%s").build()); try { config.getInfoDir().mkdirs(); curator.newNamespaceAwareEnsurePath(loadQueueLocation).ensure(curator.getZookeeperClient()); curator .newNamespaceAwareEnsurePath(servedSegmentsLocation) .ensure(curator.getZookeeperClient()); curator .newNamespaceAwareEnsurePath(liveSegmentsLocation) .ensure(curator.getZookeeperClient()); loadCache(); loadQueueCache .getListenable() .addListener( new PathChildrenCacheListener() { @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { final ChildData child = event.getData(); switch (event.getType()) { case CHILD_ADDED: final String path = child.getPath(); final DataSegmentChangeRequest segment = jsonMapper.readValue(child.getData(), DataSegmentChangeRequest.class); log.info("New node[%s] with segmentClass[%s]", path, segment.getClass()); try { segment.go(ZkCoordinator.this); curator.delete().guaranteed().forPath(path); log.info("Completed processing for node[%s]", path); } catch (Exception e) { try { curator.delete().guaranteed().forPath(path); } catch (Exception e1) { log.info( e1, "Failed to delete node[%s], but ignoring exception.", path); } log.makeAlert(e, "Segment load/unload: uncaught exception.") .addData("node", path) .addData("nodeProperties", segment) .emit(); } break; case CHILD_REMOVED: log.info("%s was removed", event.getData().getPath()); break; default: log.info("Ignoring event[%s]", event); } } }); loadQueueCache.start(); } catch (Exception e) { Throwables.propagateIfPossible(e, IOException.class); throw Throwables.propagate(e); } started = true; } }