@Test(timeout = 60000) public void testPersistWithCommitMetadata() throws Exception { final Object commitMetadata = "dummyCommitMetadata"; testPersist(commitMetadata); plumber = (RealtimePlumber) realtimePlumberSchool.findPlumber(schema, tuningConfig, metrics); Assert.assertEquals(commitMetadata, plumber.startJob()); }
@Before public void setUp() throws Exception { final File tmpDir = Files.createTempDir(); tmpDir.deleteOnExit(); ObjectMapper jsonMapper = new DefaultObjectMapper(); schema = new DataSchema( "test", jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(null, null, null))), Map.class), new AggregatorFactory[] {new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null), jsonMapper); announcer = EasyMock.createMock(DataSegmentAnnouncer.class); announcer.announceSegment(EasyMock.<DataSegment>anyObject()); EasyMock.expectLastCall().anyTimes(); segmentPublisher = EasyMock.createNiceMock(SegmentPublisher.class); dataSegmentPusher = EasyMock.createNiceMock(DataSegmentPusher.class); serverView = EasyMock.createMock(FilteredServerView.class); serverView.registerSegmentCallback( EasyMock.<Executor>anyObject(), EasyMock.<ServerView.SegmentCallback>anyObject(), EasyMock.<Predicate<DataSegment>>anyObject()); EasyMock.expectLastCall().anyTimes(); emitter = EasyMock.createMock(ServiceEmitter.class); EasyMock.replay(announcer, segmentPublisher, dataSegmentPusher, serverView, emitter); tuningConfig = new RealtimeTuningConfig( 1, null, null, null, new IntervalStartVersioningPolicy(), rejectionPolicy, null, null, null, null, null, null, null); realtimePlumberSchool = new RealtimePlumberSchool( emitter, new DefaultQueryRunnerFactoryConglomerate( Maps.<Class<? extends Query>, QueryRunnerFactory>newHashMap()), dataSegmentPusher, announcer, segmentPublisher, serverView, MoreExecutors.sameThreadExecutor()); metrics = new FireDepartmentMetrics(); plumber = (RealtimePlumber) realtimePlumberSchool.findPlumber(schema, tuningConfig, metrics); }
@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()); }