From f774d3cc0894b7d0e6a1475c8831440dfaf39b34 Mon Sep 17 00:00:00 2001 From: Kevin Conaway Date: Sun, 14 Jan 2018 21:49:45 -0500 Subject: [PATCH 01/10] Change RealtimeIndexTask to use AppenderatorDriver instead of RealtimePlumber. Related to #4774 --- .../common/task/RealtimeIndexTask.java | 330 +++++++++----- .../common/task/RealtimeIndexTaskTest.java | 417 +++++++++++++----- .../indexing/common/task/TaskSerdeTest.java | 1 + .../indexing/overlord/TaskLifecycleTest.java | 68 ++- .../indexing/RealtimeTuningConfig.java | 13 + .../segment/realtime/RealtimeManagerTest.java | 3 + .../appenderator/AppenderatorPlumberTest.java | 1 + .../appenderator/AppenderatorTester.java | 1 + ...DefaultOfflineAppenderatorFactoryTest.java | 1 + .../plumber/RealtimePlumberSchoolTest.java | 1 + .../segment/realtime/plumber/SinkTest.java | 1 + .../cli/validate/DruidJsonValidatorTest.java | 1 + 12 files changed, 602 insertions(+), 236 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index fa6246a1df86..f90ee3657061 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -28,59 +28,69 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; +import io.druid.data.input.InputRow; import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.discovery.LookupNodeService; +import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; +import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockAcquireAction; import io.druid.indexing.common.actions.LockReleaseAction; +import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; +import io.druid.java.util.common.parsers.ParseException; import io.druid.query.DruidMetrics; -import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.NoopQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; -import io.druid.query.QueryRunnerFactory; -import io.druid.query.QueryRunnerFactoryConglomerate; -import io.druid.query.QueryToolChest; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.FireDepartment; import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.RealtimeMetricsMonitor; -import io.druid.segment.realtime.SegmentPublisher; +import io.druid.segment.realtime.appenderator.Appenderator; +import io.druid.segment.realtime.appenderator.AppenderatorDriver; +import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; +import io.druid.segment.realtime.appenderator.Appenderators; +import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import io.druid.segment.realtime.firehose.ClippedFirehoseFactory; import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; import io.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; import io.druid.segment.realtime.plumber.Committers; -import io.druid.segment.realtime.plumber.Plumber; -import io.druid.segment.realtime.plumber.PlumberSchool; -import io.druid.segment.realtime.plumber.Plumbers; -import io.druid.segment.realtime.plumber.RealtimePlumberSchool; import io.druid.segment.realtime.plumber.VersioningPolicy; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; -import org.joda.time.Interval; import java.io.File; import java.io.IOException; +import java.util.Collections; import java.util.Map; +import java.util.Queue; import java.util.Random; import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; public class RealtimeIndexTask extends AbstractTask { @@ -123,7 +133,16 @@ private static String makeDatasource(FireDepartment fireDepartment) private final FireDepartment spec; @JsonIgnore - private volatile Plumber plumber = null; + private final Queue> pendingHandoffs; + + @JsonIgnore + private final AtomicReference throwableDuringPublishing; + + @JsonIgnore + private volatile AppenderatorDriver driver = null; + + @JsonIgnore + private volatile Appenderator appenderator = null; @JsonIgnore private volatile Firehose firehose = null; @@ -140,9 +159,6 @@ private static String makeDatasource(FireDepartment fireDepartment) @JsonIgnore private volatile Thread runThread = null; - @JsonIgnore - private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null; - @JsonCreator public RealtimeIndexTask( @JsonProperty("id") String id, @@ -159,6 +175,8 @@ public RealtimeIndexTask( context ); this.spec = fireDepartment; + this.pendingHandoffs = new ConcurrentLinkedQueue<>(); + this.throwableDuringPublishing = new AtomicReference<>(); } @Override @@ -182,14 +200,12 @@ public String getNodeType() @Override public QueryRunner getQueryRunner(Query query) { - if (plumber != null) { - QueryRunnerFactory> factory = queryRunnerFactoryConglomerate.findFactory(query); - QueryToolChest> toolChest = factory.getToolchest(); - - return new FinalizeResultsQueryRunner(plumber.getQueryRunner(query), toolChest); - } else { - return null; + if (appenderator == null) { + // Not yet initialized, no data yet, just return a noop runner. + return new NoopQueryRunner<>(); } + + return (queryPlus, responseContext) -> queryPlus.run(appenderator, responseContext); } @Override @@ -203,23 +219,10 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception { runThread = Thread.currentThread(); - if (this.plumber != null) { - throw new IllegalStateException("WTF?!? run with non-null plumber??!"); - } - setupTimeoutAlert(); boolean normalExit = true; - // 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 SegmentPublisher segmentPublisher = new TaskActionSegmentPublisher(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 long lockTimeoutMs = getContextValue(Tasks.LOCK_TIMEOUT_KEY, Tasks.DEFAULT_LOCK_TIMEOUT); // Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, http timeout error can occur while waiting for a @@ -281,31 +284,25 @@ public void unannounceSegments(Iterable segments) throws IOExceptio } }; - // NOTE: getVersion will block if there is lock contention, which will block plumber.getSink - // NOTE: (and thus the firehose) + // NOTE: getVersion will block (and thus block the firehose) if there is lock contention // 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 LockAcquireAction action = new LockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs); - final TaskLock lock = Preconditions.checkNotNull( - toolbox.getTaskActionClient().submit(action), - "Cannot acquire a lock for interval[%s]", - interval - ); + final VersioningPolicy versioningPolicy = interval -> { + try { + // Side effect: Calling getVersion causes a lock to be acquired + final LockAcquireAction action = new LockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs); + final TaskLock lock = Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit(action), + "Cannot acquire a lock for interval[%s]", + interval + ); - return lock.getVersion(); - } - catch (IOException e) { - throw Throwables.propagate(e); - } + return lock.getVersion(); + } + catch (IOException e) { + throw Throwables.propagate(e); } }; @@ -327,29 +324,6 @@ public String getVersion(final Interval interval) DruidMetrics.TASK_ID, new String[]{getId()} ) ); - this.queryRunnerFactoryConglomerate = toolbox.getQueryRunnerFactoryConglomerate(); - - // 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 - // NOTE: (partitionNum_index.zip for HDFS data storage) and descriptor.json (partitionNum_descriptor.json for - // NOTE: HDFS data storage) to mismatch, or it can cause historical nodes to load different instances of - // NOTE: the "same" segment. - final PlumberSchool plumberSchool = new RealtimePlumberSchool( - toolbox.getEmitter(), - toolbox.getQueryRunnerFactoryConglomerate(), - toolbox.getSegmentPusher(), - lockingSegmentAnnouncer, - segmentPublisher, - toolbox.getSegmentHandoffNotifierFactory(), - toolbox.getQueryExecutorService(), - toolbox.getIndexMergerV9(), - toolbox.getIndexIO(), - toolbox.getCache(), - toolbox.getCacheConfig(), - toolbox.getObjectMapper() - ); - - this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, metrics); Supplier committerSupplier = null; final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); @@ -366,12 +340,14 @@ public String getVersion(final Interval interval) ) ); + appenderator = newAppenderator(dataSchema, tuningConfig, fireDepartment.getMetrics(), toolbox, lockingSegmentAnnouncer); + driver = newDriver(dataSchema, appenderator, toolbox, fireDepartment.getMetrics()); + try { toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); - - plumber.startJob(); + driver.startJob(); // Set up metrics emission toolbox.getMonitorScheduler().addMonitor(metricsMonitor); @@ -391,16 +367,90 @@ public String getVersion(final Interval interval) } } + int sequenceNumber = 0; + String sequenceName = makeSequenceName(sequenceNumber); + + final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { + final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments); + return toolbox.getTaskActionClient().submit(action).isSuccess(); + }; + // Time to read data! while (firehose != null && (!gracefullyStopped || firehoseDrainableByClosing) && firehose.hasMore()) { - Plumbers.addNextRow( - committerSupplier, - firehose, - plumber, - tuningConfig.isReportParseExceptions(), - metrics - ); + if (throwableDuringPublishing.get() != null) { + Throwables.propagate(throwableDuringPublishing.get()); + } + + try { + InputRow inputRow = firehose.nextRow(); + + if (inputRow == null) { + log.debug("Discarded null row, considering thrownAway."); + metrics.incrementThrownAway(); + } else { + AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName, committerSupplier); + + if (addResult.isOk()) { + if (addResult.isPersistRequired()) { + driver.persist(committerSupplier.get()); + } + + if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) { + publishSegments(publisher, committerSupplier, sequenceName); + + sequenceNumber++; + sequenceName = makeSequenceName(sequenceNumber); + } + } else { + // Failure to allocate segment puts determinism at risk, bail out to be safe. + // May want configurable behavior here at some point. + // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. + throw new ISE("Could not allocate segment for row with timestamp[%s]", inputRow.getTimestamp()); + } + + metrics.incrementProcessed(); + } + } + catch (ParseException e) { + if (tuningConfig.isReportParseExceptions()) { + throw e; + } else { + log.debug(e, "Discarded row due to exception, considering unparseable."); + metrics.incrementUnparseable(); + } + } + } + + if (!gracefullyStopped) { + synchronized (this) { + if (gracefullyStopped) { + // Someone called stopGracefully after we checked the flag. That's okay, just stop now. + log.info("Gracefully stopping."); + } else { + finishingJob = true; + } + } + + if (finishingJob) { + log.info("Finishing job..."); + // Publish any remaining segments + publishSegments(publisher, committerSupplier, sequenceName); + + if (!pendingHandoffs.isEmpty()) { + ListenableFuture allHandoffs = Futures.allAsList(pendingHandoffs); + log.info("Waiting for handoffs"); + + long handoffTimeout = tuningConfig.getHandoffConditionTimeout(); + + if (handoffTimeout > 0) { + allHandoffs.get(handoffTimeout, TimeUnit.MILLISECONDS); + } else { + allHandoffs.get(); + } + } + } } + } catch (Throwable e) { normalExit = false; @@ -417,7 +467,7 @@ public String getVersion(final Interval interval) final Committer committer = committerSupplier.get(); final CountDownLatch persistLatch = new CountDownLatch(1); - plumber.persist( + driver.persist( new Committer() { @Override @@ -440,24 +490,6 @@ public void run() ); persistLatch.await(); } - - if (gracefullyStopped) { - log.info("Gracefully stopping."); - } else { - log.info("Finishing the job."); - synchronized (this) { - if (gracefullyStopped) { - // Someone called stopGracefully after we checked the flag. That's okay, just stop now. - log.info("Gracefully stopping."); - } else { - finishingJob = true; - } - } - - if (finishingJob) { - plumber.finishJob(); - } - } } catch (InterruptedException e) { log.debug(e, "Interrupted while finishing the job"); @@ -474,6 +506,14 @@ public void run() } } + if (appenderator != null) { + appenderator.close(); + } + + if (driver != null) { + driver.close(); + } + toolbox.getDataSegmentServerAnnouncer().unannounce(); toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); } @@ -556,22 +596,6 @@ && isFirehoseDrainableByClosing(((TimedShutoffFirehoseFactory) firehoseFactory). && isFirehoseDrainableByClosing(((ClippedFirehoseFactory) firehoseFactory).getDelegate())); } - public static class TaskActionSegmentPublisher implements SegmentPublisher - { - final TaskToolbox taskToolbox; - - public TaskActionSegmentPublisher(TaskToolbox taskToolbox) - { - this.taskToolbox = taskToolbox; - } - - @Override - public void publishSegment(DataSegment segment) throws IOException - { - taskToolbox.publishSegments(ImmutableList.of(segment)); - } - } - private void setupTimeoutAlert() { if (spec.getTuningConfig().getAlertTimeout() > 0) { @@ -593,4 +617,68 @@ public void run() ); } } + + private Appenderator newAppenderator( + final DataSchema dataSchema, + final RealtimeTuningConfig tuningConfig, + final FireDepartmentMetrics metrics, + final TaskToolbox toolbox, + final DataSegmentAnnouncer segmentAnnouncer + ) + { + return Appenderators.createRealtime( + dataSchema, + tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + toolbox.getSegmentPusher(), + toolbox.getObjectMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + toolbox.getQueryRunnerFactoryConglomerate(), + segmentAnnouncer, + toolbox.getEmitter(), + toolbox.getQueryExecutorService(), + toolbox.getCache(), + toolbox.getCacheConfig() + ); + } + + private AppenderatorDriver newDriver( + final DataSchema dataSchema, + final Appenderator appenderator, + final TaskToolbox toolbox, + final FireDepartmentMetrics metrics + ) + { + return new AppenderatorDriver( + appenderator, + new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema), + toolbox.getSegmentHandoffNotifierFactory(), + new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), + toolbox.getObjectMapper(), + metrics + ); + } + + private void publishSegments( + TransactionalSegmentPublisher publisher, + Supplier committerSupplier, + String sequenceName + ) + { + ListenableFuture publishFuture = driver.publish( + publisher, + committerSupplier.get(), + Collections.singletonList(sequenceName) + ); + + ListenableFuture handoffFuture = Futures.transform(publishFuture, driver::registerHandoff); + + pendingHandoffs.add(handoffFuture); + } + + private String makeSequenceName(int sequenceNumber) + { + return String.format("%s_%d", getId(), sequenceNumber); + } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 5606219396f3..d1b4bebea305 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -23,10 +23,8 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; 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.Sets; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; @@ -58,15 +56,15 @@ import io.druid.indexing.common.actions.TaskActionToolbox; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskStorageConfig; +import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.HeapMemoryTaskStorage; -import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import io.druid.indexing.overlord.SegmentPublishResult; import io.druid.indexing.overlord.TaskLockbox; import io.druid.indexing.overlord.TaskStorage; import io.druid.indexing.overlord.supervisor.SupervisorManager; import io.druid.indexing.test.TestDataSegmentAnnouncer; import io.druid.indexing.test.TestDataSegmentKiller; import io.druid.indexing.test.TestDataSegmentPusher; -import io.druid.indexing.test.TestIndexerMetadataStorageCoordinator; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; @@ -79,6 +77,8 @@ import io.druid.java.util.common.parsers.ParseException; import io.druid.math.expr.ExprMacroTable; import io.druid.metadata.EntryExistsException; +import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import io.druid.metadata.TestDerbyConnector; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Druids; import io.druid.query.IntervalChunkingQueryRunnerDecorator; @@ -119,6 +119,9 @@ import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.LinearShardSpec; +import io.druid.timeline.partition.NumberedShardSpec; +import org.apache.commons.io.FileUtils; import org.easymock.EasyMock; import org.hamcrest.CoreMatchers; import org.joda.time.DateTime; @@ -137,13 +140,19 @@ import java.io.IOException; import java.nio.file.Files; import java.util.Arrays; +import java.util.Collection; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; public class RealtimeIndexTaskTest { @@ -240,9 +249,15 @@ public Firehose connect(InputRowParser parser, File temporaryDirectory) throws I @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + private DateTime now; private ListeningExecutorService taskExec; private Map> handOffCallbacks; + private Collection publishedSegments; + private CountDownLatch segmentLatch; + private CountDownLatch handoffLatch; @Before public void setUp() @@ -251,6 +266,12 @@ public void setUp() emitter.start(); taskExec = MoreExecutors.listeningDecorator(Execs.singleThreaded("realtime-index-task-test-%d")); now = DateTimes.nowUtc(); + + TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); + derbyConnector.createDataSourceTable(); + derbyConnector.createTaskTables(); + derbyConnector.createSegmentTable(); + derbyConnector.createPendingSegmentsTable(); } @After @@ -279,9 +300,9 @@ public void testDefaultResource() throws Exception @Test(timeout = 60_000L, expected = ExecutionException.class) public void testHandoffTimeout() throws Exception { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + expectPublishedSegments(1); final RealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, true, 100L); - final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); final ListenableFuture statusFuture = runTask(task, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -300,14 +321,10 @@ public void testHandoffTimeout() throws Exception // Stop the firehose, this will drain out existing events. firehose.close(); - // Wait for publish. - while (mdc.getPublished().isEmpty()) { - Thread.sleep(50); - } + Collection publishedSegments = awaitSegments(); Assert.assertEquals(1, task.getMetrics().processed()); - Assert.assertNotNull(Iterables.getOnlyElement(mdc.getPublished())); - + Assert.assertNotNull(Iterables.getOnlyElement(publishedSegments)); // handoff would timeout, resulting in exception statusFuture.get(); @@ -316,11 +333,10 @@ public void testHandoffTimeout() throws Exception @Test(timeout = 60_000L) public void testBasics() throws Exception { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + expectPublishedSegments(1); final RealtimeIndexTask task = makeRealtimeTask(null); - final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); final ListenableFuture statusFuture = runTask(task, taskToolbox); - final DataSegment publishedSegment; // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -332,7 +348,6 @@ public void testBasics() throws Exception firehose.addRows( ImmutableList.of( ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), - ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", 2.0), ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) ) ); @@ -341,32 +356,168 @@ public void testBasics() throws Exception firehose.close(); // Wait for publish. - while (mdc.getPublished().isEmpty()) { + Collection publishedSegments = awaitSegments(); + + // Check metrics. + Assert.assertEquals(2, task.getMetrics().processed()); + Assert.assertEquals(0, task.getMetrics().thrownAway()); + Assert.assertEquals(0, task.getMetrics().unparseable()); + + // Do some queries. + Assert.assertEquals(2, sumMetric(task, null, "rows")); + Assert.assertEquals(3, sumMetric(task, null, "met1")); + + awaitHandoffs(); + + for (DataSegment publishedSegment : publishedSegments) { + Optional>> optional = handOffCallbacks.entrySet().stream() + .filter(e -> e.getKey().equals(new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ))) + .findFirst(); + + Assert.assertTrue( + publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, + optional.isPresent() + ); + Pair executorRunnablePair = optional.get().getValue(); + + // Simulate handoff. + executorRunnablePair.lhs.execute(executorRunnablePair.rhs); + } + handOffCallbacks.clear(); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + } + + @Test(timeout = 60_000L) + public void testLateData() throws Exception + { + expectPublishedSegments(1); + final RealtimeIndexTask task = makeRealtimeTask(null); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final ListenableFuture statusFuture = runTask(task, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { Thread.sleep(50); } - publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), + // Data is from 2 days ago, should still be processed + ImmutableMap.of("t", now.minus(new Period("P2D")).getMillis(), "dim2", "bar", "met1", 2.0) + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for publish. + Collection publishedSegments = awaitSegments(); // Check metrics. Assert.assertEquals(2, task.getMetrics().processed()); - Assert.assertEquals(1, task.getMetrics().thrownAway()); + Assert.assertEquals(0, task.getMetrics().thrownAway()); Assert.assertEquals(0, task.getMetrics().unparseable()); // Do some queries. Assert.assertEquals(2, sumMetric(task, null, "rows")); Assert.assertEquals(3, sumMetric(task, null, "met1")); - // Simulate handoff. - for (Map.Entry> entry : handOffCallbacks.entrySet()) { - final Pair executorRunnablePair = entry.getValue(); - Assert.assertEquals( - new SegmentDescriptor( + awaitHandoffs(); + + for (DataSegment publishedSegment : publishedSegments) { + Optional>> optional = handOffCallbacks.entrySet().stream() + .filter(e -> e.getKey().equals(new SegmentDescriptor( publishedSegment.getInterval(), publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() - ), - entry.getKey() + ))) + .findFirst(); + + Assert.assertTrue( + publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, + optional.isPresent() + ); + Pair executorRunnablePair = optional.get().getValue(); + + // Simulate handoff. + executorRunnablePair.lhs.execute(executorRunnablePair.rhs); + } + handOffCallbacks.clear(); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + } + + @Test(timeout = 60_000L) + public void testMaxRowsPerSegment() throws Exception + { + // Expect 2 segments as we will hit maxRowsPerSegment + expectPublishedSegments(2); + + final RealtimeIndexTask task = makeRealtimeTask(null); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final ListenableFuture statusFuture = runTask(task, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + // maxRowsPerSegment is 1000 as configured in #makeRealtimeTask + for (int i = 0; i < 2000; i++) { + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo-" + i, "met1", "1") + ) + ); + } + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for publish. + Collection publishedSegments = awaitSegments(); + + // Check metrics. + Assert.assertEquals(2000, task.getMetrics().processed()); + Assert.assertEquals(0, task.getMetrics().thrownAway()); + Assert.assertEquals(0, task.getMetrics().unparseable()); + + // Do some queries. + Assert.assertEquals(2000, sumMetric(task, null, "rows")); + Assert.assertEquals(2000, sumMetric(task, null, "met1")); + + awaitHandoffs(); + + for (DataSegment publishedSegment : publishedSegments) { + Optional>> optional = handOffCallbacks.entrySet().stream() + .filter(e -> e.getKey().equals(new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ))) + .findFirst(); + + Assert.assertTrue( + publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, + optional.isPresent() ); + Pair executorRunnablePair = optional.get().getValue(); + + // Simulate handoff. executorRunnablePair.lhs.execute(executorRunnablePair.rhs); } handOffCallbacks.clear(); @@ -379,7 +530,8 @@ public void testBasics() throws Exception @Test(timeout = 60_000L) public void testTransformSpec() throws Exception { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + expectPublishedSegments(2); + final TransformSpec transformSpec = new TransformSpec( new SelectorDimFilter("dim1", "foo", null), ImmutableList.of( @@ -387,9 +539,8 @@ public void testTransformSpec() throws Exception ) ); final RealtimeIndexTask task = makeRealtimeTask(null, transformSpec, true, 0); - final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); final ListenableFuture statusFuture = runTask(task, taskToolbox); - final DataSegment publishedSegment; // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -409,35 +560,37 @@ public void testTransformSpec() throws Exception // Stop the firehose, this will drain out existing events. firehose.close(); - // Wait for publish. - while (mdc.getPublished().isEmpty()) { - Thread.sleep(50); - } - - publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); + Collection publishedSegments = awaitSegments(); // Check metrics. - Assert.assertEquals(1, task.getMetrics().processed()); - Assert.assertEquals(2, task.getMetrics().thrownAway()); + Assert.assertEquals(2, task.getMetrics().processed()); + Assert.assertEquals(1, task.getMetrics().thrownAway()); Assert.assertEquals(0, task.getMetrics().unparseable()); // Do some queries. - Assert.assertEquals(1, sumMetric(task, null, "rows")); - Assert.assertEquals(1, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows")); + Assert.assertEquals(2, sumMetric(task, null, "rows")); + Assert.assertEquals(2, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows")); Assert.assertEquals(0, sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "rows")); - Assert.assertEquals(1, sumMetric(task, null, "met1")); + Assert.assertEquals(3, sumMetric(task, null, "met1")); - // Simulate handoff. - for (Map.Entry> entry : handOffCallbacks.entrySet()) { - final Pair executorRunnablePair = entry.getValue(); - Assert.assertEquals( - new SegmentDescriptor( + awaitHandoffs(); + + for (DataSegment publishedSegment : publishedSegments) { + Optional>> optional = handOffCallbacks.entrySet().stream() + .filter(e -> e.getKey().equals(new SegmentDescriptor( publishedSegment.getInterval(), publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() - ), - entry.getKey() + ))) + .findFirst(); + + Assert.assertTrue( + publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, + optional.isPresent() ); + Pair executorRunnablePair = optional.get().getValue(); + + // Simulate handoff. executorRunnablePair.lhs.execute(executorRunnablePair.rhs); } handOffCallbacks.clear(); @@ -450,9 +603,9 @@ public void testTransformSpec() throws Exception @Test(timeout = 60_000L) public void testReportParseExceptionsOnBadMetric() throws Exception { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + expectPublishedSegments(0); final RealtimeIndexTask task = makeRealtimeTask(null, true); - final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); final ListenableFuture statusFuture = runTask(task, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -500,11 +653,11 @@ public void testReportParseExceptionsOnBadMetric() throws Exception @Test(timeout = 60_000L) public void testNoReportParseExceptions() throws Exception { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + expectPublishedSegments(1); + final RealtimeIndexTask task = makeRealtimeTask(null, false); - final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); final ListenableFuture statusFuture = runTask(task, taskToolbox); - final DataSegment publishedSegment; // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -527,9 +680,6 @@ public void testNoReportParseExceptions() throws Exception // Bad row- will be unparseable. ImmutableMap.of("dim1", "foo", "met1", 2.0, FAIL_DIM, "x"), - // Old row- will be thrownAway. - ImmutableMap.of("t", now.minus(Period.days(1)).getMillis(), "dim1", "foo", "met1", 2.0), - // Good row- will be processed. ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) ) @@ -539,21 +689,21 @@ public void testNoReportParseExceptions() throws Exception firehose.close(); // Wait for publish. - while (mdc.getPublished().isEmpty()) { - Thread.sleep(50); - } + Collection publishedSegments = awaitSegments(); - publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); + DataSegment publishedSegment = Iterables.getOnlyElement(publishedSegments); // Check metrics. Assert.assertEquals(3, task.getMetrics().processed()); - Assert.assertEquals(1, task.getMetrics().thrownAway()); + Assert.assertEquals(0, task.getMetrics().thrownAway()); Assert.assertEquals(2, task.getMetrics().unparseable()); // Do some queries. Assert.assertEquals(3, sumMetric(task, null, "rows")); Assert.assertEquals(3, sumMetric(task, null, "met1")); + awaitHandoffs(); + // Simulate handoff. for (Map.Entry> entry : handOffCallbacks.entrySet()) { final Pair executorRunnablePair = entry.getValue(); @@ -577,14 +727,15 @@ public void testNoReportParseExceptions() throws Exception @Test(timeout = 60_000L) public void testRestore() throws Exception { + expectPublishedSegments(0); + final File directory = tempFolder.newFolder(); final RealtimeIndexTask task1 = makeRealtimeTask(null); final DataSegment publishedSegment; // First run: { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory); + final TaskToolbox taskToolbox = makeToolbox(task1, directory); final ListenableFuture statusFuture = runTask(task1, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -608,14 +759,14 @@ public void testRestore() throws Exception Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); // Nothing should be published. - Assert.assertEquals(Sets.newHashSet(), mdc.getPublished()); + Assert.assertTrue(publishedSegments.isEmpty()); } // Second run: { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + expectPublishedSegments(1); final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, mdc, directory); + final TaskToolbox taskToolbox = makeToolbox(task2, directory); final ListenableFuture statusFuture = runTask(task2, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -637,16 +788,15 @@ public void testRestore() throws Exception // Stop the firehose, this will drain out existing events. firehose.close(); - // Wait for publish. - while (mdc.getPublished().isEmpty()) { - Thread.sleep(50); - } + Collection publishedSegments = awaitSegments(); - publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); + publishedSegment = Iterables.getOnlyElement(publishedSegments); // Do a query. Assert.assertEquals(2, sumMetric(task2, null, "rows")); + awaitHandoffs(); + // Simulate handoff. for (Map.Entry> entry : handOffCallbacks.entrySet()) { final Pair executorRunnablePair = entry.getValue(); @@ -672,14 +822,14 @@ public void testRestore() throws Exception public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception { final TaskStorage taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); final File directory = tempFolder.newFolder(); final RealtimeIndexTask task1 = makeRealtimeTask(null); final DataSegment publishedSegment; // First run: { - final TaskToolbox taskToolbox = makeToolbox(task1, taskStorage, mdc, directory); + expectPublishedSegments(1); + final TaskToolbox taskToolbox = makeToolbox(task1, taskStorage, directory); final ListenableFuture statusFuture = runTask(task1, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -698,12 +848,9 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception // Stop the firehose, this will trigger a finishJob. firehose.close(); - // Wait for publish. - while (mdc.getPublished().isEmpty()) { - Thread.sleep(50); - } + Collection publishedSegments = awaitSegments(); - publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); + publishedSegment = Iterables.getOnlyElement(publishedSegments); // Do a query. Assert.assertEquals(1, sumMetric(task1, null, "rows")); @@ -719,8 +866,9 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception // Second run: { + expectPublishedSegments(1); final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, taskStorage, mdc, directory); + final TaskToolbox taskToolbox = makeToolbox(task2, taskStorage, directory); final ListenableFuture statusFuture = runTask(task2, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -734,13 +882,7 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception // Stop the firehose, this will trigger a finishJob. firehose.close(); - // publishedSegment is still published. No reason it shouldn't be. - Assert.assertEquals(ImmutableSet.of(publishedSegment), mdc.getPublished()); - - // Wait for a handoffCallback to show up. - while (handOffCallbacks.isEmpty()) { - Thread.sleep(50); - } + awaitHandoffs(); // Simulate handoff. for (Map.Entry> entry : handOffCallbacks.entrySet()) { @@ -771,8 +913,9 @@ public void testRestoreCorruptData() throws Exception // First run: { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory); + expectPublishedSegments(0); + + final TaskToolbox taskToolbox = makeToolbox(task1, directory); final ListenableFuture statusFuture = runTask(task1, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -796,28 +939,26 @@ public void testRestoreCorruptData() throws Exception Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); // Nothing should be published. - Assert.assertEquals(Sets.newHashSet(), mdc.getPublished()); + Assert.assertTrue(publishedSegments.isEmpty()); } + Optional optional = FileUtils.listFiles(directory, null, true).stream() + .filter(f -> f.getName().equals("00000.smoosh")) + .findFirst(); + + Assert.assertTrue("Could not find smoosh file", optional.isPresent()); + // Corrupt the data: - final File smooshFile = new File( - StringUtils.format( - "%s/persistent/task/%s/work/persist/%s/%s_%s/0/00000.smoosh", - directory, - task1.getId(), - task1.getDataSource(), - Granularities.DAY.bucketStart(now), - Granularities.DAY.bucketEnd(now) - ) - ); + final File smooshFile = optional.get(); Files.write(smooshFile.toPath(), StringUtils.toUtf8("oops!")); // Second run: { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + expectPublishedSegments(0); + final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, mdc, directory); + final TaskToolbox taskToolbox = makeToolbox(task2, directory); final ListenableFuture statusFuture = runTask(task2, taskToolbox); // Wait for the task to finish. @@ -825,7 +966,7 @@ public void testRestoreCorruptData() throws Exception try { statusFuture.get(); } - catch (Exception e) { + catch (Exception expected) { caught = true; } Assert.assertTrue("expected exception", caught); @@ -835,12 +976,13 @@ public void testRestoreCorruptData() throws Exception @Test(timeout = 60_000L) public void testStopBeforeStarting() throws Exception { + expectPublishedSegments(0); + final File directory = tempFolder.newFolder(); final RealtimeIndexTask task1 = makeRealtimeTask(null); task1.stopGracefully(); - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory); + final TaskToolbox taskToolbox = makeToolbox(task1, directory); final ListenableFuture statusFuture = runTask(task1, taskToolbox); // Wait for the task to finish. @@ -916,6 +1058,7 @@ private RealtimeIndexTask makeRealtimeTask( null ); RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( + 1000, 1000, new Period("P1Y"), new Period("PT10M"), @@ -950,25 +1093,86 @@ protected boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory) private TaskToolbox makeToolbox( final Task task, - final IndexerMetadataStorageCoordinator mdc, final File directory ) { return makeToolbox( task, new HeapMemoryTaskStorage(new TaskStorageConfig(null)), - mdc, directory ); } + private void expectPublishedSegments(int count) + { + segmentLatch = new CountDownLatch(count); + handoffLatch = new CountDownLatch(count); + } + + private Collection awaitSegments() throws InterruptedException + { + Assert.assertTrue( + "Timed out waiting for segments to be published", + segmentLatch.await(1, TimeUnit.MINUTES) + ); + + return publishedSegments; + } + + private void awaitHandoffs() throws InterruptedException + { + Assert.assertTrue( + "Timed out waiting for segments to be handed off", + handoffLatch.await(1, TimeUnit.MINUTES) + ); + } + private TaskToolbox makeToolbox( final Task task, final TaskStorage taskStorage, - final IndexerMetadataStorageCoordinator mdc, final File directory ) { + publishedSegments = new CopyOnWriteArrayList<>(); + + Assert.assertFalse( + "Segment latch not initialized, did you forget to call expectPublishSegments?", + segmentLatch == null + ); + + ObjectMapper mapper = new DefaultObjectMapper(); + mapper.registerSubtypes(LinearShardSpec.class); + mapper.registerSubtypes(NumberedShardSpec.class); + IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator( + mapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnectorRule.getConnector() + ) + { + @Override + public Set announceHistoricalSegments(Set segments) throws IOException + { + Set result = super.announceHistoricalSegments(segments); + + publishedSegments.addAll(result); + segments.forEach(s -> segmentLatch.countDown()); + + return result; + } + + @Override + public SegmentPublishResult announceHistoricalSegments( + Set segments, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata + ) throws IOException + { + SegmentPublishResult result = super.announceHistoricalSegments(segments, startMetadata, endMetadata); + + publishedSegments.addAll(result.getSegments()); + result.getSegments().forEach(s -> segmentLatch.countDown()); + + return result; + } + }; final TaskConfig taskConfig = new TaskConfig(directory.getPath(), null, null, 50000, null, false, null, null); final TaskLockbox taskLockbox = new TaskLockbox(taskStorage); try { @@ -1033,6 +1237,7 @@ public boolean registerSegmentHandoffCallback( ) { handOffCallbacks.put(descriptor, new Pair<>(exec, handOffRunnable)); + handoffLatch.countDown(); return true; } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index b0484c882239..bcd3631ab10e 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -495,6 +495,7 @@ public Plumber findPlumber( ), new RealtimeTuningConfig( + 1, 1, new Period("PT10M"), null, diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 50912834680b..e5e152c46d56 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -83,6 +83,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Comparators; import io.druid.metadata.DerbyMetadataStorageActionHandlerFactory; +import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; import io.druid.metadata.TestDerbyConnector; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.SegmentDescriptor; @@ -142,6 +143,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; @@ -226,6 +228,7 @@ public int compare(DataSegment dataSegment, DataSegment dataSegment2) private int announcedSinks; private SegmentHandoffNotifierFactory handoffNotifierFactory; private Map> handOffCallbacks; + private CountDownLatch handoffLatch; private static CountDownLatch publishCountDown; @@ -359,6 +362,7 @@ public void setUp() throws Exception EmittingLogger.registerEmitter(emitter); mapper = TEST_UTILS.getTestObjectMapper(); handOffCallbacks = new ConcurrentHashMap<>(); + handoffLatch = new CountDownLatch(1); // Set up things, the order does matter as if it is messed up then the setUp // should fail because of the Precondition checks in the respective setUp methods @@ -377,6 +381,16 @@ public void setUp() throws Exception taskRunner = setUpThreadPoolTaskRunner(tb); taskQueue = setUpTaskQueue(taskStorage, taskRunner); + + TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); + mapper.registerSubtypes( + new NamedType(MockExceptionalFirehoseFactory.class, "mockExcepFirehoseFactory"), + new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory") + ); + derbyConnector.createDataSourceTable(); + derbyConnector.createTaskTables(); + derbyConnector.createSegmentTable(); + derbyConnector.createPendingSegmentsTable(); } private TaskStorage setUpTaskStorage() @@ -398,12 +412,6 @@ private TaskStorage setUpTaskStorage() case METADATA_TASK_STORAGE: { TestDerbyConnector testDerbyConnector = derbyConnectorRule.getConnector(); - mapper.registerSubtypes( - new NamedType(MockExceptionalFirehoseFactory.class, "mockExcepFirehoseFactory"), - new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory") - ); - testDerbyConnector.createTaskTables(); - testDerbyConnector.createSegmentTable(); taskStorage = new MetadataTaskStorage( testDerbyConnector, new TaskStorageConfig(null), @@ -443,6 +451,7 @@ public boolean registerSegmentHandoffCallback( ) { handOffCallbacks.put(descriptor, new Pair<>(exec, handOffRunnable)); + handoffLatch.countDown(); return true; } @@ -518,7 +527,7 @@ public Set announceHistoricalSegments(Set segments) private TaskToolboxFactory setUpTaskToolboxFactory( DataSegmentPusher dataSegmentPusher, SegmentHandoffNotifierFactory handoffNotifierFactory, - TestIndexerMetadataStorageCoordinator mdc + IndexerMetadataStorageCoordinator mdc ) throws IOException { Preconditions.checkNotNull(queryRunnerFactoryConglomerate); @@ -974,6 +983,45 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception public void testRealtimeIndexTask() throws Exception { publishCountDown = new CountDownLatch(1); + List publishedSegments = new CopyOnWriteArrayList<>(); + + IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator( + mapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnectorRule.getConnector() + ) + { + @Override + public Set announceHistoricalSegments(Set segments) throws IOException + { + Set result = super.announceHistoricalSegments(segments); + + publishedSegments.addAll(result); + publishCountDown.countDown(); + + return result; + } + + @Override + public SegmentPublishResult announceHistoricalSegments( + Set segments, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata + ) throws IOException + { + SegmentPublishResult result = super.announceHistoricalSegments(segments, startMetadata, endMetadata); + + publishedSegments.addAll(result.getSegments()); + publishCountDown.countDown(); + + return result; + } + }; + + tb = setUpTaskToolboxFactory(dataSegmentPusher, handoffNotifierFactory, mdc); + + taskRunner = setUpThreadPoolTaskRunner(tb); + + taskQueue = setUpTaskQueue(taskStorage, taskRunner); + monitorScheduler.addMonitor(EasyMock.anyObject(Monitor.class)); EasyMock.expectLastCall().atLeastOnce(); monitorScheduler.removeMonitor(EasyMock.anyObject(Monitor.class)); @@ -987,6 +1035,7 @@ public void testRealtimeIndexTask() throws Exception taskQueue.add(realtimeIndexTask); //wait for task to process events and publish segment publishCountDown.await(); + handoffLatch.await(); // Realtime Task has published the segment, simulate loading of segment to a historical node so that task finishes with SUCCESS status Assert.assertEquals(1, handOffCallbacks.size()); @@ -1003,8 +1052,8 @@ public void testRealtimeIndexTask() throws Exception Assert.assertEquals(1, announcedSinks); Assert.assertEquals(1, pushedSegments); - Assert.assertEquals(1, mdc.getPublished().size()); - DataSegment segment = mdc.getPublished().iterator().next(); + Assert.assertEquals(1, publishedSegments.size()); + DataSegment segment = publishedSegments.iterator().next(); Assert.assertEquals("test_ds", segment.getDataSource()); Assert.assertEquals(ImmutableList.of("dim1", "dim2"), segment.getDimensions()); Assert.assertEquals( @@ -1202,6 +1251,7 @@ private RealtimeIndexTask newRealtimeIndexTask() null ); RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( + 1000, 1000, new Period("P1Y"), null, //default window period of 10 minutes diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 32c2f5ec962e..984468a6d7b4 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -42,6 +42,7 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig { private static final int defaultMaxRowsInMemory = 75000; + private static final int defaultMaxRowsPerSegment = 5_000_000; private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final Period defaultWindowPeriod = new Period("PT10M"); private static final VersioningPolicy defaultVersioningPolicy = new IntervalStartVersioningPolicy(); @@ -63,6 +64,7 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File { return new RealtimeTuningConfig( defaultMaxRowsInMemory, + defaultMaxRowsPerSegment, defaultIntermediatePersistPeriod, defaultWindowPeriod, basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory, @@ -82,6 +84,7 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File } private final int maxRowsInMemory; + private final int maxRowsPerSegment; private final Period intermediatePersistPeriod; private final Period windowPeriod; private final File basePersistDirectory; @@ -101,6 +104,7 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File @JsonCreator public RealtimeTuningConfig( @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -120,6 +124,7 @@ public RealtimeTuningConfig( ) { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; + this.maxRowsPerSegment = maxRowsPerSegment == null ? defaultMaxRowsPerSegment : maxRowsPerSegment; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaultIntermediatePersistPeriod : intermediatePersistPeriod; @@ -154,6 +159,12 @@ public int getMaxRowsInMemory() return maxRowsInMemory; } + @JsonProperty + public int getMaxRowsPerSegment() + { + return maxRowsPerSegment; + } + @Override @JsonProperty public Period getIntermediatePersistPeriod() @@ -259,6 +270,7 @@ public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) { return new RealtimeTuningConfig( maxRowsInMemory, + maxRowsPerSegment, intermediatePersistPeriod, windowPeriod, basePersistDirectory, @@ -281,6 +293,7 @@ public RealtimeTuningConfig withBasePersistDirectory(File dir) { return new RealtimeTuningConfig( maxRowsInMemory, + maxRowsPerSegment, intermediatePersistPeriod, windowPeriod, dir, diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 59f145555e85..6309fb246aaf 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -199,6 +199,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException } ); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( + 1, 1, new Period("P1Y"), null, @@ -258,6 +259,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException ); tuningConfig_0 = new RealtimeTuningConfig( + 1, 1, new Period("P1Y"), null, @@ -277,6 +279,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException ); tuningConfig_1 = new RealtimeTuningConfig( + 1, 1, new Period("P1Y"), null, diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java index d7b0aa4a6477..6968dd69aa06 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java @@ -67,6 +67,7 @@ EasyMock. anyObject(), EasyMock. anyObject())).andReturn(true).anyTimes(); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( + 1, 1, null, null, diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index 186434c01167..9da3e2f9e03d 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -136,6 +136,7 @@ public AppenderatorTester( maxRowsInMemory, null, null, + null, basePersistDirectory, null, null, diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index 08157eaf7f61..b837bf5ee453 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -136,6 +136,7 @@ public int columnCacheSizeBytes() 75000, null, null, + null, temporaryFolder.newFolder(), null, null, diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index cfddca4aa90e..47aec71fa41c 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -196,6 +196,7 @@ public void setUp() throws Exception EasyMock.replay(announcer, segmentPublisher, dataSegmentPusher, handoffNotifierFactory, handoffNotifier, emitter); tuningConfig = new RealtimeTuningConfig( + 1, 1, null, null, diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index dce3e7a89780..87af620e01b6 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -60,6 +60,7 @@ public void testSwap() throws Exception final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( + 100, 100, new Period("P1Y"), null, diff --git a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java index 49518d6483dd..64d14e7a7649 100644 --- a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java @@ -168,6 +168,7 @@ public Plumber findPlumber( ), new RealtimeTuningConfig( + 1, 1, new Period("PT10M"), null, From 5d9487b4e04dce1b953cc37b797ab7fa9c7cd0b2 Mon Sep 17 00:00:00 2001 From: Kevin Conaway Date: Tue, 16 Jan 2018 09:56:39 -0500 Subject: [PATCH 02/10] Remove unused throwableDuringPublishing --- .../io/druid/indexing/common/task/RealtimeIndexTask.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index f90ee3657061..2041c77e1d2b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -90,7 +90,6 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; public class RealtimeIndexTask extends AbstractTask { @@ -135,9 +134,6 @@ private static String makeDatasource(FireDepartment fireDepartment) @JsonIgnore private final Queue> pendingHandoffs; - @JsonIgnore - private final AtomicReference throwableDuringPublishing; - @JsonIgnore private volatile AppenderatorDriver driver = null; @@ -176,7 +172,6 @@ public RealtimeIndexTask( ); this.spec = fireDepartment; this.pendingHandoffs = new ConcurrentLinkedQueue<>(); - this.throwableDuringPublishing = new AtomicReference<>(); } @Override @@ -377,10 +372,6 @@ public void unannounceSegments(Iterable segments) throws IOExceptio // Time to read data! while (firehose != null && (!gracefullyStopped || firehoseDrainableByClosing) && firehose.hasMore()) { - if (throwableDuringPublishing.get() != null) { - Throwables.propagate(throwableDuringPublishing.get()); - } - try { InputRow inputRow = firehose.nextRow(); From ce0b22cfc62b764b35cf17c0ec6f36347d822cc9 Mon Sep 17 00:00:00 2001 From: Kevin Conaway Date: Tue, 16 Jan 2018 16:42:13 -0500 Subject: [PATCH 03/10] Fix usage of forbidden API --- .../java/io/druid/indexing/common/task/RealtimeIndexTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 2041c77e1d2b..5655a44909a1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -670,6 +670,6 @@ private void publishSegments( private String makeSequenceName(int sequenceNumber) { - return String.format("%s_%d", getId(), sequenceNumber); + return getId() + "_" + sequenceNumber; } } From f43d922193159310b89445dd68f16554aa6f872c Mon Sep 17 00:00:00 2001 From: Kevin Conaway Date: Tue, 23 Jan 2018 11:24:52 -0500 Subject: [PATCH 04/10] Update realtime index IT to account for not skipping older data any more --- .../java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java | 1 + .../resources/indexer/wikipedia_realtime_index_queries.json | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java index 530250c14a6e..f447cb7139cd 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java @@ -218,6 +218,7 @@ public void postEvents() throws Exception while ((line = reader.readLine()) != null) { if (i == 15) { // for the 15th line, use a time before the window dt = dt.minusMinutes(10); + dtFirst = dt; } else if (i == 16) { // remember this time to use in the expected response from the groupBy query dtGroupBy = dt; } else if (i == 18) { // use a time 6 seconds ago so it will be out of order diff --git a/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_queries.json b/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_queries.json index b579b347c348..acd88ca893e0 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_queries.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_queries.json @@ -31,8 +31,8 @@ { "timestamp" : "%%TIMESERIES_RESPONSE_TIMESTAMP%%", "result" : { - "chars_added" : 1595.0, - "edit_count" : 21 + "chars_added" : 1642.0, + "edit_count" : 22 } } ] From 55af8be24285dfe2e65398391617c2cbd84962a0 Mon Sep 17 00:00:00 2001 From: Kevin Conaway Date: Tue, 23 Jan 2018 11:28:09 -0500 Subject: [PATCH 05/10] Separate out waiting on publish futures and handoff futures to avoid a race condition where the handoff timeout expires before the segment is published --- .../common/task/RealtimeIndexTask.java | 42 ++++++++++++++++++- 1 file changed, 40 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 5655a44909a1..94f89e4a2483 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -28,8 +28,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; @@ -79,6 +81,7 @@ import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.Collections; @@ -131,6 +134,9 @@ private static String makeDatasource(FireDepartment fireDepartment) @JsonIgnore private final FireDepartment spec; + @JsonIgnore + private final Queue> pendingPublishes; + @JsonIgnore private final Queue> pendingHandoffs; @@ -171,6 +177,7 @@ public RealtimeIndexTask( context ); this.spec = fireDepartment; + this.pendingPublishes = new ConcurrentLinkedQueue<>(); this.pendingHandoffs = new ConcurrentLinkedQueue<>(); } @@ -427,6 +434,13 @@ public void unannounceSegments(Iterable segments) throws IOExceptio // Publish any remaining segments publishSegments(publisher, committerSupplier, sequenceName); + if (!pendingPublishes.isEmpty()) { + ListenableFuture allPublishes = Futures.allAsList(pendingPublishes); + log.info("Waiting for segments to publish"); + + allPublishes.get(); + } + if (!pendingHandoffs.isEmpty()) { ListenableFuture allHandoffs = Futures.allAsList(pendingHandoffs); log.info("Waiting for handoffs"); @@ -663,9 +677,33 @@ private void publishSegments( Collections.singletonList(sequenceName) ); - ListenableFuture handoffFuture = Futures.transform(publishFuture, driver::registerHandoff); + // Use a separate future to ensure that the publish future is not completed until after + // the handoff future is registered in the pending list + SettableFuture publishResultFuture = SettableFuture.create(); - pendingHandoffs.add(handoffFuture); + pendingPublishes.add(publishResultFuture); + + Futures.addCallback(publishFuture, new FutureCallback() + { + @Override + public void onSuccess(@Nullable SegmentsAndMetadata published) + { + ListenableFuture handoffFuture = driver.registerHandoff(published); + + log.info("Registering pending handoff for [%s]", published); + + pendingHandoffs.add(handoffFuture); + + publishResultFuture.set(published); + } + + @Override + public void onFailure(@Nullable Throwable throwable) + { + log.error(throwable, "Error occurred publishing segments"); + publishResultFuture.setException(throwable); + } + }); } private String makeSequenceName(int sequenceNumber) From 9ae8c21aa688b01ae96ea657e6d8cd6fc75f9644 Mon Sep 17 00:00:00 2001 From: Kevin Conaway Date: Fri, 26 Jan 2018 16:31:59 -0500 Subject: [PATCH 06/10] #5261 Add separate AppenderatorDriverRealtimeIndexTask and revert changes to RealtimeIndexTask --- .../AppenderatorDriverRealtimeIndexTask.java | 659 +++++++++ .../common/task/RealtimeIndexTask.java | 361 ++--- .../io/druid/indexing/common/task/Task.java | 1 + ...penderatorDriverRealtimeIndexTaskTest.java | 1312 +++++++++++++++++ .../common/task/RealtimeIndexTaskTest.java | 416 ++---- .../indexing/overlord/TaskLifecycleTest.java | 67 +- .../indexer/ITRealtimeIndexTaskTest.java | 1 - .../wikipedia_realtime_index_queries.json | 4 +- 8 files changed, 2211 insertions(+), 610 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java new file mode 100644 index 000000000000..a146b5c3b321 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -0,0 +1,659 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import com.metamx.emitter.EmittingLogger; +import io.druid.data.input.Committer; +import io.druid.data.input.Firehose; +import io.druid.data.input.FirehoseFactory; +import io.druid.data.input.InputRow; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeDiscoveryProvider; +import io.druid.discovery.LookupNodeService; +import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; +import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; +import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.LockAcquireAction; +import io.druid.indexing.common.actions.LockReleaseAction; +import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.guava.CloseQuietly; +import io.druid.java.util.common.parsers.ParseException; +import io.druid.query.DruidMetrics; +import io.druid.query.NoopQueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeIOConfig; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.realtime.FireDepartment; +import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.RealtimeMetricsMonitor; +import io.druid.segment.realtime.appenderator.Appenderator; +import io.druid.segment.realtime.appenderator.AppenderatorDriver; +import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; +import io.druid.segment.realtime.appenderator.Appenderators; +import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import io.druid.segment.realtime.firehose.ClippedFirehoseFactory; +import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; +import io.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; +import io.druid.segment.realtime.plumber.Committers; +import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.timeline.DataSegment; +import org.apache.commons.io.FileUtils; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.Queue; +import java.util.Random; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +public class AppenderatorDriverRealtimeIndexTask extends AbstractTask +{ + public static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; + + private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class); + private static final Random random = new Random(); + + private static String makeDatasource(FireDepartment fireDepartment) + { + return fireDepartment.getDataSchema().getDataSource(); + } + + @JsonIgnore + private final FireDepartment spec; + + @JsonIgnore + private final Queue> pendingPublishes; + + @JsonIgnore + private final Queue> pendingHandoffs; + + @JsonIgnore + private volatile Appenderator appenderator = null; + + @JsonIgnore + private volatile Firehose firehose = null; + + @JsonIgnore + private volatile FireDepartmentMetrics metrics = null; + + @JsonIgnore + private volatile boolean gracefullyStopped = false; + + @JsonIgnore + private volatile boolean finishingJob = false; + + @JsonIgnore + private volatile Thread runThread = null; + + @JsonCreator + public AppenderatorDriverRealtimeIndexTask( + @JsonProperty("id") String id, + @JsonProperty("resource") TaskResource taskResource, + @JsonProperty("spec") FireDepartment fireDepartment, + @JsonProperty("context") Map context + ) + { + super( + id == null ? RealtimeIndexTask.makeTaskId(fireDepartment) : id, + StringUtils.format("index_realtime_appenderator_%s", makeDatasource(fireDepartment)), + taskResource, + makeDatasource(fireDepartment), + context + ); + this.spec = fireDepartment; + this.pendingPublishes = new ConcurrentLinkedQueue<>(); + this.pendingHandoffs = new ConcurrentLinkedQueue<>(); + } + + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY); + } + + @Override + public String getType() + { + return "index_realtime_appenderator"; + } + + @Override + public String getNodeType() + { + return "realtime"; + } + + @Override + public QueryRunner getQueryRunner(Query query) + { + if (appenderator == null) { + // Not yet initialized, no data yet, just return a noop runner. + return new NoopQueryRunner<>(); + } + + return (queryPlus, responseContext) -> queryPlus.run(appenderator, responseContext); + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return true; + } + + @Override + public TaskStatus run(final TaskToolbox toolbox) throws Exception + { + runThread = Thread.currentThread(); + + setupTimeoutAlert(); + + boolean normalExit = true; + + // Wrap default DataSegmentAnnouncer such that we unlock intervals as we unannounce segments + final long lockTimeoutMs = getContextValue(Tasks.LOCK_TIMEOUT_KEY, Tasks.DEFAULT_LOCK_TIMEOUT); + // Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, http timeout error can occur while waiting for a + // lock to be acquired. + final DataSegmentAnnouncer lockingSegmentAnnouncer = new DataSegmentAnnouncer() + { + @Override + public void announceSegment(final DataSegment segment) throws IOException + { + // Side effect: Calling announceSegment causes a lock to be acquired + Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) + ), + "Cannot acquire a lock for interval[%s]", + 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 segments) throws IOException + { + // Side effect: Calling announceSegments causes locks to be acquired + for (DataSegment segment : segments) { + Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) + ), + "Cannot acquire a lock for interval[%s]", + segment.getInterval() + ); + } + toolbox.getSegmentAnnouncer().announceSegments(segments); + } + + @Override + public void unannounceSegments(Iterable segments) throws IOException + { + try { + toolbox.getSegmentAnnouncer().unannounceSegments(segments); + } + finally { + for (DataSegment segment : segments) { + toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval())); + } + } + } + }; + + DataSchema dataSchema = spec.getDataSchema(); + RealtimeIOConfig realtimeIOConfig = spec.getIOConfig(); + RealtimeTuningConfig tuningConfig = spec.getTuningConfig() + .withBasePersistDirectory(toolbox.getPersistDir()); + + final FireDepartment fireDepartment = new FireDepartment( + dataSchema, + realtimeIOConfig, + tuningConfig + ); + this.metrics = fireDepartment.getMetrics(); + final RealtimeMetricsMonitor metricsMonitor = new RealtimeMetricsMonitor( + ImmutableList.of(fireDepartment), + ImmutableMap.of( + DruidMetrics.TASK_ID, new String[]{getId()} + ) + ); + + Supplier committerSupplier = null; + final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); + + LookupNodeService lookupNodeService = getContextValue(CTX_KEY_LOOKUP_TIER) == null ? + toolbox.getLookupNodeService() : + new LookupNodeService((String) getContextValue(CTX_KEY_LOOKUP_TIER)); + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( + toolbox.getDruidNode(), + DruidNodeDiscoveryProvider.NODE_TYPE_PEON, + ImmutableMap.of( + toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), + lookupNodeService.getName(), lookupNodeService + ) + ); + + appenderator = newAppenderator(dataSchema, tuningConfig, fireDepartment.getMetrics(), toolbox, lockingSegmentAnnouncer); + AppenderatorDriver driver = newDriver(dataSchema, appenderator, toolbox, fireDepartment.getMetrics()); + + try { + toolbox.getDataSegmentServerAnnouncer().announce(); + toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); + + driver.startJob(); + + // Set up metrics emission + toolbox.getMonitorScheduler().addMonitor(metricsMonitor); + + // Firehose temporary directory is automatically removed when this RealtimeIndexTask completes. + FileUtils.forceMkdir(firehoseTempDir); + + // Delay firehose connection to avoid claiming input resources while the plumber is starting up. + final FirehoseFactory firehoseFactory = spec.getIOConfig().getFirehoseFactory(); + final boolean firehoseDrainableByClosing = isFirehoseDrainableByClosing(firehoseFactory); + + // Skip connecting firehose if we've been stopped before we got started. + synchronized (this) { + if (!gracefullyStopped) { + firehose = firehoseFactory.connect(spec.getDataSchema().getParser(), firehoseTempDir); + committerSupplier = Committers.supplierFromFirehose(firehose); + } + } + + int sequenceNumber = 0; + String sequenceName = makeSequenceName(getId(), sequenceNumber); + + final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { + final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments); + return toolbox.getTaskActionClient().submit(action).isSuccess(); + }; + + // Time to read data! + while (firehose != null && (!gracefullyStopped || firehoseDrainableByClosing) && firehose.hasMore()) { + try { + InputRow inputRow = firehose.nextRow(); + + if (inputRow == null) { + log.debug("Discarded null row, considering thrownAway."); + metrics.incrementThrownAway(); + } else { + AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName, committerSupplier); + + if (addResult.isOk()) { + if (addResult.isPersistRequired()) { + driver.persist(committerSupplier.get()); + } + + if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) { + publishSegments(driver, publisher, committerSupplier, sequenceName); + + sequenceNumber++; + sequenceName = makeSequenceName(getId(), sequenceNumber); + } + } else { + // Failure to allocate segment puts determinism at risk, bail out to be safe. + // May want configurable behavior here at some point. + // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. + throw new ISE("Could not allocate segment for row with timestamp[%s]", inputRow.getTimestamp()); + } + + metrics.incrementProcessed(); + } + } + catch (ParseException e) { + if (tuningConfig.isReportParseExceptions()) { + throw e; + } else { + log.debug(e, "Discarded row due to exception, considering unparseable."); + metrics.incrementUnparseable(); + } + } + } + + if (!gracefullyStopped) { + synchronized (this) { + if (gracefullyStopped) { + // Someone called stopGracefully after we checked the flag. That's okay, just stop now. + log.info("Gracefully stopping."); + } else { + finishingJob = true; + } + } + + if (finishingJob) { + log.info("Finishing job..."); + // Publish any remaining segments + publishSegments(driver, publisher, committerSupplier, sequenceName); + + if (!pendingPublishes.isEmpty()) { + ListenableFuture allPublishes = Futures.allAsList(pendingPublishes); + log.info("Waiting for segments to publish"); + + allPublishes.get(); + } + + if (!pendingHandoffs.isEmpty()) { + ListenableFuture allHandoffs = Futures.allAsList(pendingHandoffs); + log.info("Waiting for handoffs"); + + long handoffTimeout = tuningConfig.getHandoffConditionTimeout(); + + if (handoffTimeout > 0) { + allHandoffs.get(handoffTimeout, TimeUnit.MILLISECONDS); + } else { + allHandoffs.get(); + } + } + } + } + + } + catch (Throwable e) { + normalExit = false; + log.makeAlert(e, "Exception aborted realtime processing[%s]", dataSchema.getDataSource()) + .emit(); + throw e; + } + finally { + if (normalExit && gracefullyStopped) { + try { + // Persist if we had actually started. + if (firehose != null) { + log.info("Persisting remaining data."); + + final Committer committer = committerSupplier.get(); + final CountDownLatch persistLatch = new CountDownLatch(1); + driver.persist( + new Committer() + { + @Override + public Object getMetadata() + { + return committer.getMetadata(); + } + + @Override + public void run() + { + try { + committer.run(); + } + finally { + persistLatch.countDown(); + } + } + } + ); + persistLatch.await(); + } + } + catch (InterruptedException e) { + log.debug(e, "Interrupted while finishing the job"); + } + catch (Exception e) { + log.makeAlert(e, "Failed to finish realtime task").emit(); + throw e; + } + finally { + if (firehose != null) { + CloseQuietly.close(firehose); + } + toolbox.getMonitorScheduler().removeMonitor(metricsMonitor); + } + } + + if (appenderator != null) { + appenderator.close(); + } + + if (driver != null) { + driver.close(); + } + + toolbox.getDataSegmentServerAnnouncer().unannounce(); + toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); + } + + log.info("Job done!"); + return TaskStatus.success(getId()); + } + + @Override + public boolean canRestore() + { + return true; + } + + @Override + public void stopGracefully() + { + try { + synchronized (this) { + if (!gracefullyStopped) { + gracefullyStopped = true; + if (firehose == null) { + log.info("stopGracefully: Firehose not started yet, so nothing to stop."); + } else if (finishingJob) { + log.info("stopGracefully: Interrupting finishJob."); + runThread.interrupt(); + } else if (isFirehoseDrainableByClosing(spec.getIOConfig().getFirehoseFactory())) { + log.info("stopGracefully: Draining firehose."); + firehose.close(); + } else { + log.info("stopGracefully: Cannot drain firehose by closing, interrupting run thread."); + runThread.interrupt(); + } + } + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * Public for tests. + */ + @JsonIgnore + public Firehose getFirehose() + { + return firehose; + } + + /** + * Public for tests. + */ + @JsonIgnore + public FireDepartmentMetrics getMetrics() + { + return metrics; + } + + @JsonProperty("spec") + public FireDepartment getRealtimeIngestionSchema() + { + return spec; + } + + /** + * Is a firehose from this factory drainable by closing it? If so, we should drain on stopGracefully rather than + * abruptly stopping. + * + * This is a hack to get around the fact that the Firehose and FirehoseFactory interfaces do not help us do this. + * + * Protected for tests. + */ + protected boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory) + { + return firehoseFactory instanceof EventReceiverFirehoseFactory + || (firehoseFactory instanceof TimedShutoffFirehoseFactory + && isFirehoseDrainableByClosing(((TimedShutoffFirehoseFactory) firehoseFactory).getDelegateFactory())) + || (firehoseFactory instanceof ClippedFirehoseFactory + && isFirehoseDrainableByClosing(((ClippedFirehoseFactory) firehoseFactory).getDelegate())); + } + + private void setupTimeoutAlert() + { + if (spec.getTuningConfig().getAlertTimeout() > 0) { + Timer timer = new Timer("RealtimeIndexTask-Timer", true); + timer.schedule( + new TimerTask() + { + @Override + public void run() + { + log.makeAlert( + "RealtimeIndexTask for dataSource [%s] hasn't finished in configured time [%d] ms.", + spec.getDataSchema().getDataSource(), + spec.getTuningConfig().getAlertTimeout() + ).emit(); + } + }, + spec.getTuningConfig().getAlertTimeout() + ); + } + } + + private void publishSegments( + AppenderatorDriver driver, + TransactionalSegmentPublisher publisher, + Supplier committerSupplier, + String sequenceName + ) + { + ListenableFuture publishFuture = driver.publish( + publisher, + committerSupplier.get(), + Collections.singletonList(sequenceName) + ); + + // Use a separate future to ensure that the publish future is not completed until after + // the handoff future is registered in the pending list + SettableFuture publishResultFuture = SettableFuture.create(); + + pendingPublishes.add(publishResultFuture); + + Futures.addCallback(publishFuture, new FutureCallback() + { + @Override + public void onSuccess(@Nullable SegmentsAndMetadata published) + { + ListenableFuture handoffFuture = driver.registerHandoff(published); + + log.info("Registering pending handoff for [%s]", published); + + pendingHandoffs.add(handoffFuture); + + publishResultFuture.set(published); + } + + @Override + public void onFailure(@Nullable Throwable throwable) + { + log.error(throwable, "Error occurred publishing segments"); + publishResultFuture.setException(throwable); + } + }); + } + + private static Appenderator newAppenderator( + final DataSchema dataSchema, + final RealtimeTuningConfig tuningConfig, + final FireDepartmentMetrics metrics, + final TaskToolbox toolbox, + final DataSegmentAnnouncer segmentAnnouncer + ) + { + return Appenderators.createRealtime( + dataSchema, + tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + toolbox.getSegmentPusher(), + toolbox.getObjectMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + toolbox.getQueryRunnerFactoryConglomerate(), + segmentAnnouncer, + toolbox.getEmitter(), + toolbox.getQueryExecutorService(), + toolbox.getCache(), + toolbox.getCacheConfig() + ); + } + + private static AppenderatorDriver newDriver( + final DataSchema dataSchema, + final Appenderator appenderator, + final TaskToolbox toolbox, + final FireDepartmentMetrics metrics + ) + { + return new AppenderatorDriver( + appenderator, + new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema), + toolbox.getSegmentHandoffNotifierFactory(), + new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), + toolbox.getObjectMapper(), + metrics + ); + } + + private static String makeSequenceName(String taskId, int sequenceNumber) + { + return taskId + "_" + sequenceNumber; + } +} + diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 94f89e4a2483..db32634c1e3b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -28,71 +28,59 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.primitives.Ints; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.SettableFuture; import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.InputRow; import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.discovery.LookupNodeService; -import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; -import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockAcquireAction; import io.druid.indexing.common.actions.LockReleaseAction; -import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.java.util.common.DateTimes; -import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.java.util.common.parsers.ParseException; import io.druid.query.DruidMetrics; -import io.druid.query.NoopQueryRunner; +import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.query.QueryToolChest; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.FireDepartment; import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.RealtimeMetricsMonitor; -import io.druid.segment.realtime.appenderator.Appenderator; -import io.druid.segment.realtime.appenderator.AppenderatorDriver; -import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; -import io.druid.segment.realtime.appenderator.Appenderators; -import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; -import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import io.druid.segment.realtime.SegmentPublisher; import io.druid.segment.realtime.firehose.ClippedFirehoseFactory; import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; import io.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; import io.druid.segment.realtime.plumber.Committers; +import io.druid.segment.realtime.plumber.Plumber; +import io.druid.segment.realtime.plumber.PlumberSchool; +import io.druid.segment.realtime.plumber.Plumbers; +import io.druid.segment.realtime.plumber.RealtimePlumberSchool; import io.druid.segment.realtime.plumber.VersioningPolicy; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; +import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.util.Collections; import java.util.Map; -import java.util.Queue; import java.util.Random; import java.util.Timer; import java.util.TimerTask; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; public class RealtimeIndexTask extends AbstractTask { @@ -101,7 +89,7 @@ public class RealtimeIndexTask extends AbstractTask private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class); private static final Random random = new Random(); - private static String makeTaskId(FireDepartment fireDepartment) + static String makeTaskId(FireDepartment fireDepartment) { return makeTaskId( fireDepartment.getDataSchema().getDataSource(), @@ -135,16 +123,7 @@ private static String makeDatasource(FireDepartment fireDepartment) private final FireDepartment spec; @JsonIgnore - private final Queue> pendingPublishes; - - @JsonIgnore - private final Queue> pendingHandoffs; - - @JsonIgnore - private volatile AppenderatorDriver driver = null; - - @JsonIgnore - private volatile Appenderator appenderator = null; + private volatile Plumber plumber = null; @JsonIgnore private volatile Firehose firehose = null; @@ -161,6 +140,9 @@ private static String makeDatasource(FireDepartment fireDepartment) @JsonIgnore private volatile Thread runThread = null; + @JsonIgnore + private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null; + @JsonCreator public RealtimeIndexTask( @JsonProperty("id") String id, @@ -177,8 +159,6 @@ public RealtimeIndexTask( context ); this.spec = fireDepartment; - this.pendingPublishes = new ConcurrentLinkedQueue<>(); - this.pendingHandoffs = new ConcurrentLinkedQueue<>(); } @Override @@ -202,12 +182,14 @@ public String getNodeType() @Override public QueryRunner getQueryRunner(Query query) { - if (appenderator == null) { - // Not yet initialized, no data yet, just return a noop runner. - return new NoopQueryRunner<>(); - } + if (plumber != null) { + QueryRunnerFactory> factory = queryRunnerFactoryConglomerate.findFactory(query); + QueryToolChest> toolChest = factory.getToolchest(); - return (queryPlus, responseContext) -> queryPlus.run(appenderator, responseContext); + return new FinalizeResultsQueryRunner(plumber.getQueryRunner(query), toolChest); + } else { + return null; + } } @Override @@ -221,10 +203,23 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception { runThread = Thread.currentThread(); + if (this.plumber != null) { + throw new IllegalStateException("WTF?!? run with non-null plumber??!"); + } + setupTimeoutAlert(); boolean normalExit = true; + // 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 SegmentPublisher segmentPublisher = new TaskActionSegmentPublisher(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 long lockTimeoutMs = getContextValue(Tasks.LOCK_TIMEOUT_KEY, Tasks.DEFAULT_LOCK_TIMEOUT); // Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, http timeout error can occur while waiting for a @@ -286,25 +281,31 @@ public void unannounceSegments(Iterable segments) throws IOExceptio } }; - // NOTE: getVersion will block (and thus block the firehose) if there is lock contention + // 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 = interval -> { - try { - // Side effect: Calling getVersion causes a lock to be acquired - final LockAcquireAction action = new LockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs); - final TaskLock lock = Preconditions.checkNotNull( - toolbox.getTaskActionClient().submit(action), - "Cannot acquire a lock for interval[%s]", - interval - ); + final VersioningPolicy versioningPolicy = new VersioningPolicy() + { + @Override + public String getVersion(final Interval interval) + { + try { + // Side effect: Calling getVersion causes a lock to be acquired + final LockAcquireAction action = new LockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs); + final TaskLock lock = Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit(action), + "Cannot acquire a lock for interval[%s]", + interval + ); - return lock.getVersion(); - } - catch (IOException e) { - throw Throwables.propagate(e); + return lock.getVersion(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } } }; @@ -326,6 +327,29 @@ public void unannounceSegments(Iterable segments) throws IOExceptio DruidMetrics.TASK_ID, new String[]{getId()} ) ); + this.queryRunnerFactoryConglomerate = toolbox.getQueryRunnerFactoryConglomerate(); + + // 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 + // NOTE: (partitionNum_index.zip for HDFS data storage) and descriptor.json (partitionNum_descriptor.json for + // NOTE: HDFS data storage) to mismatch, or it can cause historical nodes to load different instances of + // NOTE: the "same" segment. + final PlumberSchool plumberSchool = new RealtimePlumberSchool( + toolbox.getEmitter(), + toolbox.getQueryRunnerFactoryConglomerate(), + toolbox.getSegmentPusher(), + lockingSegmentAnnouncer, + segmentPublisher, + toolbox.getSegmentHandoffNotifierFactory(), + toolbox.getQueryExecutorService(), + toolbox.getIndexMergerV9(), + toolbox.getIndexIO(), + toolbox.getCache(), + toolbox.getCacheConfig(), + toolbox.getObjectMapper() + ); + + this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, metrics); Supplier committerSupplier = null; final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); @@ -342,14 +366,12 @@ public void unannounceSegments(Iterable segments) throws IOExceptio ) ); - appenderator = newAppenderator(dataSchema, tuningConfig, fireDepartment.getMetrics(), toolbox, lockingSegmentAnnouncer); - driver = newDriver(dataSchema, appenderator, toolbox, fireDepartment.getMetrics()); - try { toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); - driver.startJob(); + + plumber.startJob(); // Set up metrics emission toolbox.getMonitorScheduler().addMonitor(metricsMonitor); @@ -369,93 +391,16 @@ public void unannounceSegments(Iterable segments) throws IOExceptio } } - int sequenceNumber = 0; - String sequenceName = makeSequenceName(sequenceNumber); - - final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { - final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments); - return toolbox.getTaskActionClient().submit(action).isSuccess(); - }; - // Time to read data! while (firehose != null && (!gracefullyStopped || firehoseDrainableByClosing) && firehose.hasMore()) { - try { - InputRow inputRow = firehose.nextRow(); - - if (inputRow == null) { - log.debug("Discarded null row, considering thrownAway."); - metrics.incrementThrownAway(); - } else { - AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName, committerSupplier); - - if (addResult.isOk()) { - if (addResult.isPersistRequired()) { - driver.persist(committerSupplier.get()); - } - - if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) { - publishSegments(publisher, committerSupplier, sequenceName); - - sequenceNumber++; - sequenceName = makeSequenceName(sequenceNumber); - } - } else { - // Failure to allocate segment puts determinism at risk, bail out to be safe. - // May want configurable behavior here at some point. - // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. - throw new ISE("Could not allocate segment for row with timestamp[%s]", inputRow.getTimestamp()); - } - - metrics.incrementProcessed(); - } - } - catch (ParseException e) { - if (tuningConfig.isReportParseExceptions()) { - throw e; - } else { - log.debug(e, "Discarded row due to exception, considering unparseable."); - metrics.incrementUnparseable(); - } - } - } - - if (!gracefullyStopped) { - synchronized (this) { - if (gracefullyStopped) { - // Someone called stopGracefully after we checked the flag. That's okay, just stop now. - log.info("Gracefully stopping."); - } else { - finishingJob = true; - } - } - - if (finishingJob) { - log.info("Finishing job..."); - // Publish any remaining segments - publishSegments(publisher, committerSupplier, sequenceName); - - if (!pendingPublishes.isEmpty()) { - ListenableFuture allPublishes = Futures.allAsList(pendingPublishes); - log.info("Waiting for segments to publish"); - - allPublishes.get(); - } - - if (!pendingHandoffs.isEmpty()) { - ListenableFuture allHandoffs = Futures.allAsList(pendingHandoffs); - log.info("Waiting for handoffs"); - - long handoffTimeout = tuningConfig.getHandoffConditionTimeout(); - - if (handoffTimeout > 0) { - allHandoffs.get(handoffTimeout, TimeUnit.MILLISECONDS); - } else { - allHandoffs.get(); - } - } - } + Plumbers.addNextRow( + committerSupplier, + firehose, + plumber, + tuningConfig.isReportParseExceptions(), + metrics + ); } - } catch (Throwable e) { normalExit = false; @@ -472,7 +417,7 @@ public void unannounceSegments(Iterable segments) throws IOExceptio final Committer committer = committerSupplier.get(); final CountDownLatch persistLatch = new CountDownLatch(1); - driver.persist( + plumber.persist( new Committer() { @Override @@ -495,6 +440,24 @@ public void run() ); persistLatch.await(); } + + if (gracefullyStopped) { + log.info("Gracefully stopping."); + } else { + log.info("Finishing the job."); + synchronized (this) { + if (gracefullyStopped) { + // Someone called stopGracefully after we checked the flag. That's okay, just stop now. + log.info("Gracefully stopping."); + } else { + finishingJob = true; + } + } + + if (finishingJob) { + plumber.finishJob(); + } + } } catch (InterruptedException e) { log.debug(e, "Interrupted while finishing the job"); @@ -511,14 +474,6 @@ public void run() } } - if (appenderator != null) { - appenderator.close(); - } - - if (driver != null) { - driver.close(); - } - toolbox.getDataSegmentServerAnnouncer().unannounce(); toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); } @@ -601,6 +556,22 @@ && isFirehoseDrainableByClosing(((TimedShutoffFirehoseFactory) firehoseFactory). && isFirehoseDrainableByClosing(((ClippedFirehoseFactory) firehoseFactory).getDelegate())); } + public static class TaskActionSegmentPublisher implements SegmentPublisher + { + final TaskToolbox taskToolbox; + + public TaskActionSegmentPublisher(TaskToolbox taskToolbox) + { + this.taskToolbox = taskToolbox; + } + + @Override + public void publishSegment(DataSegment segment) throws IOException + { + taskToolbox.publishSegments(ImmutableList.of(segment)); + } + } + private void setupTimeoutAlert() { if (spec.getTuningConfig().getAlertTimeout() > 0) { @@ -622,92 +593,4 @@ public void run() ); } } - - private Appenderator newAppenderator( - final DataSchema dataSchema, - final RealtimeTuningConfig tuningConfig, - final FireDepartmentMetrics metrics, - final TaskToolbox toolbox, - final DataSegmentAnnouncer segmentAnnouncer - ) - { - return Appenderators.createRealtime( - dataSchema, - tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), - metrics, - toolbox.getSegmentPusher(), - toolbox.getObjectMapper(), - toolbox.getIndexIO(), - toolbox.getIndexMergerV9(), - toolbox.getQueryRunnerFactoryConglomerate(), - segmentAnnouncer, - toolbox.getEmitter(), - toolbox.getQueryExecutorService(), - toolbox.getCache(), - toolbox.getCacheConfig() - ); - } - - private AppenderatorDriver newDriver( - final DataSchema dataSchema, - final Appenderator appenderator, - final TaskToolbox toolbox, - final FireDepartmentMetrics metrics - ) - { - return new AppenderatorDriver( - appenderator, - new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema), - toolbox.getSegmentHandoffNotifierFactory(), - new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), - toolbox.getObjectMapper(), - metrics - ); - } - - private void publishSegments( - TransactionalSegmentPublisher publisher, - Supplier committerSupplier, - String sequenceName - ) - { - ListenableFuture publishFuture = driver.publish( - publisher, - committerSupplier.get(), - Collections.singletonList(sequenceName) - ); - - // Use a separate future to ensure that the publish future is not completed until after - // the handoff future is registered in the pending list - SettableFuture publishResultFuture = SettableFuture.create(); - - pendingPublishes.add(publishResultFuture); - - Futures.addCallback(publishFuture, new FutureCallback() - { - @Override - public void onSuccess(@Nullable SegmentsAndMetadata published) - { - ListenableFuture handoffFuture = driver.registerHandoff(published); - - log.info("Registering pending handoff for [%s]", published); - - pendingHandoffs.add(handoffFuture); - - publishResultFuture.set(published); - } - - @Override - public void onFailure(@Nullable Throwable throwable) - { - log.error(throwable, "Error occurred publishing segments"); - publishResultFuture.setException(throwable); - } - }); - } - - private String makeSequenceName(int sequenceNumber) - { - return getId() + "_" + sequenceNumber; - } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index 8e18d725e436..e3eb3e77c5e5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -55,6 +55,7 @@ @JsonSubTypes.Type(name = "hadoop_convert_segment", value = HadoopConverterTask.class), @JsonSubTypes.Type(name = "hadoop_convert_segment_sub", value = HadoopConverterTask.ConverterSubTask.class), @JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class), + @JsonSubTypes.Type(name = "index_realtime_appenderator", value = AppenderatorDriverRealtimeIndexTask.class), @JsonSubTypes.Type(name = "noop", value = NoopTask.class), @JsonSubTypes.Type(name = "version_converter", value = ConvertSegmentBackwardsCompatibleTask.class), // Backwards compat - Deprecated @JsonSubTypes.Type(name = "version_converter_sub", value = ConvertSegmentBackwardsCompatibleTask.SubTask.class), // backwards compat - Deprecated diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java new file mode 100644 index 000000000000..deb233aa137a --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -0,0 +1,1312 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +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; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.core.NoopEmitter; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.metrics.MonitorScheduler; +import io.druid.client.cache.CacheConfig; +import io.druid.client.cache.MapCache; +import io.druid.data.input.Firehose; +import io.druid.data.input.FirehoseFactory; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.TimeAndDimsParseSpec; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.discovery.DataNodeService; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.LookupNodeService; +import io.druid.indexer.TaskState; +import io.druid.indexing.common.SegmentLoaderFactory; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.TaskToolboxFactory; +import io.druid.indexing.common.TestUtils; +import io.druid.indexing.common.actions.LocalTaskActionClientFactory; +import io.druid.indexing.common.actions.TaskActionClientFactory; +import io.druid.indexing.common.actions.TaskActionToolbox; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.common.config.TaskStorageConfig; +import io.druid.indexing.overlord.DataSourceMetadata; +import io.druid.indexing.overlord.HeapMemoryTaskStorage; +import io.druid.indexing.overlord.SegmentPublishResult; +import io.druid.indexing.overlord.TaskLockbox; +import io.druid.indexing.overlord.TaskStorage; +import io.druid.indexing.overlord.supervisor.SupervisorManager; +import io.druid.indexing.test.TestDataSegmentAnnouncer; +import io.druid.indexing.test.TestDataSegmentKiller; +import io.druid.indexing.test.TestDataSegmentPusher; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Pair; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.java.util.common.jackson.JacksonUtils; +import io.druid.java.util.common.logger.Logger; +import io.druid.java.util.common.parsers.ParseException; +import io.druid.math.expr.ExprMacroTable; +import io.druid.metadata.EntryExistsException; +import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import io.druid.metadata.TestDerbyConnector; +import io.druid.query.DefaultQueryRunnerFactoryConglomerate; +import io.druid.query.Druids; +import io.druid.query.IntervalChunkingQueryRunnerDecorator; +import io.druid.query.Query; +import io.druid.query.QueryPlus; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.query.QueryToolChest; +import io.druid.query.QueryWatcher; +import io.druid.query.Result; +import io.druid.query.SegmentDescriptor; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.filter.DimFilter; +import io.druid.query.filter.SelectorDimFilter; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryEngine; +import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.TestHelper; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeIOConfig; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.loading.SegmentLoaderConfig; +import io.druid.segment.loading.SegmentLoaderLocalCacheManager; +import io.druid.segment.loading.StorageLocationConfig; +import io.druid.segment.realtime.FireDepartment; +import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; +import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import io.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory; +import io.druid.segment.transform.ExpressionTransform; +import io.druid.segment.transform.TransformSpec; +import io.druid.server.DruidNode; +import io.druid.server.coordination.DataSegmentServerAnnouncer; +import io.druid.server.coordination.ServerType; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.LinearShardSpec; +import io.druid.timeline.partition.NumberedShardSpec; +import org.apache.commons.io.FileUtils; +import org.easymock.EasyMock; +import org.hamcrest.CoreMatchers; +import org.joda.time.DateTime; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableCauseMatcher; +import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; + +public class AppenderatorDriverRealtimeIndexTaskTest +{ + private static final Logger log = new Logger(AppenderatorDriverRealtimeIndexTaskTest.class); + private static final ServiceEmitter emitter = new ServiceEmitter( + "service", + "host", + new NoopEmitter() + ); + + private static final String FAIL_DIM = "__fail__"; + + private static class TestFirehose implements Firehose + { + private final InputRowParser> parser; + private final List> queue = new LinkedList<>(); + private boolean closed = false; + + public TestFirehose(final InputRowParser> parser) + { + this.parser = parser; + } + + public void addRows(List> rows) + { + synchronized (this) { + queue.addAll(rows); + notifyAll(); + } + } + + @Override + public boolean hasMore() + { + try { + synchronized (this) { + while (queue.isEmpty() && !closed) { + wait(); + } + return !queue.isEmpty(); + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + } + + @Override + public InputRow nextRow() + { + synchronized (this) { + final InputRow row = parser.parseBatch(queue.remove(0)).get(0); + if (row != null && row.getRaw(FAIL_DIM) != null) { + throw new ParseException(FAIL_DIM); + } + return row; + } + } + + @Override + public Runnable commit() + { + return () -> {}; + } + + @Override + public void close() throws IOException + { + synchronized (this) { + closed = true; + notifyAll(); + } + } + } + + private static class TestFirehoseFactory implements FirehoseFactory + { + public TestFirehoseFactory() + { + } + + @Override + @SuppressWarnings("unchecked") + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws ParseException + { + return new TestFirehose(parser); + } + } + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + private DateTime now; + private ListeningExecutorService taskExec; + private Map> handOffCallbacks; + private Collection publishedSegments; + private CountDownLatch segmentLatch; + private CountDownLatch handoffLatch; + + @Before + public void setUp() + { + EmittingLogger.registerEmitter(emitter); + emitter.start(); + taskExec = MoreExecutors.listeningDecorator(Execs.singleThreaded("realtime-index-task-test-%d")); + now = DateTimes.nowUtc(); + + TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); + derbyConnector.createDataSourceTable(); + derbyConnector.createTaskTables(); + derbyConnector.createSegmentTable(); + derbyConnector.createPendingSegmentsTable(); + } + + @After + public void tearDown() + { + taskExec.shutdownNow(); + } + + @Test(timeout = 60_000L) + public void testDefaultResource() throws Exception + { + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); + Assert.assertEquals(task.getId(), task.getTaskResource().getAvailabilityGroup()); + } + + + @Test(timeout = 60_000L, expected = ExecutionException.class) + public void testHandoffTimeout() throws Exception + { + expectPublishedSegments(1); + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, true, 100L); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final ListenableFuture statusFuture = runTask(task, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1") + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + Collection publishedSegments = awaitSegments(); + + Assert.assertEquals(1, task.getMetrics().processed()); + Assert.assertNotNull(Iterables.getOnlyElement(publishedSegments)); + + // handoff would timeout, resulting in exception + statusFuture.get(); + } + + @Test(timeout = 60_000L) + public void testBasics() throws Exception + { + expectPublishedSegments(1); + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final ListenableFuture statusFuture = runTask(task, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), + ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for publish. + Collection publishedSegments = awaitSegments(); + + // Check metrics. + Assert.assertEquals(2, task.getMetrics().processed()); + Assert.assertEquals(0, task.getMetrics().thrownAway()); + Assert.assertEquals(0, task.getMetrics().unparseable()); + + // Do some queries. + Assert.assertEquals(2, sumMetric(task, null, "rows")); + Assert.assertEquals(3, sumMetric(task, null, "met1")); + + awaitHandoffs(); + + for (DataSegment publishedSegment : publishedSegments) { + Optional>> optional = handOffCallbacks.entrySet().stream() + .filter(e -> e.getKey().equals(new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ))) + .findFirst(); + + Assert.assertTrue( + publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, + optional.isPresent() + ); + Pair executorRunnablePair = optional.get().getValue(); + + // Simulate handoff. + executorRunnablePair.lhs.execute(executorRunnablePair.rhs); + } + handOffCallbacks.clear(); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + } + + @Test(timeout = 60_000L) + public void testLateData() throws Exception + { + expectPublishedSegments(1); + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final ListenableFuture statusFuture = runTask(task, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), + // Data is from 2 days ago, should still be processed + ImmutableMap.of("t", now.minus(new Period("P2D")).getMillis(), "dim2", "bar", "met1", 2.0) + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for publish. + Collection publishedSegments = awaitSegments(); + + // Check metrics. + Assert.assertEquals(2, task.getMetrics().processed()); + Assert.assertEquals(0, task.getMetrics().thrownAway()); + Assert.assertEquals(0, task.getMetrics().unparseable()); + + // Do some queries. + Assert.assertEquals(2, sumMetric(task, null, "rows")); + Assert.assertEquals(3, sumMetric(task, null, "met1")); + + awaitHandoffs(); + + for (DataSegment publishedSegment : publishedSegments) { + Optional>> optional = handOffCallbacks.entrySet().stream() + .filter(e -> e.getKey().equals(new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ))) + .findFirst(); + + Assert.assertTrue( + publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, + optional.isPresent() + ); + Pair executorRunnablePair = optional.get().getValue(); + + // Simulate handoff. + executorRunnablePair.lhs.execute(executorRunnablePair.rhs); + } + handOffCallbacks.clear(); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + } + + @Test(timeout = 60_000L) + public void testMaxRowsPerSegment() throws Exception + { + // Expect 2 segments as we will hit maxRowsPerSegment + expectPublishedSegments(2); + + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final ListenableFuture statusFuture = runTask(task, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + // maxRowsPerSegment is 1000 as configured in #makeRealtimeTask + for (int i = 0; i < 2000; i++) { + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo-" + i, "met1", "1") + ) + ); + } + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for publish. + Collection publishedSegments = awaitSegments(); + + // Check metrics. + Assert.assertEquals(2000, task.getMetrics().processed()); + Assert.assertEquals(0, task.getMetrics().thrownAway()); + Assert.assertEquals(0, task.getMetrics().unparseable()); + + // Do some queries. + Assert.assertEquals(2000, sumMetric(task, null, "rows")); + Assert.assertEquals(2000, sumMetric(task, null, "met1")); + + awaitHandoffs(); + + for (DataSegment publishedSegment : publishedSegments) { + Optional>> optional = handOffCallbacks.entrySet().stream() + .filter(e -> e.getKey().equals(new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ))) + .findFirst(); + + Assert.assertTrue( + publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, + optional.isPresent() + ); + Pair executorRunnablePair = optional.get().getValue(); + + // Simulate handoff. + executorRunnablePair.lhs.execute(executorRunnablePair.rhs); + } + handOffCallbacks.clear(); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + } + + @Test(timeout = 60_000L) + public void testTransformSpec() throws Exception + { + expectPublishedSegments(2); + + final TransformSpec transformSpec = new TransformSpec( + new SelectorDimFilter("dim1", "foo", null), + ImmutableList.of( + new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) + ) + ); + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, transformSpec, true, 0); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final ListenableFuture statusFuture = runTask(task, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), + ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", 2.0), + ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + Collection publishedSegments = awaitSegments(); + + // Check metrics. + Assert.assertEquals(2, task.getMetrics().processed()); + Assert.assertEquals(1, task.getMetrics().thrownAway()); + Assert.assertEquals(0, task.getMetrics().unparseable()); + + // Do some queries. + Assert.assertEquals(2, sumMetric(task, null, "rows")); + Assert.assertEquals(2, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows")); + Assert.assertEquals(0, sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "rows")); + Assert.assertEquals(3, sumMetric(task, null, "met1")); + + awaitHandoffs(); + + for (DataSegment publishedSegment : publishedSegments) { + Optional>> optional = handOffCallbacks.entrySet().stream() + .filter(e -> e.getKey().equals(new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ))) + .findFirst(); + + Assert.assertTrue( + publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, + optional.isPresent() + ); + Pair executorRunnablePair = optional.get().getValue(); + + // Simulate handoff. + executorRunnablePair.lhs.execute(executorRunnablePair.rhs); + } + handOffCallbacks.clear(); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + } + + @Test(timeout = 60_000L) + public void testReportParseExceptionsOnBadMetric() throws Exception + { + expectPublishedSegments(0); + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, true); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final ListenableFuture statusFuture = runTask(task, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "foo"), + ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", "foo"), + ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for the task to finish. + expectedException.expect(ExecutionException.class); + expectedException.expectCause(CoreMatchers.instanceOf(ParseException.class)); + expectedException.expectCause( + ThrowableMessageMatcher.hasMessage( + CoreMatchers.containsString("Encountered parse error for aggregator[met1]") + ) + ); + expectedException.expect( + ThrowableCauseMatcher.hasCause( + ThrowableCauseMatcher.hasCause( + CoreMatchers.allOf( + CoreMatchers.instanceOf(ParseException.class), + ThrowableMessageMatcher.hasMessage( + CoreMatchers.containsString("Unable to parse value[foo] for field[met1]") + ) + ) + ) + ) + ); + statusFuture.get(); + } + + @Test(timeout = 60_000L) + public void testNoReportParseExceptions() throws Exception + { + expectPublishedSegments(1); + + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, false); + final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final ListenableFuture statusFuture = runTask(task, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + firehose.addRows( + Arrays.asList( + // Good row- will be processed. + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), + + // Null row- will be thrown away. + null, + + // Bad metric- will count as processed, but that particular metric won't update. + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "foo"), + + // Bad row- will be unparseable. + ImmutableMap.of("dim1", "foo", "met1", 2.0, FAIL_DIM, "x"), + + // Good row- will be processed. + ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for publish. + Collection publishedSegments = awaitSegments(); + + DataSegment publishedSegment = Iterables.getOnlyElement(publishedSegments); + + // Check metrics. + Assert.assertEquals(3, task.getMetrics().processed()); + Assert.assertEquals(0, task.getMetrics().thrownAway()); + Assert.assertEquals(2, task.getMetrics().unparseable()); + + // Do some queries. + Assert.assertEquals(3, sumMetric(task, null, "rows")); + Assert.assertEquals(3, sumMetric(task, null, "met1")); + + awaitHandoffs(); + + // Simulate handoff. + for (Map.Entry> entry : handOffCallbacks.entrySet()) { + final Pair executorRunnablePair = entry.getValue(); + Assert.assertEquals( + new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ), + entry.getKey() + ); + executorRunnablePair.lhs.execute(executorRunnablePair.rhs); + } + handOffCallbacks.clear(); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + } + + @Test(timeout = 60_000L) + public void testRestore() throws Exception + { + expectPublishedSegments(0); + + final File directory = tempFolder.newFolder(); + final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); + final DataSegment publishedSegment; + + // First run: + { + final TaskToolbox taskToolbox = makeToolbox(task1, directory); + final ListenableFuture statusFuture = runTask(task1, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task1.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task1.getFirehose(); + + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo") + ) + ); + + // Trigger graceful shutdown. + task1.stopGracefully(); + + // Wait for the task to finish. The status doesn't really matter, but we'll check it anyway. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + + // Nothing should be published. + Assert.assertTrue(publishedSegments.isEmpty()); + } + + // Second run: + { + expectPublishedSegments(1); + final AppenderatorDriverRealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); + final TaskToolbox taskToolbox = makeToolbox(task2, directory); + final ListenableFuture statusFuture = runTask(task2, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task2.getFirehose() == null) { + Thread.sleep(50); + } + + // Do a query, at this point the previous data should be loaded. + Assert.assertEquals(1, sumMetric(task2, null, "rows")); + + final TestFirehose firehose = (TestFirehose) task2.getFirehose(); + + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim2", "bar") + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + Collection publishedSegments = awaitSegments(); + + publishedSegment = Iterables.getOnlyElement(publishedSegments); + + // Do a query. + Assert.assertEquals(2, sumMetric(task2, null, "rows")); + + awaitHandoffs(); + + // Simulate handoff. + for (Map.Entry> entry : handOffCallbacks.entrySet()) { + final Pair executorRunnablePair = entry.getValue(); + Assert.assertEquals( + new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ), + entry.getKey() + ); + executorRunnablePair.lhs.execute(executorRunnablePair.rhs); + } + handOffCallbacks.clear(); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + } + } + + @Test(timeout = 60_000L) + public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception + { + final TaskStorage taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); + final File directory = tempFolder.newFolder(); + final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); + final DataSegment publishedSegment; + + // First run: + { + expectPublishedSegments(1); + final TaskToolbox taskToolbox = makeToolbox(task1, taskStorage, directory); + final ListenableFuture statusFuture = runTask(task1, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task1.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task1.getFirehose(); + + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo") + ) + ); + + // Stop the firehose, this will trigger a finishJob. + firehose.close(); + + Collection publishedSegments = awaitSegments(); + + publishedSegment = Iterables.getOnlyElement(publishedSegments); + + // Do a query. + Assert.assertEquals(1, sumMetric(task1, null, "rows")); + + // Trigger graceful shutdown. + task1.stopGracefully(); + + // Wait for the task to finish. The status doesn't really matter. + while (!statusFuture.isDone()) { + Thread.sleep(50); + } + } + + // Second run: + { + expectPublishedSegments(1); + final AppenderatorDriverRealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); + final TaskToolbox taskToolbox = makeToolbox(task2, taskStorage, directory); + final ListenableFuture statusFuture = runTask(task2, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task2.getFirehose() == null) { + Thread.sleep(50); + } + + // Stop the firehose again, this will start another handoff. + final TestFirehose firehose = (TestFirehose) task2.getFirehose(); + + // Stop the firehose, this will trigger a finishJob. + firehose.close(); + + awaitHandoffs(); + + // Simulate handoff. + for (Map.Entry> entry : handOffCallbacks.entrySet()) { + final Pair executorRunnablePair = entry.getValue(); + Assert.assertEquals( + new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ), + entry.getKey() + ); + executorRunnablePair.lhs.execute(executorRunnablePair.rhs); + } + handOffCallbacks.clear(); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + } + } + + @Test(timeout = 60_000L) + public void testRestoreCorruptData() throws Exception + { + final File directory = tempFolder.newFolder(); + final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); + + // First run: + { + expectPublishedSegments(0); + + final TaskToolbox taskToolbox = makeToolbox(task1, directory); + final ListenableFuture statusFuture = runTask(task1, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task1.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task1.getFirehose(); + + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo") + ) + ); + + // Trigger graceful shutdown. + task1.stopGracefully(); + + // Wait for the task to finish. The status doesn't really matter, but we'll check it anyway. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + + // Nothing should be published. + Assert.assertTrue(publishedSegments.isEmpty()); + } + + Optional optional = FileUtils.listFiles(directory, null, true).stream() + .filter(f -> f.getName().equals("00000.smoosh")) + .findFirst(); + + Assert.assertTrue("Could not find smoosh file", optional.isPresent()); + + // Corrupt the data: + final File smooshFile = optional.get(); + + Files.write(smooshFile.toPath(), StringUtils.toUtf8("oops!")); + + // Second run: + { + expectPublishedSegments(0); + + final AppenderatorDriverRealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); + final TaskToolbox taskToolbox = makeToolbox(task2, directory); + final ListenableFuture statusFuture = runTask(task2, taskToolbox); + + // Wait for the task to finish. + boolean caught = false; + try { + statusFuture.get(); + } + catch (Exception expected) { + caught = true; + } + Assert.assertTrue("expected exception", caught); + } + } + + @Test(timeout = 60_000L) + public void testStopBeforeStarting() throws Exception + { + expectPublishedSegments(0); + + final File directory = tempFolder.newFolder(); + final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); + + task1.stopGracefully(); + final TaskToolbox taskToolbox = makeToolbox(task1, directory); + final ListenableFuture statusFuture = runTask(task1, taskToolbox); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + } + + private ListenableFuture runTask(final Task task, final TaskToolbox toolbox) + { + return taskExec.submit( + new Callable() + { + @Override + public TaskStatus call() throws Exception + { + try { + if (task.isReady(toolbox.getTaskActionClient())) { + return task.run(toolbox); + } else { + throw new ISE("Task is not ready"); + } + } + catch (Exception e) { + log.warn(e, "Task failed"); + throw e; + } + } + } + ); + } + + private AppenderatorDriverRealtimeIndexTask makeRealtimeTask(final String taskId) + { + return makeRealtimeTask(taskId, TransformSpec.NONE, true, 0); + } + + private AppenderatorDriverRealtimeIndexTask makeRealtimeTask(final String taskId, boolean reportParseExceptions) + { + return makeRealtimeTask(taskId, TransformSpec.NONE, reportParseExceptions, 0); + } + + private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( + final String taskId, + final TransformSpec transformSpec, + final boolean reportParseExceptions, + final long handoffTimeout + ) + { + ObjectMapper objectMapper = new DefaultObjectMapper(); + DataSchema dataSchema = new DataSchema( + "test_ds", + TestHelper.makeJsonMapper().convertValue( + new MapInputRowParser( + new TimeAndDimsParseSpec( + new TimestampSpec("t", "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim1t")), + null, + null + ) + ) + ), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ), + new AggregatorFactory[]{new CountAggregatorFactory("rows"), new LongSumAggregatorFactory("met1", "met1")}, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + transformSpec, + objectMapper + ); + RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig( + new TestFirehoseFactory(), + null, + null + ); + RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( + 1000, + 1000, + new Period("P1Y"), + new Period("PT10M"), + null, + null, + new ServerTimeRejectionPolicyFactory(), + null, + null, + null, + true, + 0, + 0, + reportParseExceptions, + handoffTimeout, + null, + null + ); + return new AppenderatorDriverRealtimeIndexTask( + taskId, + null, + new FireDepartment(dataSchema, realtimeIOConfig, realtimeTuningConfig), + null + ) + { + @Override + protected boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory) + { + return true; + } + }; + } + + private TaskToolbox makeToolbox( + final Task task, + final File directory + ) + { + return makeToolbox( + task, + new HeapMemoryTaskStorage(new TaskStorageConfig(null)), + directory + ); + } + + private void expectPublishedSegments(int count) + { + segmentLatch = new CountDownLatch(count); + handoffLatch = new CountDownLatch(count); + } + + private Collection awaitSegments() throws InterruptedException + { + Assert.assertTrue( + "Timed out waiting for segments to be published", + segmentLatch.await(1, TimeUnit.MINUTES) + ); + + return publishedSegments; + } + + private void awaitHandoffs() throws InterruptedException + { + Assert.assertTrue( + "Timed out waiting for segments to be handed off", + handoffLatch.await(1, TimeUnit.MINUTES) + ); + } + + private TaskToolbox makeToolbox( + final Task task, + final TaskStorage taskStorage, + final File directory + ) + { + publishedSegments = new CopyOnWriteArrayList<>(); + + Assert.assertFalse( + "Segment latch not initialized, did you forget to call expectPublishSegments?", + segmentLatch == null + ); + + ObjectMapper mapper = new DefaultObjectMapper(); + mapper.registerSubtypes(LinearShardSpec.class); + mapper.registerSubtypes(NumberedShardSpec.class); + IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator( + mapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnectorRule.getConnector() + ) + { + @Override + public Set announceHistoricalSegments(Set segments) throws IOException + { + Set result = super.announceHistoricalSegments(segments); + + publishedSegments.addAll(result); + segments.forEach(s -> segmentLatch.countDown()); + + return result; + } + + @Override + public SegmentPublishResult announceHistoricalSegments( + Set segments, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata + ) throws IOException + { + SegmentPublishResult result = super.announceHistoricalSegments(segments, startMetadata, endMetadata); + + publishedSegments.addAll(result.getSegments()); + result.getSegments().forEach(s -> segmentLatch.countDown()); + + return result; + } + }; + final TaskConfig taskConfig = new TaskConfig(directory.getPath(), null, null, 50000, null, false, null, null); + final TaskLockbox taskLockbox = new TaskLockbox(taskStorage); + try { + taskStorage.insert(task, TaskStatus.running(task.getId())); + } + catch (EntryExistsException e) { + // suppress + } + taskLockbox.syncFromStorage(); + final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( + taskLockbox, + mdc, + emitter, + EasyMock.createMock(SupervisorManager.class) + ); + final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( + taskStorage, + taskActionToolbox + ); + IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( + null, + null, + null + ) + { + @Override + public QueryRunner decorate( + QueryRunner delegate, QueryToolChest> toolChest + ) + { + return delegate; + } + }; + final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate( + ImmutableMap., QueryRunnerFactory>of( + TimeseriesQuery.class, + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest(queryRunnerDecorator), + new TimeseriesQueryEngine(), + new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + // do nothing + } + } + ) + ) + ); + handOffCallbacks = new ConcurrentHashMap<>(); + final SegmentHandoffNotifierFactory handoffNotifierFactory = new SegmentHandoffNotifierFactory() + { + @Override + public SegmentHandoffNotifier createSegmentHandoffNotifier(String dataSource) + { + return new SegmentHandoffNotifier() + { + @Override + public boolean registerSegmentHandoffCallback( + SegmentDescriptor descriptor, Executor exec, Runnable handOffRunnable + ) + { + handOffCallbacks.put(descriptor, new Pair<>(exec, handOffRunnable)); + handoffLatch.countDown(); + return true; + } + + @Override + public void start() + { + //Noop + } + + @Override + public void close() + { + //Noop + } + + Map> getHandOffCallbacks() + { + return handOffCallbacks; + } + }; + } + }; + final TestUtils testUtils = new TestUtils(); + SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return Lists.newArrayList(); + } + }; + final TaskToolboxFactory toolboxFactory = new TaskToolboxFactory( + taskConfig, + taskActionClientFactory, + emitter, + new TestDataSegmentPusher(), + new TestDataSegmentKiller(), + null, // DataSegmentMover + null, // DataSegmentArchiver + new TestDataSegmentAnnouncer(), + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), + handoffNotifierFactory, + () -> conglomerate, + MoreExecutors.sameThreadExecutor(), // queryExecutorService + EasyMock.createMock(MonitorScheduler.class), + new SegmentLoaderFactory( + new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) + ), + testUtils.getTestObjectMapper(), + testUtils.getTestIndexIO(), + MapCache.create(1024), + new CacheConfig(), + testUtils.getTestIndexMergerV9(), + EasyMock.createNiceMock(DruidNodeAnnouncer.class), + EasyMock.createNiceMock(DruidNode.class), + new LookupNodeService("tier"), + new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0) + ); + + return toolboxFactory.build(task); + } + + public long sumMetric(final Task task, final DimFilter filter, final String metric) throws Exception + { + // Do a query. + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test_ds") + .filters(filter) + .aggregators( + ImmutableList.of( + new LongSumAggregatorFactory(metric, metric) + ) + ).granularity(Granularities.ALL) + .intervals("2000/3000") + .build(); + + List> results = + task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList(); + return results.isEmpty() ? 0 : results.get(0).getValue().getLongMetric(metric); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index d1b4bebea305..f9675bac1929 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -23,8 +23,10 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; 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.Sets; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; @@ -56,15 +58,15 @@ import io.druid.indexing.common.actions.TaskActionToolbox; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskStorageConfig; -import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.HeapMemoryTaskStorage; -import io.druid.indexing.overlord.SegmentPublishResult; +import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import io.druid.indexing.overlord.TaskLockbox; import io.druid.indexing.overlord.TaskStorage; import io.druid.indexing.overlord.supervisor.SupervisorManager; import io.druid.indexing.test.TestDataSegmentAnnouncer; import io.druid.indexing.test.TestDataSegmentKiller; import io.druid.indexing.test.TestDataSegmentPusher; +import io.druid.indexing.test.TestIndexerMetadataStorageCoordinator; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; @@ -77,8 +79,6 @@ import io.druid.java.util.common.parsers.ParseException; import io.druid.math.expr.ExprMacroTable; import io.druid.metadata.EntryExistsException; -import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; -import io.druid.metadata.TestDerbyConnector; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Druids; import io.druid.query.IntervalChunkingQueryRunnerDecorator; @@ -119,9 +119,6 @@ import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; -import io.druid.timeline.partition.LinearShardSpec; -import io.druid.timeline.partition.NumberedShardSpec; -import org.apache.commons.io.FileUtils; import org.easymock.EasyMock; import org.hamcrest.CoreMatchers; import org.joda.time.DateTime; @@ -140,19 +137,13 @@ import java.io.IOException; import java.nio.file.Files; import java.util.Arrays; -import java.util.Collection; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Optional; -import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; -import java.util.concurrent.TimeUnit; public class RealtimeIndexTaskTest { @@ -249,15 +240,9 @@ public Firehose connect(InputRowParser parser, File temporaryDirectory) throws I @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); - @Rule - public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private DateTime now; private ListeningExecutorService taskExec; private Map> handOffCallbacks; - private Collection publishedSegments; - private CountDownLatch segmentLatch; - private CountDownLatch handoffLatch; @Before public void setUp() @@ -266,12 +251,6 @@ public void setUp() emitter.start(); taskExec = MoreExecutors.listeningDecorator(Execs.singleThreaded("realtime-index-task-test-%d")); now = DateTimes.nowUtc(); - - TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); - derbyConnector.createDataSourceTable(); - derbyConnector.createTaskTables(); - derbyConnector.createSegmentTable(); - derbyConnector.createPendingSegmentsTable(); } @After @@ -300,9 +279,9 @@ public void testDefaultResource() throws Exception @Test(timeout = 60_000L, expected = ExecutionException.class) public void testHandoffTimeout() throws Exception { - expectPublishedSegments(1); + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); final RealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, true, 100L); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); final ListenableFuture statusFuture = runTask(task, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -321,10 +300,14 @@ public void testHandoffTimeout() throws Exception // Stop the firehose, this will drain out existing events. firehose.close(); - Collection publishedSegments = awaitSegments(); + // Wait for publish. + while (mdc.getPublished().isEmpty()) { + Thread.sleep(50); + } Assert.assertEquals(1, task.getMetrics().processed()); - Assert.assertNotNull(Iterables.getOnlyElement(publishedSegments)); + Assert.assertNotNull(Iterables.getOnlyElement(mdc.getPublished())); + // handoff would timeout, resulting in exception statusFuture.get(); @@ -333,10 +316,11 @@ public void testHandoffTimeout() throws Exception @Test(timeout = 60_000L) public void testBasics() throws Exception { - expectPublishedSegments(1); + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); final RealtimeIndexTask task = makeRealtimeTask(null); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); final ListenableFuture statusFuture = runTask(task, taskToolbox); + final DataSegment publishedSegment; // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -348,6 +332,7 @@ public void testBasics() throws Exception firehose.addRows( ImmutableList.of( ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), + ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", 2.0), ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) ) ); @@ -356,168 +341,32 @@ public void testBasics() throws Exception firehose.close(); // Wait for publish. - Collection publishedSegments = awaitSegments(); - - // Check metrics. - Assert.assertEquals(2, task.getMetrics().processed()); - Assert.assertEquals(0, task.getMetrics().thrownAway()); - Assert.assertEquals(0, task.getMetrics().unparseable()); - - // Do some queries. - Assert.assertEquals(2, sumMetric(task, null, "rows")); - Assert.assertEquals(3, sumMetric(task, null, "met1")); - - awaitHandoffs(); - - for (DataSegment publishedSegment : publishedSegments) { - Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey().equals(new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ))) - .findFirst(); - - Assert.assertTrue( - publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, - optional.isPresent() - ); - Pair executorRunnablePair = optional.get().getValue(); - - // Simulate handoff. - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - @Test(timeout = 60_000L) - public void testLateData() throws Exception - { - expectPublishedSegments(1); - final RealtimeIndexTask task = makeRealtimeTask(null); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { + while (mdc.getPublished().isEmpty()) { Thread.sleep(50); } - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), - // Data is from 2 days ago, should still be processed - ImmutableMap.of("t", now.minus(new Period("P2D")).getMillis(), "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - Collection publishedSegments = awaitSegments(); + publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); // Check metrics. Assert.assertEquals(2, task.getMetrics().processed()); - Assert.assertEquals(0, task.getMetrics().thrownAway()); + Assert.assertEquals(1, task.getMetrics().thrownAway()); Assert.assertEquals(0, task.getMetrics().unparseable()); // Do some queries. Assert.assertEquals(2, sumMetric(task, null, "rows")); Assert.assertEquals(3, sumMetric(task, null, "met1")); - awaitHandoffs(); - - for (DataSegment publishedSegment : publishedSegments) { - Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey().equals(new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ))) - .findFirst(); - - Assert.assertTrue( - publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, - optional.isPresent() - ); - Pair executorRunnablePair = optional.get().getValue(); - - // Simulate handoff. - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - @Test(timeout = 60_000L) - public void testMaxRowsPerSegment() throws Exception - { - // Expect 2 segments as we will hit maxRowsPerSegment - expectPublishedSegments(2); - - final RealtimeIndexTask task = makeRealtimeTask(null); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - // maxRowsPerSegment is 1000 as configured in #makeRealtimeTask - for (int i = 0; i < 2000; i++) { - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo-" + i, "met1", "1") - ) - ); - } - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - Collection publishedSegments = awaitSegments(); - - // Check metrics. - Assert.assertEquals(2000, task.getMetrics().processed()); - Assert.assertEquals(0, task.getMetrics().thrownAway()); - Assert.assertEquals(0, task.getMetrics().unparseable()); - - // Do some queries. - Assert.assertEquals(2000, sumMetric(task, null, "rows")); - Assert.assertEquals(2000, sumMetric(task, null, "met1")); - - awaitHandoffs(); - - for (DataSegment publishedSegment : publishedSegments) { - Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey().equals(new SegmentDescriptor( + // Simulate handoff. + for (Map.Entry> entry : handOffCallbacks.entrySet()) { + final Pair executorRunnablePair = entry.getValue(); + Assert.assertEquals( + new SegmentDescriptor( publishedSegment.getInterval(), publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() - ))) - .findFirst(); - - Assert.assertTrue( - publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, - optional.isPresent() + ), + entry.getKey() ); - Pair executorRunnablePair = optional.get().getValue(); - - // Simulate handoff. executorRunnablePair.lhs.execute(executorRunnablePair.rhs); } handOffCallbacks.clear(); @@ -530,8 +379,7 @@ public void testMaxRowsPerSegment() throws Exception @Test(timeout = 60_000L) public void testTransformSpec() throws Exception { - expectPublishedSegments(2); - + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); final TransformSpec transformSpec = new TransformSpec( new SelectorDimFilter("dim1", "foo", null), ImmutableList.of( @@ -539,8 +387,9 @@ public void testTransformSpec() throws Exception ) ); final RealtimeIndexTask task = makeRealtimeTask(null, transformSpec, true, 0); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); final ListenableFuture statusFuture = runTask(task, taskToolbox); + final DataSegment publishedSegment; // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -560,37 +409,35 @@ public void testTransformSpec() throws Exception // Stop the firehose, this will drain out existing events. firehose.close(); - Collection publishedSegments = awaitSegments(); + // Wait for publish. + while (mdc.getPublished().isEmpty()) { + Thread.sleep(50); + } + + publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); // Check metrics. - Assert.assertEquals(2, task.getMetrics().processed()); - Assert.assertEquals(1, task.getMetrics().thrownAway()); + Assert.assertEquals(1, task.getMetrics().processed()); + Assert.assertEquals(2, task.getMetrics().thrownAway()); Assert.assertEquals(0, task.getMetrics().unparseable()); // Do some queries. - Assert.assertEquals(2, sumMetric(task, null, "rows")); - Assert.assertEquals(2, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows")); + Assert.assertEquals(1, sumMetric(task, null, "rows")); + Assert.assertEquals(1, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows")); Assert.assertEquals(0, sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "rows")); - Assert.assertEquals(3, sumMetric(task, null, "met1")); - - awaitHandoffs(); + Assert.assertEquals(1, sumMetric(task, null, "met1")); - for (DataSegment publishedSegment : publishedSegments) { - Optional>> optional = handOffCallbacks.entrySet().stream() - .filter(e -> e.getKey().equals(new SegmentDescriptor( + // Simulate handoff. + for (Map.Entry> entry : handOffCallbacks.entrySet()) { + final Pair executorRunnablePair = entry.getValue(); + Assert.assertEquals( + new SegmentDescriptor( publishedSegment.getInterval(), publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() - ))) - .findFirst(); - - Assert.assertTrue( - publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, - optional.isPresent() + ), + entry.getKey() ); - Pair executorRunnablePair = optional.get().getValue(); - - // Simulate handoff. executorRunnablePair.lhs.execute(executorRunnablePair.rhs); } handOffCallbacks.clear(); @@ -603,9 +450,9 @@ public void testTransformSpec() throws Exception @Test(timeout = 60_000L) public void testReportParseExceptionsOnBadMetric() throws Exception { - expectPublishedSegments(0); + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); final RealtimeIndexTask task = makeRealtimeTask(null, true); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); final ListenableFuture statusFuture = runTask(task, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -653,11 +500,11 @@ public void testReportParseExceptionsOnBadMetric() throws Exception @Test(timeout = 60_000L) public void testNoReportParseExceptions() throws Exception { - expectPublishedSegments(1); - + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); final RealtimeIndexTask task = makeRealtimeTask(null, false); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); + final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); final ListenableFuture statusFuture = runTask(task, taskToolbox); + final DataSegment publishedSegment; // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -680,6 +527,9 @@ public void testNoReportParseExceptions() throws Exception // Bad row- will be unparseable. ImmutableMap.of("dim1", "foo", "met1", 2.0, FAIL_DIM, "x"), + // Old row- will be thrownAway. + ImmutableMap.of("t", now.minus(Period.days(1)).getMillis(), "dim1", "foo", "met1", 2.0), + // Good row- will be processed. ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) ) @@ -689,21 +539,21 @@ public void testNoReportParseExceptions() throws Exception firehose.close(); // Wait for publish. - Collection publishedSegments = awaitSegments(); + while (mdc.getPublished().isEmpty()) { + Thread.sleep(50); + } - DataSegment publishedSegment = Iterables.getOnlyElement(publishedSegments); + publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); // Check metrics. Assert.assertEquals(3, task.getMetrics().processed()); - Assert.assertEquals(0, task.getMetrics().thrownAway()); + Assert.assertEquals(1, task.getMetrics().thrownAway()); Assert.assertEquals(2, task.getMetrics().unparseable()); // Do some queries. Assert.assertEquals(3, sumMetric(task, null, "rows")); Assert.assertEquals(3, sumMetric(task, null, "met1")); - awaitHandoffs(); - // Simulate handoff. for (Map.Entry> entry : handOffCallbacks.entrySet()) { final Pair executorRunnablePair = entry.getValue(); @@ -727,15 +577,14 @@ public void testNoReportParseExceptions() throws Exception @Test(timeout = 60_000L) public void testRestore() throws Exception { - expectPublishedSegments(0); - final File directory = tempFolder.newFolder(); final RealtimeIndexTask task1 = makeRealtimeTask(null); final DataSegment publishedSegment; // First run: { - final TaskToolbox taskToolbox = makeToolbox(task1, directory); + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory); final ListenableFuture statusFuture = runTask(task1, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -759,14 +608,14 @@ public void testRestore() throws Exception Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); // Nothing should be published. - Assert.assertTrue(publishedSegments.isEmpty()); + Assert.assertEquals(Sets.newHashSet(), mdc.getPublished()); } // Second run: { - expectPublishedSegments(1); + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, directory); + final TaskToolbox taskToolbox = makeToolbox(task2, mdc, directory); final ListenableFuture statusFuture = runTask(task2, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -788,15 +637,16 @@ public void testRestore() throws Exception // Stop the firehose, this will drain out existing events. firehose.close(); - Collection publishedSegments = awaitSegments(); + // Wait for publish. + while (mdc.getPublished().isEmpty()) { + Thread.sleep(50); + } - publishedSegment = Iterables.getOnlyElement(publishedSegments); + publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); // Do a query. Assert.assertEquals(2, sumMetric(task2, null, "rows")); - awaitHandoffs(); - // Simulate handoff. for (Map.Entry> entry : handOffCallbacks.entrySet()) { final Pair executorRunnablePair = entry.getValue(); @@ -822,14 +672,14 @@ public void testRestore() throws Exception public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception { final TaskStorage taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); final File directory = tempFolder.newFolder(); final RealtimeIndexTask task1 = makeRealtimeTask(null); final DataSegment publishedSegment; // First run: { - expectPublishedSegments(1); - final TaskToolbox taskToolbox = makeToolbox(task1, taskStorage, directory); + final TaskToolbox taskToolbox = makeToolbox(task1, taskStorage, mdc, directory); final ListenableFuture statusFuture = runTask(task1, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -848,9 +698,12 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception // Stop the firehose, this will trigger a finishJob. firehose.close(); - Collection publishedSegments = awaitSegments(); + // Wait for publish. + while (mdc.getPublished().isEmpty()) { + Thread.sleep(50); + } - publishedSegment = Iterables.getOnlyElement(publishedSegments); + publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); // Do a query. Assert.assertEquals(1, sumMetric(task1, null, "rows")); @@ -866,9 +719,8 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception // Second run: { - expectPublishedSegments(1); final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, taskStorage, directory); + final TaskToolbox taskToolbox = makeToolbox(task2, taskStorage, mdc, directory); final ListenableFuture statusFuture = runTask(task2, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -882,7 +734,13 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception // Stop the firehose, this will trigger a finishJob. firehose.close(); - awaitHandoffs(); + // publishedSegment is still published. No reason it shouldn't be. + Assert.assertEquals(ImmutableSet.of(publishedSegment), mdc.getPublished()); + + // Wait for a handoffCallback to show up. + while (handOffCallbacks.isEmpty()) { + Thread.sleep(50); + } // Simulate handoff. for (Map.Entry> entry : handOffCallbacks.entrySet()) { @@ -913,9 +771,8 @@ public void testRestoreCorruptData() throws Exception // First run: { - expectPublishedSegments(0); - - final TaskToolbox taskToolbox = makeToolbox(task1, directory); + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory); final ListenableFuture statusFuture = runTask(task1, taskToolbox); // Wait for firehose to show up, it starts off null. @@ -939,26 +796,28 @@ public void testRestoreCorruptData() throws Exception Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); // Nothing should be published. - Assert.assertTrue(publishedSegments.isEmpty()); + Assert.assertEquals(Sets.newHashSet(), mdc.getPublished()); } - Optional optional = FileUtils.listFiles(directory, null, true).stream() - .filter(f -> f.getName().equals("00000.smoosh")) - .findFirst(); - - Assert.assertTrue("Could not find smoosh file", optional.isPresent()); - // Corrupt the data: - final File smooshFile = optional.get(); + final File smooshFile = new File( + StringUtils.format( + "%s/persistent/task/%s/work/persist/%s/%s_%s/0/00000.smoosh", + directory, + task1.getId(), + task1.getDataSource(), + Granularities.DAY.bucketStart(now), + Granularities.DAY.bucketEnd(now) + ) + ); Files.write(smooshFile.toPath(), StringUtils.toUtf8("oops!")); // Second run: { - expectPublishedSegments(0); - + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, directory); + final TaskToolbox taskToolbox = makeToolbox(task2, mdc, directory); final ListenableFuture statusFuture = runTask(task2, taskToolbox); // Wait for the task to finish. @@ -966,7 +825,7 @@ public void testRestoreCorruptData() throws Exception try { statusFuture.get(); } - catch (Exception expected) { + catch (Exception e) { caught = true; } Assert.assertTrue("expected exception", caught); @@ -976,13 +835,12 @@ public void testRestoreCorruptData() throws Exception @Test(timeout = 60_000L) public void testStopBeforeStarting() throws Exception { - expectPublishedSegments(0); - final File directory = tempFolder.newFolder(); final RealtimeIndexTask task1 = makeRealtimeTask(null); task1.stopGracefully(); - final TaskToolbox taskToolbox = makeToolbox(task1, directory); + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory); final ListenableFuture statusFuture = runTask(task1, taskToolbox); // Wait for the task to finish. @@ -1093,86 +951,25 @@ protected boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory) private TaskToolbox makeToolbox( final Task task, + final IndexerMetadataStorageCoordinator mdc, final File directory ) { return makeToolbox( task, new HeapMemoryTaskStorage(new TaskStorageConfig(null)), + mdc, directory ); } - private void expectPublishedSegments(int count) - { - segmentLatch = new CountDownLatch(count); - handoffLatch = new CountDownLatch(count); - } - - private Collection awaitSegments() throws InterruptedException - { - Assert.assertTrue( - "Timed out waiting for segments to be published", - segmentLatch.await(1, TimeUnit.MINUTES) - ); - - return publishedSegments; - } - - private void awaitHandoffs() throws InterruptedException - { - Assert.assertTrue( - "Timed out waiting for segments to be handed off", - handoffLatch.await(1, TimeUnit.MINUTES) - ); - } - private TaskToolbox makeToolbox( final Task task, final TaskStorage taskStorage, + final IndexerMetadataStorageCoordinator mdc, final File directory ) { - publishedSegments = new CopyOnWriteArrayList<>(); - - Assert.assertFalse( - "Segment latch not initialized, did you forget to call expectPublishSegments?", - segmentLatch == null - ); - - ObjectMapper mapper = new DefaultObjectMapper(); - mapper.registerSubtypes(LinearShardSpec.class); - mapper.registerSubtypes(NumberedShardSpec.class); - IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator( - mapper, - derbyConnectorRule.metadataTablesConfigSupplier().get(), - derbyConnectorRule.getConnector() - ) - { - @Override - public Set announceHistoricalSegments(Set segments) throws IOException - { - Set result = super.announceHistoricalSegments(segments); - - publishedSegments.addAll(result); - segments.forEach(s -> segmentLatch.countDown()); - - return result; - } - - @Override - public SegmentPublishResult announceHistoricalSegments( - Set segments, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata - ) throws IOException - { - SegmentPublishResult result = super.announceHistoricalSegments(segments, startMetadata, endMetadata); - - publishedSegments.addAll(result.getSegments()); - result.getSegments().forEach(s -> segmentLatch.countDown()); - - return result; - } - }; final TaskConfig taskConfig = new TaskConfig(directory.getPath(), null, null, 50000, null, false, null, null); final TaskLockbox taskLockbox = new TaskLockbox(taskStorage); try { @@ -1237,7 +1034,6 @@ public boolean registerSegmentHandoffCallback( ) { handOffCallbacks.put(descriptor, new Pair<>(exec, handOffRunnable)); - handoffLatch.countDown(); return true; } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index e5e152c46d56..2dc0a3142ac8 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -83,7 +83,6 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Comparators; import io.druid.metadata.DerbyMetadataStorageActionHandlerFactory; -import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; import io.druid.metadata.TestDerbyConnector; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.SegmentDescriptor; @@ -143,7 +142,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; @@ -228,7 +226,6 @@ public int compare(DataSegment dataSegment, DataSegment dataSegment2) private int announcedSinks; private SegmentHandoffNotifierFactory handoffNotifierFactory; private Map> handOffCallbacks; - private CountDownLatch handoffLatch; private static CountDownLatch publishCountDown; @@ -362,7 +359,6 @@ public void setUp() throws Exception EmittingLogger.registerEmitter(emitter); mapper = TEST_UTILS.getTestObjectMapper(); handOffCallbacks = new ConcurrentHashMap<>(); - handoffLatch = new CountDownLatch(1); // Set up things, the order does matter as if it is messed up then the setUp // should fail because of the Precondition checks in the respective setUp methods @@ -381,16 +377,6 @@ public void setUp() throws Exception taskRunner = setUpThreadPoolTaskRunner(tb); taskQueue = setUpTaskQueue(taskStorage, taskRunner); - - TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); - mapper.registerSubtypes( - new NamedType(MockExceptionalFirehoseFactory.class, "mockExcepFirehoseFactory"), - new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory") - ); - derbyConnector.createDataSourceTable(); - derbyConnector.createTaskTables(); - derbyConnector.createSegmentTable(); - derbyConnector.createPendingSegmentsTable(); } private TaskStorage setUpTaskStorage() @@ -412,6 +398,12 @@ private TaskStorage setUpTaskStorage() case METADATA_TASK_STORAGE: { TestDerbyConnector testDerbyConnector = derbyConnectorRule.getConnector(); + mapper.registerSubtypes( + new NamedType(MockExceptionalFirehoseFactory.class, "mockExcepFirehoseFactory"), + new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory") + ); + testDerbyConnector.createTaskTables(); + testDerbyConnector.createSegmentTable(); taskStorage = new MetadataTaskStorage( testDerbyConnector, new TaskStorageConfig(null), @@ -451,7 +443,6 @@ public boolean registerSegmentHandoffCallback( ) { handOffCallbacks.put(descriptor, new Pair<>(exec, handOffRunnable)); - handoffLatch.countDown(); return true; } @@ -527,7 +518,7 @@ public Set announceHistoricalSegments(Set segments) private TaskToolboxFactory setUpTaskToolboxFactory( DataSegmentPusher dataSegmentPusher, SegmentHandoffNotifierFactory handoffNotifierFactory, - IndexerMetadataStorageCoordinator mdc + TestIndexerMetadataStorageCoordinator mdc ) throws IOException { Preconditions.checkNotNull(queryRunnerFactoryConglomerate); @@ -983,45 +974,6 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception public void testRealtimeIndexTask() throws Exception { publishCountDown = new CountDownLatch(1); - List publishedSegments = new CopyOnWriteArrayList<>(); - - IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator( - mapper, - derbyConnectorRule.metadataTablesConfigSupplier().get(), - derbyConnectorRule.getConnector() - ) - { - @Override - public Set announceHistoricalSegments(Set segments) throws IOException - { - Set result = super.announceHistoricalSegments(segments); - - publishedSegments.addAll(result); - publishCountDown.countDown(); - - return result; - } - - @Override - public SegmentPublishResult announceHistoricalSegments( - Set segments, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata - ) throws IOException - { - SegmentPublishResult result = super.announceHistoricalSegments(segments, startMetadata, endMetadata); - - publishedSegments.addAll(result.getSegments()); - publishCountDown.countDown(); - - return result; - } - }; - - tb = setUpTaskToolboxFactory(dataSegmentPusher, handoffNotifierFactory, mdc); - - taskRunner = setUpThreadPoolTaskRunner(tb); - - taskQueue = setUpTaskQueue(taskStorage, taskRunner); - monitorScheduler.addMonitor(EasyMock.anyObject(Monitor.class)); EasyMock.expectLastCall().atLeastOnce(); monitorScheduler.removeMonitor(EasyMock.anyObject(Monitor.class)); @@ -1035,7 +987,6 @@ public SegmentPublishResult announceHistoricalSegments( taskQueue.add(realtimeIndexTask); //wait for task to process events and publish segment publishCountDown.await(); - handoffLatch.await(); // Realtime Task has published the segment, simulate loading of segment to a historical node so that task finishes with SUCCESS status Assert.assertEquals(1, handOffCallbacks.size()); @@ -1052,8 +1003,8 @@ public SegmentPublishResult announceHistoricalSegments( Assert.assertEquals(1, announcedSinks); Assert.assertEquals(1, pushedSegments); - Assert.assertEquals(1, publishedSegments.size()); - DataSegment segment = publishedSegments.iterator().next(); + Assert.assertEquals(1, mdc.getPublished().size()); + DataSegment segment = mdc.getPublished().iterator().next(); Assert.assertEquals("test_ds", segment.getDataSource()); Assert.assertEquals(ImmutableList.of("dim1", "dim2"), segment.getDimensions()); Assert.assertEquals( diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java index f447cb7139cd..530250c14a6e 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java @@ -218,7 +218,6 @@ public void postEvents() throws Exception while ((line = reader.readLine()) != null) { if (i == 15) { // for the 15th line, use a time before the window dt = dt.minusMinutes(10); - dtFirst = dt; } else if (i == 16) { // remember this time to use in the expected response from the groupBy query dtGroupBy = dt; } else if (i == 18) { // use a time 6 seconds ago so it will be out of order diff --git a/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_queries.json b/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_queries.json index acd88ca893e0..b579b347c348 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_queries.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_queries.json @@ -31,8 +31,8 @@ { "timestamp" : "%%TIMESERIES_RESPONSE_TIMESTAMP%%", "result" : { - "chars_added" : 1642.0, - "edit_count" : 22 + "chars_added" : 1595.0, + "edit_count" : 21 } } ] From cdeac02faf9c0f3e53b8eb3311843c21a4b4c199 Mon Sep 17 00:00:00 2001 From: Kevin Conaway Date: Fri, 26 Jan 2018 16:57:55 -0500 Subject: [PATCH 07/10] #5261 Add separate AppenderatorDriverRealtimeIndexTask and revert changes to RealtimeIndexTask --- .../common/task/AppenderatorDriverRealtimeIndexTask.java | 6 ++---- .../task/AppenderatorDriverRealtimeIndexTaskTest.java | 8 ++++---- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index a146b5c3b321..f1548467ea92 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -31,7 +31,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; -import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; @@ -52,6 +51,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.parsers.ParseException; +import io.druid.java.util.emitter.EmittingLogger; import io.druid.query.DruidMetrics; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; @@ -82,7 +82,6 @@ import java.util.Collections; import java.util.Map; import java.util.Queue; -import java.util.Random; import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.ConcurrentLinkedQueue; @@ -91,10 +90,9 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask { - public static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; + private static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class); - private static final Random random = new Random(); private static String makeDatasource(FireDepartment fireDepartment) { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index deb233aa137a..044863217615 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -28,10 +28,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.NoopEmitter; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.metrics.MonitorScheduler; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.MapCache; import io.druid.data.input.Firehose; @@ -75,6 +71,10 @@ import io.druid.java.util.common.jackson.JacksonUtils; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.ParseException; +import io.druid.java.util.emitter.EmittingLogger; +import io.druid.java.util.emitter.core.NoopEmitter; +import io.druid.java.util.emitter.service.ServiceEmitter; +import io.druid.java.util.metrics.MonitorScheduler; import io.druid.math.expr.ExprMacroTable; import io.druid.metadata.EntryExistsException; import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; From bc927c2d8ae7c2919bf8dabb00bbdee17efdbec5 Mon Sep 17 00:00:00 2001 From: Kevin Conaway Date: Thu, 1 Feb 2018 20:23:23 -0500 Subject: [PATCH 08/10] #5261 Readability improvements in AppenderatorDriverRealtimeIndexTask. Combine publish and handoff futures in to single future --- .../AppenderatorDriverRealtimeIndexTask.java | 346 +++++++++--------- ...penderatorDriverRealtimeIndexTaskTest.java | 5 - 2 files changed, 167 insertions(+), 184 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index f1548467ea92..7c2bb17fffbf 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -27,10 +27,8 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.SettableFuture; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; @@ -76,7 +74,6 @@ import io.druid.timeline.DataSegment; import org.apache.commons.io.FileUtils; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.Collections; @@ -86,7 +83,9 @@ import java.util.TimerTask; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public class AppenderatorDriverRealtimeIndexTask extends AbstractTask { @@ -102,9 +101,6 @@ private static String makeDatasource(FireDepartment fireDepartment) @JsonIgnore private final FireDepartment spec; - @JsonIgnore - private final Queue> pendingPublishes; - @JsonIgnore private final Queue> pendingHandoffs; @@ -142,7 +138,6 @@ public AppenderatorDriverRealtimeIndexTask( context ); this.spec = fireDepartment; - this.pendingPublishes = new ConcurrentLinkedQueue<>(); this.pendingHandoffs = new ConcurrentLinkedQueue<>(); } @@ -188,80 +183,21 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception setupTimeoutAlert(); - boolean normalExit = true; - - // Wrap default DataSegmentAnnouncer such that we unlock intervals as we unannounce segments - final long lockTimeoutMs = getContextValue(Tasks.LOCK_TIMEOUT_KEY, Tasks.DEFAULT_LOCK_TIMEOUT); - // Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, http timeout error can occur while waiting for a - // lock to be acquired. - final DataSegmentAnnouncer lockingSegmentAnnouncer = new DataSegmentAnnouncer() - { - @Override - public void announceSegment(final DataSegment segment) throws IOException - { - // Side effect: Calling announceSegment causes a lock to be acquired - Preconditions.checkNotNull( - toolbox.getTaskActionClient().submit( - new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) - ), - "Cannot acquire a lock for interval[%s]", - 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 segments) throws IOException - { - // Side effect: Calling announceSegments causes locks to be acquired - for (DataSegment segment : segments) { - Preconditions.checkNotNull( - toolbox.getTaskActionClient().submit( - new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) - ), - "Cannot acquire a lock for interval[%s]", - segment.getInterval() - ); - } - toolbox.getSegmentAnnouncer().announceSegments(segments); - } - - @Override - public void unannounceSegments(Iterable segments) throws IOException - { - try { - toolbox.getSegmentAnnouncer().unannounceSegments(segments); - } - finally { - for (DataSegment segment : segments) { - toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval())); - } - } - } - }; + final DataSegmentAnnouncer lockingSegmentAnnouncer = createLockingSegmentAnnouncer(toolbox); DataSchema dataSchema = spec.getDataSchema(); RealtimeIOConfig realtimeIOConfig = spec.getIOConfig(); RealtimeTuningConfig tuningConfig = spec.getTuningConfig() .withBasePersistDirectory(toolbox.getPersistDir()); + logUnusedConfiguration(tuningConfig); + final FireDepartment fireDepartment = new FireDepartment( dataSchema, realtimeIOConfig, tuningConfig ); - this.metrics = fireDepartment.getMetrics(); + final RealtimeMetricsMonitor metricsMonitor = new RealtimeMetricsMonitor( ImmutableList.of(fireDepartment), ImmutableMap.of( @@ -269,20 +205,12 @@ public void unannounceSegments(Iterable segments) throws IOExceptio ) ); + this.metrics = fireDepartment.getMetrics(); + Supplier committerSupplier = null; final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); - LookupNodeService lookupNodeService = getContextValue(CTX_KEY_LOOKUP_TIER) == null ? - toolbox.getLookupNodeService() : - new LookupNodeService((String) getContextValue(CTX_KEY_LOOKUP_TIER)); - DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( - toolbox.getDruidNode(), - DruidNodeDiscoveryProvider.NODE_TYPE_PEON, - ImmutableMap.of( - toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), - lookupNodeService.getName(), lookupNodeService - ) - ); + DiscoveryDruidNode discoveryDruidNode = createDiscoveryDruidNode(toolbox); appenderator = newAppenderator(dataSchema, tuningConfig, fireDepartment.getMetrics(), toolbox, lockingSegmentAnnouncer); AppenderatorDriver driver = newDriver(dataSchema, appenderator, toolbox, fireDepartment.getMetrics()); @@ -303,14 +231,6 @@ public void unannounceSegments(Iterable segments) throws IOExceptio final FirehoseFactory firehoseFactory = spec.getIOConfig().getFirehoseFactory(); final boolean firehoseDrainableByClosing = isFirehoseDrainableByClosing(firehoseFactory); - // Skip connecting firehose if we've been stopped before we got started. - synchronized (this) { - if (!gracefullyStopped) { - firehose = firehoseFactory.connect(spec.getDataSchema().getParser(), firehoseTempDir); - committerSupplier = Committers.supplierFromFirehose(firehose); - } - } - int sequenceNumber = 0; String sequenceName = makeSequenceName(getId(), sequenceNumber); @@ -319,8 +239,16 @@ public void unannounceSegments(Iterable segments) throws IOExceptio return toolbox.getTaskActionClient().submit(action).isSuccess(); }; + // Skip connecting firehose if we've been stopped before we got started. + synchronized (this) { + if (!gracefullyStopped) { + firehose = firehoseFactory.connect(spec.getDataSchema().getParser(), firehoseTempDir); + committerSupplier = Committers.supplierFromFirehose(firehose); + } + } + // Time to read data! - while (firehose != null && (!gracefullyStopped || firehoseDrainableByClosing) && firehose.hasMore()) { + while (!gracefullyStopped && firehoseDrainableByClosing && firehose.hasMore()) { try { InputRow inputRow = firehose.nextRow(); @@ -376,90 +304,25 @@ public void unannounceSegments(Iterable segments) throws IOExceptio // Publish any remaining segments publishSegments(driver, publisher, committerSupplier, sequenceName); - if (!pendingPublishes.isEmpty()) { - ListenableFuture allPublishes = Futures.allAsList(pendingPublishes); - log.info("Waiting for segments to publish"); - - allPublishes.get(); - } - - if (!pendingHandoffs.isEmpty()) { - ListenableFuture allHandoffs = Futures.allAsList(pendingHandoffs); - log.info("Waiting for handoffs"); - - long handoffTimeout = tuningConfig.getHandoffConditionTimeout(); - - if (handoffTimeout > 0) { - allHandoffs.get(handoffTimeout, TimeUnit.MILLISECONDS); - } else { - allHandoffs.get(); - } - } + waitForSegmentHandoff(tuningConfig.getHandoffConditionTimeout()); } - } + } else if (firehose != null) { + log.info("Task was gracefully stopped, will persist data before exiting"); + persistAndWait(driver, committerSupplier.get()); + } } catch (Throwable e) { - normalExit = false; log.makeAlert(e, "Exception aborted realtime processing[%s]", dataSchema.getDataSource()) .emit(); throw e; } finally { - if (normalExit && gracefullyStopped) { - try { - // Persist if we had actually started. - if (firehose != null) { - log.info("Persisting remaining data."); - - final Committer committer = committerSupplier.get(); - final CountDownLatch persistLatch = new CountDownLatch(1); - driver.persist( - new Committer() - { - @Override - public Object getMetadata() - { - return committer.getMetadata(); - } - - @Override - public void run() - { - try { - committer.run(); - } - finally { - persistLatch.countDown(); - } - } - } - ); - persistLatch.await(); - } - } - catch (InterruptedException e) { - log.debug(e, "Interrupted while finishing the job"); - } - catch (Exception e) { - log.makeAlert(e, "Failed to finish realtime task").emit(); - throw e; - } - finally { - if (firehose != null) { - CloseQuietly.close(firehose); - } - toolbox.getMonitorScheduler().removeMonitor(metricsMonitor); - } - } + CloseQuietly.close(firehose); + CloseQuietly.close(appenderator); + CloseQuietly.close(driver); - if (appenderator != null) { - appenderator.close(); - } - - if (driver != null) { - driver.close(); - } + toolbox.getMonitorScheduler().removeMonitor(metricsMonitor); toolbox.getDataSegmentServerAnnouncer().unannounce(); toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); @@ -578,33 +441,158 @@ private void publishSegments( Collections.singletonList(sequenceName) ); - // Use a separate future to ensure that the publish future is not completed until after - // the handoff future is registered in the pending list - SettableFuture publishResultFuture = SettableFuture.create(); + ListenableFuture handoffFuture = Futures.transform(publishFuture, driver::registerHandoff); + + pendingHandoffs.add(handoffFuture); + } + + private void waitForSegmentHandoff(long handoffTimeout) throws InterruptedException, ExecutionException, + TimeoutException + { + if (!pendingHandoffs.isEmpty()) { + ListenableFuture allHandoffs = Futures.allAsList(pendingHandoffs); + log.info("Waiting for handoffs"); + + + if (handoffTimeout > 0) { + allHandoffs.get(handoffTimeout, TimeUnit.MILLISECONDS); + } else { + allHandoffs.get(); + } + } + } + + private void persistAndWait(AppenderatorDriver driver, Committer committer) + { + try { + final CountDownLatch persistLatch = new CountDownLatch(1); + driver.persist( + new Committer() + { + @Override + public Object getMetadata() + { + return committer.getMetadata(); + } + + @Override + public void run() + { + try { + committer.run(); + } + finally { + persistLatch.countDown(); + } + } + } + ); + persistLatch.await(); + } + catch (InterruptedException e) { + log.debug(e, "Interrupted while finishing the job"); + } + catch (Exception e) { + log.makeAlert(e, "Failed to finish realtime task").emit(); + throw e; + } + } - pendingPublishes.add(publishResultFuture); + private DiscoveryDruidNode createDiscoveryDruidNode(TaskToolbox toolbox) + { + LookupNodeService lookupNodeService = getContextValue(CTX_KEY_LOOKUP_TIER) == null ? + toolbox.getLookupNodeService() : + new LookupNodeService(getContextValue(CTX_KEY_LOOKUP_TIER)); + return new DiscoveryDruidNode( + toolbox.getDruidNode(), + DruidNodeDiscoveryProvider.NODE_TYPE_PEON, + ImmutableMap.of( + toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), + lookupNodeService.getName(), lookupNodeService + ) + ); + } - Futures.addCallback(publishFuture, new FutureCallback() + private DataSegmentAnnouncer createLockingSegmentAnnouncer(TaskToolbox toolbox) + { + // Wrap default DataSegmentAnnouncer such that we unlock intervals as we unannounce segments + final long lockTimeoutMs = getContextValue(Tasks.LOCK_TIMEOUT_KEY, Tasks.DEFAULT_LOCK_TIMEOUT); + // Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, http timeout error can occur while waiting for a + // lock to be acquired. + return new DataSegmentAnnouncer() { @Override - public void onSuccess(@Nullable SegmentsAndMetadata published) + public void announceSegment(final DataSegment segment) throws IOException { - ListenableFuture handoffFuture = driver.registerHandoff(published); - - log.info("Registering pending handoff for [%s]", published); + // Side effect: Calling announceSegment causes a lock to be acquired + Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) + ), + "Cannot acquire a lock for interval[%s]", + segment.getInterval() + ); + toolbox.getSegmentAnnouncer().announceSegment(segment); + } - pendingHandoffs.add(handoffFuture); + @Override + public void unannounceSegment(final DataSegment segment) throws IOException + { + try { + toolbox.getSegmentAnnouncer().unannounceSegment(segment); + } + finally { + toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval())); + } + } - publishResultFuture.set(published); + @Override + public void announceSegments(Iterable segments) throws IOException + { + // Side effect: Calling announceSegments causes locks to be acquired + for (DataSegment segment : segments) { + Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit( + new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) + ), + "Cannot acquire a lock for interval[%s]", + segment.getInterval() + ); + } + toolbox.getSegmentAnnouncer().announceSegments(segments); } @Override - public void onFailure(@Nullable Throwable throwable) + public void unannounceSegments(Iterable segments) throws IOException { - log.error(throwable, "Error occurred publishing segments"); - publishResultFuture.setException(throwable); + try { + toolbox.getSegmentAnnouncer().unannounceSegments(segments); + } + finally { + for (DataSegment segment : segments) { + toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval())); + } + } } - }); + }; + } + + private void logUnusedConfiguration(RealtimeTuningConfig tuningConfig) + { + warnIfNotNull(tuningConfig.getWindowPeriod(), "windowPeriod is not used by this task"); + warnIfNotNull(tuningConfig.getVersioningPolicy(), "versioningPolicy is not used by this task"); + warnIfNotNull(tuningConfig.getRejectionPolicyFactory(), "rejectionPolicyFactory is not used by this task"); + warnIfNotNull(tuningConfig.getPersistThreadPriority(), "persistThreadPriority is not used by this task"); + warnIfNotNull(tuningConfig.getMergeThreadPriority(), "mergeThreadPriority is not used by this task"); + } + + private void warnIfNotNull(Object value, String message) + { + if (value == null) { + return; + } + + log.warn(message); } private static Appenderator newAppenderator( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 044863217615..b62de640fdfb 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -312,11 +312,6 @@ public void testHandoffTimeout() throws Exception // Stop the firehose, this will drain out existing events. firehose.close(); - Collection publishedSegments = awaitSegments(); - - Assert.assertEquals(1, task.getMetrics().processed()); - Assert.assertNotNull(Iterables.getOnlyElement(publishedSegments)); - // handoff would timeout, resulting in exception statusFuture.get(); } From 764d97ec782c77315969cab7cdc86eee62978d6c Mon Sep 17 00:00:00 2001 From: Kevin Conaway Date: Fri, 2 Feb 2018 21:34:34 -0500 Subject: [PATCH 09/10] #5261 Add separate tuningConfig for RealtimeAppenderatorIndexTask. Revert changes to RealtimeTuningConfig --- .../RealtimeAppenderatorIngestionSpec.java | 39 ++++ .../RealtimeAppenderatorTuningConfig.java | 195 ++++++++++++++++++ .../AppenderatorDriverRealtimeIndexTask.java | 168 ++++----------- .../common/task/RealtimeIndexTask.java | 4 +- ...penderatorDriverRealtimeIndexTaskTest.java | 133 +++++------- .../common/task/RealtimeIndexTaskTest.java | 9 +- .../indexing/common/task/TaskSerdeTest.java | 3 +- .../indexing/overlord/TaskLifecycleTest.java | 9 +- .../indexing/RealtimeTuningConfig.java | 15 +- .../segment/realtime/RealtimeManagerTest.java | 3 - .../appenderator/AppenderatorPlumberTest.java | 1 - .../appenderator/AppenderatorTester.java | 9 +- ...DefaultOfflineAppenderatorFactoryTest.java | 1 - .../plumber/RealtimePlumberSchoolTest.java | 9 +- .../segment/realtime/plumber/SinkTest.java | 1 - .../cli/validate/DruidJsonValidatorTest.java | 1 - 16 files changed, 348 insertions(+), 252 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorIngestionSpec.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorIngestionSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorIngestionSpec.java new file mode 100644 index 000000000000..3716aef3219c --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorIngestionSpec.java @@ -0,0 +1,39 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.druid.indexing.common.index; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.IngestionSpec; +import io.druid.segment.indexing.RealtimeIOConfig; + +public class RealtimeAppenderatorIngestionSpec extends IngestionSpec +{ + + @JsonCreator + public RealtimeAppenderatorIngestionSpec( + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("ioConfig") RealtimeIOConfig ioConfig, + @JsonProperty("tuningConfig") RealtimeAppenderatorTuningConfig tuningConfig + ) + { + super(dataSchema, ioConfig, tuningConfig); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java new file mode 100644 index 000000000000..bcc3e7982bb7 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -0,0 +1,195 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.druid.indexing.common.index; + +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.io.Files; +import io.druid.segment.IndexSpec; +import io.druid.segment.indexing.TuningConfig; +import io.druid.segment.realtime.appenderator.AppenderatorConfig; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.timeline.partition.NoneShardSpec; +import io.druid.timeline.partition.ShardSpec; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.io.File; + +@JsonTypeName("appenderator") +public class RealtimeAppenderatorTuningConfig implements TuningConfig, AppenderatorConfig +{ + private static final int defaultMaxRowsInMemory = 75000; + private static final int defaultMaxRowsPerSegment = 5_000_000; + private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); + private static final int defaultMaxPendingPersists = 0; + private static final ShardSpec defaultShardSpec = NoneShardSpec.instance(); + private static final IndexSpec defaultIndexSpec = new IndexSpec(); + private static final Boolean defaultReportParseExceptions = Boolean.FALSE; + private static final long defaultPublishAndHandoffTimeout = 0; + private static final long defaultAlertTimeout = 0; + + private static File createNewBasePersistDirectory() + { + return Files.createTempDir(); + } + + private final int maxRowsInMemory; + private final int maxRowsPerSegment; + private final Period intermediatePersistPeriod; + private final File basePersistDirectory; + private final int maxPendingPersists; + private final ShardSpec shardSpec; + private final IndexSpec indexSpec; + private final boolean reportParseExceptions; + private final long publishAndHandoffTimeout; + private final long alertTimeout; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + + @JsonCreator + public RealtimeAppenderatorTuningConfig( + @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, + @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") File basePersistDirectory, + @JsonProperty("maxPendingPersists") Integer maxPendingPersists, + @JsonProperty("shardSpec") ShardSpec shardSpec, + @JsonProperty("indexSpec") IndexSpec indexSpec, + @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, + @JsonProperty("publishAndHandoffTimeout") Long publishAndHandoffTimeout, + @JsonProperty("alertTimeout") Long alertTimeout, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + ) + { + this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; + this.maxRowsPerSegment = maxRowsPerSegment == null ? defaultMaxRowsPerSegment : maxRowsPerSegment; + this.intermediatePersistPeriod = intermediatePersistPeriod == null + ? defaultIntermediatePersistPeriod + : intermediatePersistPeriod; + this.basePersistDirectory = basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory; + this.maxPendingPersists = maxPendingPersists == null ? defaultMaxPendingPersists : maxPendingPersists; + this.shardSpec = shardSpec == null ? defaultShardSpec : shardSpec; + this.indexSpec = indexSpec == null ? defaultIndexSpec : indexSpec; + this.reportParseExceptions = reportParseExceptions == null + ? defaultReportParseExceptions + : reportParseExceptions; + this.publishAndHandoffTimeout = publishAndHandoffTimeout == null + ? defaultPublishAndHandoffTimeout + : publishAndHandoffTimeout; + Preconditions.checkArgument(this.publishAndHandoffTimeout >= 0, "publishAndHandoffTimeout must be >= 0"); + + this.alertTimeout = alertTimeout == null ? defaultAlertTimeout : alertTimeout; + Preconditions.checkArgument(this.alertTimeout >= 0, "alertTimeout must be >= 0"); + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + } + + @Override + @JsonProperty + public int getMaxRowsInMemory() + { + return maxRowsInMemory; + } + + @JsonProperty + public int getMaxRowsPerSegment() + { + return maxRowsPerSegment; + } + + @Override + @JsonProperty + public Period getIntermediatePersistPeriod() + { + return intermediatePersistPeriod; + } + + @Override + @JsonProperty + public File getBasePersistDirectory() + { + return basePersistDirectory; + } + + @Override + @JsonProperty + public int getMaxPendingPersists() + { + return maxPendingPersists; + } + + @JsonProperty + public ShardSpec getShardSpec() + { + return shardSpec; + } + + @Override + @JsonProperty + public IndexSpec getIndexSpec() + { + return indexSpec; + } + + @Override + @JsonProperty + public boolean isReportParseExceptions() + { + return reportParseExceptions; + } + + @JsonProperty + public long getPublishAndHandoffTimeout() + { + return publishAndHandoffTimeout; + } + + @JsonProperty + public long getAlertTimeout() + { + return alertTimeout; + } + + @Override + @JsonProperty + @Nullable + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + + public RealtimeAppenderatorTuningConfig withBasePersistDirectory(File dir) + { + return new RealtimeAppenderatorTuningConfig( + maxRowsInMemory, + maxRowsPerSegment, + intermediatePersistPeriod, + dir, + maxPendingPersists, + shardSpec, + indexSpec, + reportParseExceptions, + publishAndHandoffTimeout, + alertTimeout, + segmentWriteOutMediumFactory + ); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 7c2bb17fffbf..e117d1138a6a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -22,11 +22,11 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.primitives.Ints; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import io.druid.data.input.Committer; @@ -38,13 +38,13 @@ import io.druid.discovery.LookupNodeService; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; -import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; -import io.druid.indexing.common.actions.LockAcquireAction; -import io.druid.indexing.common.actions.LockReleaseAction; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.index.RealtimeAppenderatorIngestionSpec; +import io.druid.indexing.common.index.RealtimeAppenderatorTuningConfig; +import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; @@ -56,7 +56,6 @@ import io.druid.query.QueryRunner; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; -import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.FireDepartment; import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.RealtimeMetricsMonitor; @@ -70,15 +69,13 @@ import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; import io.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; import io.druid.segment.realtime.plumber.Committers; -import io.druid.server.coordination.DataSegmentAnnouncer; -import io.druid.timeline.DataSegment; import org.apache.commons.io.FileUtils; import java.io.File; -import java.io.IOException; import java.util.Collections; import java.util.Map; import java.util.Queue; +import java.util.Random; import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.ConcurrentLinkedQueue; @@ -92,14 +89,25 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask private static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class); + private static final Random random = new Random(); - private static String makeDatasource(FireDepartment fireDepartment) + private static String makeTaskId(RealtimeAppenderatorIngestionSpec spec) { - return fireDepartment.getDataSchema().getDataSource(); + final StringBuilder suffix = new StringBuilder(8); + for (int i = 0; i < Ints.BYTES * 2; ++i) { + suffix.append((char) ('a' + ((random.nextInt() >>> (i * 4)) & 0x0F))); + } + return StringUtils.format( + "index_realtime_%s_%d_%s_%s", + spec.getDataSchema().getDataSource(), + spec.getTuningConfig().getShardSpec().getPartitionNum(), + DateTimes.nowUtc(), + suffix + ); } @JsonIgnore - private final FireDepartment spec; + private final RealtimeAppenderatorIngestionSpec spec; @JsonIgnore private final Queue> pendingHandoffs; @@ -126,18 +134,18 @@ private static String makeDatasource(FireDepartment fireDepartment) public AppenderatorDriverRealtimeIndexTask( @JsonProperty("id") String id, @JsonProperty("resource") TaskResource taskResource, - @JsonProperty("spec") FireDepartment fireDepartment, + @JsonProperty("spec") RealtimeAppenderatorIngestionSpec spec, @JsonProperty("context") Map context ) { super( - id == null ? RealtimeIndexTask.makeTaskId(fireDepartment) : id, - StringUtils.format("index_realtime_appenderator_%s", makeDatasource(fireDepartment)), + id == null ? makeTaskId(spec) : id, + StringUtils.format("index_realtime_appenderator_%s", spec.getDataSchema().getDataSource()), taskResource, - makeDatasource(fireDepartment), + spec.getDataSchema().getDataSource(), context ); - this.spec = fireDepartment; + this.spec = spec; this.pendingHandoffs = new ConcurrentLinkedQueue<>(); } @@ -171,7 +179,7 @@ public QueryRunner getQueryRunner(Query query) } @Override - public boolean isReady(TaskActionClient taskActionClient) throws Exception + public boolean isReady(TaskActionClient taskActionClient) { return true; } @@ -183,37 +191,30 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception setupTimeoutAlert(); - final DataSegmentAnnouncer lockingSegmentAnnouncer = createLockingSegmentAnnouncer(toolbox); - DataSchema dataSchema = spec.getDataSchema(); - RealtimeIOConfig realtimeIOConfig = spec.getIOConfig(); - RealtimeTuningConfig tuningConfig = spec.getTuningConfig() - .withBasePersistDirectory(toolbox.getPersistDir()); + RealtimeAppenderatorTuningConfig tuningConfig = spec.getTuningConfig() + .withBasePersistDirectory(toolbox.getPersistDir()); - logUnusedConfiguration(tuningConfig); - - final FireDepartment fireDepartment = new FireDepartment( - dataSchema, - realtimeIOConfig, - tuningConfig + final FireDepartment fireDepartmentForMetrics = new FireDepartment( + dataSchema, new RealtimeIOConfig(null, null, null), null ); final RealtimeMetricsMonitor metricsMonitor = new RealtimeMetricsMonitor( - ImmutableList.of(fireDepartment), + ImmutableList.of(fireDepartmentForMetrics), ImmutableMap.of( DruidMetrics.TASK_ID, new String[]{getId()} ) ); - this.metrics = fireDepartment.getMetrics(); + this.metrics = fireDepartmentForMetrics.getMetrics(); Supplier committerSupplier = null; final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); DiscoveryDruidNode discoveryDruidNode = createDiscoveryDruidNode(toolbox); - appenderator = newAppenderator(dataSchema, tuningConfig, fireDepartment.getMetrics(), toolbox, lockingSegmentAnnouncer); - AppenderatorDriver driver = newDriver(dataSchema, appenderator, toolbox, fireDepartment.getMetrics()); + appenderator = newAppenderator(dataSchema, tuningConfig, metrics, toolbox); + AppenderatorDriver driver = newDriver(dataSchema, appenderator, toolbox, metrics); try { toolbox.getDataSegmentServerAnnouncer().announce(); @@ -259,10 +260,6 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName, committerSupplier); if (addResult.isOk()) { - if (addResult.isPersistRequired()) { - driver.persist(committerSupplier.get()); - } - if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) { publishSegments(driver, publisher, committerSupplier, sequenceName); @@ -304,7 +301,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception // Publish any remaining segments publishSegments(driver, publisher, committerSupplier, sequenceName); - waitForSegmentHandoff(tuningConfig.getHandoffConditionTimeout()); + waitForSegmentPublishAndHandoff(tuningConfig.getPublishAndHandoffTimeout()); } } else if (firehose != null) { log.info("Task was gracefully stopped, will persist data before exiting"); @@ -384,7 +381,7 @@ public FireDepartmentMetrics getMetrics() } @JsonProperty("spec") - public FireDepartment getRealtimeIngestionSchema() + public RealtimeAppenderatorIngestionSpec getSpec() { return spec; } @@ -446,7 +443,7 @@ private void publishSegments( pendingHandoffs.add(handoffFuture); } - private void waitForSegmentHandoff(long handoffTimeout) throws InterruptedException, ExecutionException, + private void waitForSegmentPublishAndHandoff(long timeout) throws InterruptedException, ExecutionException, TimeoutException { if (!pendingHandoffs.isEmpty()) { @@ -454,8 +451,8 @@ private void waitForSegmentHandoff(long handoffTimeout) throws InterruptedExcept log.info("Waiting for handoffs"); - if (handoffTimeout > 0) { - allHandoffs.get(handoffTimeout, TimeUnit.MILLISECONDS); + if (timeout > 0) { + allHandoffs.get(timeout, TimeUnit.MILLISECONDS); } else { allHandoffs.get(); } @@ -513,94 +510,11 @@ private DiscoveryDruidNode createDiscoveryDruidNode(TaskToolbox toolbox) ); } - private DataSegmentAnnouncer createLockingSegmentAnnouncer(TaskToolbox toolbox) - { - // Wrap default DataSegmentAnnouncer such that we unlock intervals as we unannounce segments - final long lockTimeoutMs = getContextValue(Tasks.LOCK_TIMEOUT_KEY, Tasks.DEFAULT_LOCK_TIMEOUT); - // Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, http timeout error can occur while waiting for a - // lock to be acquired. - return new DataSegmentAnnouncer() - { - @Override - public void announceSegment(final DataSegment segment) throws IOException - { - // Side effect: Calling announceSegment causes a lock to be acquired - Preconditions.checkNotNull( - toolbox.getTaskActionClient().submit( - new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) - ), - "Cannot acquire a lock for interval[%s]", - 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 segments) throws IOException - { - // Side effect: Calling announceSegments causes locks to be acquired - for (DataSegment segment : segments) { - Preconditions.checkNotNull( - toolbox.getTaskActionClient().submit( - new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) - ), - "Cannot acquire a lock for interval[%s]", - segment.getInterval() - ); - } - toolbox.getSegmentAnnouncer().announceSegments(segments); - } - - @Override - public void unannounceSegments(Iterable segments) throws IOException - { - try { - toolbox.getSegmentAnnouncer().unannounceSegments(segments); - } - finally { - for (DataSegment segment : segments) { - toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval())); - } - } - } - }; - } - - private void logUnusedConfiguration(RealtimeTuningConfig tuningConfig) - { - warnIfNotNull(tuningConfig.getWindowPeriod(), "windowPeriod is not used by this task"); - warnIfNotNull(tuningConfig.getVersioningPolicy(), "versioningPolicy is not used by this task"); - warnIfNotNull(tuningConfig.getRejectionPolicyFactory(), "rejectionPolicyFactory is not used by this task"); - warnIfNotNull(tuningConfig.getPersistThreadPriority(), "persistThreadPriority is not used by this task"); - warnIfNotNull(tuningConfig.getMergeThreadPriority(), "mergeThreadPriority is not used by this task"); - } - - private void warnIfNotNull(Object value, String message) - { - if (value == null) { - return; - } - - log.warn(message); - } - private static Appenderator newAppenderator( final DataSchema dataSchema, - final RealtimeTuningConfig tuningConfig, + final RealtimeAppenderatorTuningConfig tuningConfig, final FireDepartmentMetrics metrics, - final TaskToolbox toolbox, - final DataSegmentAnnouncer segmentAnnouncer + final TaskToolbox toolbox ) { return Appenderators.createRealtime( @@ -612,7 +526,7 @@ private static Appenderator newAppenderator( toolbox.getIndexIO(), toolbox.getIndexMergerV9(), toolbox.getQueryRunnerFactoryConglomerate(), - segmentAnnouncer, + toolbox.getSegmentAnnouncer(), toolbox.getEmitter(), toolbox.getQueryExecutorService(), toolbox.getCache(), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index e89d6a8db7d8..a7b010c08ab5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -28,7 +28,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.primitives.Ints; -import io.druid.java.util.emitter.EmittingLogger; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; @@ -45,6 +44,7 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; +import io.druid.java.util.emitter.EmittingLogger; import io.druid.query.DruidMetrics; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -89,7 +89,7 @@ public class RealtimeIndexTask extends AbstractTask private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class); private static final Random random = new Random(); - static String makeTaskId(FireDepartment fireDepartment) + private static String makeTaskId(FireDepartment fireDepartment) { return makeTaskId( fireDepartment.getDataSchema().getDataSource(), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index b62de640fdfb..2f1097c3e30f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -52,6 +52,8 @@ import io.druid.indexing.common.actions.TaskActionToolbox; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskStorageConfig; +import io.druid.indexing.common.index.RealtimeAppenderatorIngestionSpec; +import io.druid.indexing.common.index.RealtimeAppenderatorTuningConfig; import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.HeapMemoryTaskStorage; import io.druid.indexing.overlord.SegmentPublishResult; @@ -104,15 +106,12 @@ import io.druid.segment.TestHelper; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; -import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.SegmentLoaderLocalCacheManager; import io.druid.segment.loading.StorageLocationConfig; -import io.druid.segment.realtime.FireDepartment; import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; -import io.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory; import io.druid.segment.transform.ExpressionTransform; import io.druid.segment.transform.TransformSpec; import io.druid.server.DruidNode; @@ -258,9 +257,13 @@ public Firehose connect(InputRowParser parser, File temporaryDirectory) throws P private Collection publishedSegments; private CountDownLatch segmentLatch; private CountDownLatch handoffLatch; + private TaskStorage taskStorage; + private TaskLockbox taskLockbox; + private TaskToolboxFactory taskToolboxFactory; + private File baseDir; @Before - public void setUp() + public void setUp() throws IOException { EmittingLogger.registerEmitter(emitter); emitter.start(); @@ -272,6 +275,9 @@ public void setUp() derbyConnector.createTaskTables(); derbyConnector.createSegmentTable(); derbyConnector.createPendingSegmentsTable(); + + baseDir = tempFolder.newFolder(); + makeToolboxFactory(baseDir); } @After @@ -293,8 +299,7 @@ public void testHandoffTimeout() throws Exception { expectPublishedSegments(1); final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, true, 100L); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); + final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -321,8 +326,7 @@ public void testBasics() throws Exception { expectPublishedSegments(1); final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); + final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -385,8 +389,7 @@ public void testLateData() throws Exception { expectPublishedSegments(1); final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); + final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -452,8 +455,7 @@ public void testMaxRowsPerSegment() throws Exception expectPublishedSegments(2); final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); + final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -525,8 +527,7 @@ public void testTransformSpec() throws Exception ) ); final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, transformSpec, true, 0); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); + final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -591,8 +592,7 @@ public void testReportParseExceptionsOnBadMetric() throws Exception { expectPublishedSegments(0); final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, true); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); + final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -642,8 +642,7 @@ public void testNoReportParseExceptions() throws Exception expectPublishedSegments(1); final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, false); - final TaskToolbox taskToolbox = makeToolbox(task, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); + final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. while (task.getFirehose() == null) { @@ -715,14 +714,12 @@ public void testRestore() throws Exception { expectPublishedSegments(0); - final File directory = tempFolder.newFolder(); final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); final DataSegment publishedSegment; // First run: { - final TaskToolbox taskToolbox = makeToolbox(task1, directory); - final ListenableFuture statusFuture = runTask(task1, taskToolbox); + final ListenableFuture statusFuture = runTask(task1); // Wait for firehose to show up, it starts off null. while (task1.getFirehose() == null) { @@ -752,8 +749,7 @@ public void testRestore() throws Exception { expectPublishedSegments(1); final AppenderatorDriverRealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, directory); - final ListenableFuture statusFuture = runTask(task2, taskToolbox); + final ListenableFuture statusFuture = runTask(task2); // Wait for firehose to show up, it starts off null. while (task2.getFirehose() == null) { @@ -807,16 +803,13 @@ public void testRestore() throws Exception @Test(timeout = 60_000L) public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception { - final TaskStorage taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); - final File directory = tempFolder.newFolder(); final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); final DataSegment publishedSegment; // First run: { expectPublishedSegments(1); - final TaskToolbox taskToolbox = makeToolbox(task1, taskStorage, directory); - final ListenableFuture statusFuture = runTask(task1, taskToolbox); + final ListenableFuture statusFuture = runTask(task1); // Wait for firehose to show up, it starts off null. while (task1.getFirehose() == null) { @@ -854,8 +847,7 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception { expectPublishedSegments(1); final AppenderatorDriverRealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, taskStorage, directory); - final ListenableFuture statusFuture = runTask(task2, taskToolbox); + final ListenableFuture statusFuture = runTask(task2); // Wait for firehose to show up, it starts off null. while (task2.getFirehose() == null) { @@ -894,15 +886,13 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception @Test(timeout = 60_000L) public void testRestoreCorruptData() throws Exception { - final File directory = tempFolder.newFolder(); final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); // First run: { expectPublishedSegments(0); - final TaskToolbox taskToolbox = makeToolbox(task1, directory); - final ListenableFuture statusFuture = runTask(task1, taskToolbox); + final ListenableFuture statusFuture = runTask(task1); // Wait for firehose to show up, it starts off null. while (task1.getFirehose() == null) { @@ -928,7 +918,7 @@ public void testRestoreCorruptData() throws Exception Assert.assertTrue(publishedSegments.isEmpty()); } - Optional optional = FileUtils.listFiles(directory, null, true).stream() + Optional optional = FileUtils.listFiles(baseDir, null, true).stream() .filter(f -> f.getName().equals("00000.smoosh")) .findFirst(); @@ -944,8 +934,7 @@ public void testRestoreCorruptData() throws Exception expectPublishedSegments(0); final AppenderatorDriverRealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, directory); - final ListenableFuture statusFuture = runTask(task2, taskToolbox); + final ListenableFuture statusFuture = runTask(task2); // Wait for the task to finish. boolean caught = false; @@ -964,20 +953,26 @@ public void testStopBeforeStarting() throws Exception { expectPublishedSegments(0); - final File directory = tempFolder.newFolder(); final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); task1.stopGracefully(); - final TaskToolbox taskToolbox = makeToolbox(task1, directory); - final ListenableFuture statusFuture = runTask(task1, taskToolbox); + final ListenableFuture statusFuture = runTask(task1); // Wait for the task to finish. final TaskStatus taskStatus = statusFuture.get(); Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); } - private ListenableFuture runTask(final Task task, final TaskToolbox toolbox) + private ListenableFuture runTask(final Task task) { + try { + taskStorage.insert(task, TaskStatus.running(task.getId())); + } + catch (EntryExistsException e) { + // suppress + } + taskLockbox.syncFromStorage(); + final TaskToolbox toolbox = taskToolboxFactory.build(task); return taskExec.submit( new Callable() { @@ -1043,20 +1038,14 @@ private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( null, null ); - RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( + RealtimeAppenderatorTuningConfig tuningConfig = new RealtimeAppenderatorTuningConfig( 1000, 1000, - new Period("P1Y"), - new Period("PT10M"), null, null, - new ServerTimeRejectionPolicyFactory(), null, null, null, - true, - 0, - 0, reportParseExceptions, handoffTimeout, null, @@ -1065,7 +1054,7 @@ private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( return new AppenderatorDriverRealtimeIndexTask( taskId, null, - new FireDepartment(dataSchema, realtimeIOConfig, realtimeTuningConfig), + new RealtimeAppenderatorIngestionSpec(dataSchema, realtimeIOConfig, tuningConfig), null ) { @@ -1077,18 +1066,6 @@ protected boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory) }; } - private TaskToolbox makeToolbox( - final Task task, - final File directory - ) - { - return makeToolbox( - task, - new HeapMemoryTaskStorage(new TaskStorageConfig(null)), - directory - ); - } - private void expectPublishedSegments(int count) { segmentLatch = new CountDownLatch(count); @@ -1113,18 +1090,12 @@ private void awaitHandoffs() throws InterruptedException ); } - private TaskToolbox makeToolbox( - final Task task, - final TaskStorage taskStorage, - final File directory - ) + private void makeToolboxFactory(final File directory) { - publishedSegments = new CopyOnWriteArrayList<>(); + taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); + taskLockbox = new TaskLockbox(taskStorage); - Assert.assertFalse( - "Segment latch not initialized, did you forget to call expectPublishSegments?", - segmentLatch == null - ); + publishedSegments = new CopyOnWriteArrayList<>(); ObjectMapper mapper = new DefaultObjectMapper(); mapper.registerSubtypes(LinearShardSpec.class); @@ -1140,6 +1111,11 @@ public Set announceHistoricalSegments(Set segments) th { Set result = super.announceHistoricalSegments(segments); + Assert.assertFalse( + "Segment latch not initialized, did you forget to call expectPublishSegments?", + segmentLatch == null + ); + publishedSegments.addAll(result); segments.forEach(s -> segmentLatch.countDown()); @@ -1153,6 +1129,11 @@ public SegmentPublishResult announceHistoricalSegments( { SegmentPublishResult result = super.announceHistoricalSegments(segments, startMetadata, endMetadata); + Assert.assertFalse( + "Segment latch not initialized, did you forget to call expectPublishSegments?", + segmentLatch == null + ); + publishedSegments.addAll(result.getSegments()); result.getSegments().forEach(s -> segmentLatch.countDown()); @@ -1160,14 +1141,7 @@ public SegmentPublishResult announceHistoricalSegments( } }; final TaskConfig taskConfig = new TaskConfig(directory.getPath(), null, null, 50000, null, false, null, null); - final TaskLockbox taskLockbox = new TaskLockbox(taskStorage); - try { - taskStorage.insert(task, TaskStatus.running(task.getId())); - } - catch (EntryExistsException e) { - // suppress - } - taskLockbox.syncFromStorage(); + final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( taskLockbox, mdc, @@ -1255,7 +1229,8 @@ public List getLocations() return Lists.newArrayList(); } }; - final TaskToolboxFactory toolboxFactory = new TaskToolboxFactory( + + taskToolboxFactory = new TaskToolboxFactory( taskConfig, taskActionClientFactory, emitter, @@ -1282,8 +1257,6 @@ public List getLocations() new LookupNodeService("tier"), new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0) ); - - return toolboxFactory.build(task); } public long sumMetric(final Task task, final DimFilter filter, final String metric) throws Exception diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 7d7b82b47b8b..9f050fdcab47 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -30,10 +30,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import io.druid.java.util.emitter.EmittingLogger; -import io.druid.java.util.emitter.core.NoopEmitter; -import io.druid.java.util.emitter.service.ServiceEmitter; -import io.druid.java.util.metrics.MonitorScheduler; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.MapCache; import io.druid.data.input.Firehose; @@ -77,6 +73,10 @@ import io.druid.java.util.common.jackson.JacksonUtils; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.ParseException; +import io.druid.java.util.emitter.EmittingLogger; +import io.druid.java.util.emitter.core.NoopEmitter; +import io.druid.java.util.emitter.service.ServiceEmitter; +import io.druid.java.util.metrics.MonitorScheduler; import io.druid.math.expr.ExprMacroTable; import io.druid.metadata.EntryExistsException; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; @@ -916,7 +916,6 @@ private RealtimeIndexTask makeRealtimeTask( null ); RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( - 1000, 1000, new Period("P1Y"), new Period("PT10M"), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index bcd3631ab10e..578da275e082 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -35,7 +35,6 @@ import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; -import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -52,6 +51,7 @@ import io.druid.segment.realtime.firehose.LocalFirehoseFactory; import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.PlumberSchool; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.hamcrest.CoreMatchers; @@ -495,7 +495,6 @@ public Plumber findPlumber( ), new RealtimeTuningConfig( - 1, 1, new Period("PT10M"), null, diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 40c575964716..a3f8aae5f548 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -34,10 +34,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.util.concurrent.MoreExecutors; -import io.druid.java.util.emitter.EmittingLogger; -import io.druid.java.util.emitter.service.ServiceEmitter; -import io.druid.java.util.metrics.Monitor; -import io.druid.java.util.metrics.MonitorScheduler; import io.druid.client.cache.MapCache; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; @@ -82,6 +78,10 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Comparators; +import io.druid.java.util.emitter.EmittingLogger; +import io.druid.java.util.emitter.service.ServiceEmitter; +import io.druid.java.util.metrics.Monitor; +import io.druid.java.util.metrics.MonitorScheduler; import io.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import io.druid.metadata.TestDerbyConnector; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -1202,7 +1202,6 @@ private RealtimeIndexTask newRealtimeIndexTask() null ); RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( - 1000, 1000, new Period("P1Y"), null, //default window period of 10 minutes diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 984468a6d7b4..c9cdf6358aed 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -23,13 +23,13 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.io.Files; -import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import io.druid.segment.realtime.appenderator.AppenderatorConfig; import io.druid.segment.realtime.plumber.IntervalStartVersioningPolicy; import io.druid.segment.realtime.plumber.RejectionPolicyFactory; import io.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory; import io.druid.segment.realtime.plumber.VersioningPolicy; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.ShardSpec; import org.joda.time.Period; @@ -42,7 +42,6 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig { private static final int defaultMaxRowsInMemory = 75000; - private static final int defaultMaxRowsPerSegment = 5_000_000; private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final Period defaultWindowPeriod = new Period("PT10M"); private static final VersioningPolicy defaultVersioningPolicy = new IntervalStartVersioningPolicy(); @@ -64,7 +63,6 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File { return new RealtimeTuningConfig( defaultMaxRowsInMemory, - defaultMaxRowsPerSegment, defaultIntermediatePersistPeriod, defaultWindowPeriod, basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory, @@ -84,7 +82,6 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File } private final int maxRowsInMemory; - private final int maxRowsPerSegment; private final Period intermediatePersistPeriod; private final Period windowPeriod; private final File basePersistDirectory; @@ -104,7 +101,6 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File @JsonCreator public RealtimeTuningConfig( @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, - @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -124,7 +120,6 @@ public RealtimeTuningConfig( ) { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; - this.maxRowsPerSegment = maxRowsPerSegment == null ? defaultMaxRowsPerSegment : maxRowsPerSegment; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaultIntermediatePersistPeriod : intermediatePersistPeriod; @@ -159,12 +154,6 @@ public int getMaxRowsInMemory() return maxRowsInMemory; } - @JsonProperty - public int getMaxRowsPerSegment() - { - return maxRowsPerSegment; - } - @Override @JsonProperty public Period getIntermediatePersistPeriod() @@ -270,7 +259,6 @@ public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) { return new RealtimeTuningConfig( maxRowsInMemory, - maxRowsPerSegment, intermediatePersistPeriod, windowPeriod, basePersistDirectory, @@ -293,7 +281,6 @@ public RealtimeTuningConfig withBasePersistDirectory(File dir) { return new RealtimeTuningConfig( maxRowsInMemory, - maxRowsPerSegment, intermediatePersistPeriod, windowPeriod, dir, diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 6309fb246aaf..59f145555e85 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -199,7 +199,6 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException } ); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( - 1, 1, new Period("P1Y"), null, @@ -259,7 +258,6 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException ); tuningConfig_0 = new RealtimeTuningConfig( - 1, 1, new Period("P1Y"), null, @@ -279,7 +277,6 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException ); tuningConfig_1 = new RealtimeTuningConfig( - 1, 1, new Period("P1Y"), null, diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java index 6968dd69aa06..d7b0aa4a6477 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java @@ -67,7 +67,6 @@ EasyMock. anyObject(), EasyMock. anyObject())).andReturn(true).anyTimes(); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( - 1, 1, null, null, diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index 74d848e452c3..2fc9cd08c374 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -21,9 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; -import io.druid.java.util.emitter.EmittingLogger; -import io.druid.java.util.emitter.core.NoopEmitter; -import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.MapCache; import io.druid.data.input.impl.DimensionsSpec; @@ -33,7 +30,9 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; -import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.java.util.emitter.EmittingLogger; +import io.druid.java.util.emitter.core.NoopEmitter; +import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; @@ -55,6 +54,7 @@ import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; @@ -136,7 +136,6 @@ public AppenderatorTester( maxRowsInMemory, null, null, - null, basePersistDirectory, null, null, diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index b837bf5ee453..08157eaf7f61 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -136,7 +136,6 @@ public int columnCacheSizeBytes() 75000, null, null, - null, temporaryFolder.newFolder(), null, null, diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index cff7b2d3f310..76c3a1f2b797 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -26,7 +26,6 @@ import com.google.common.collect.Maps; import com.google.common.io.Files; import com.google.common.util.concurrent.MoreExecutors; -import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.client.cache.MapCache; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; @@ -39,9 +38,7 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; -import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import io.druid.segment.writeout.SegmentWriteOutMediumFactory; -import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; @@ -59,6 +56,9 @@ import io.druid.segment.realtime.FireDepartmentTest; import io.druid.segment.realtime.FireHydrant; import io.druid.segment.realtime.SegmentPublisher; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; import org.apache.commons.io.FileUtils; @@ -196,7 +196,6 @@ public void setUp() throws Exception EasyMock.replay(announcer, segmentPublisher, dataSegmentPusher, handoffNotifierFactory, handoffNotifier, emitter); tuningConfig = new RealtimeTuningConfig( - 1, 1, null, null, diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index 87af620e01b6..dce3e7a89780 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -60,7 +60,6 @@ public void testSwap() throws Exception final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( - 100, 100, new Period("P1Y"), null, diff --git a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java index 64d14e7a7649..49518d6483dd 100644 --- a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java @@ -168,7 +168,6 @@ public Plumber findPlumber( ), new RealtimeTuningConfig( - 1, 1, new Period("PT10M"), null, From cba0c4ee027d440bd5aa7bcb5a96c0569480a79f Mon Sep 17 00:00:00 2001 From: Kevin Conaway Date: Mon, 5 Feb 2018 19:36:55 -0500 Subject: [PATCH 10/10] #5261 Change JSON type to realtime_appenderator to keep the same naming pattern as RealtimeIndexTask --- .../indexing/common/index/RealtimeAppenderatorTuningConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index bcc3e7982bb7..a7084f69d436 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -34,7 +34,7 @@ import javax.annotation.Nullable; import java.io.File; -@JsonTypeName("appenderator") +@JsonTypeName("realtime_appenderator") public class RealtimeAppenderatorTuningConfig implements TuningConfig, AppenderatorConfig { private static final int defaultMaxRowsInMemory = 75000;