From d35083075a5e4a05c1bb49b9f74955d4819423fc Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Thu, 24 Aug 2017 17:08:18 -0500 Subject: [PATCH 01/14] Kafka Index Task that supports Incremental handoffs - Incrementally handoff segments when they hit maxRowsPerSegment limit - Decouple segment partitioning from Kafka partitioning, all records from consumed partitions go to a single druid segment - Support for restoring task on middle manager restarts by check pointing end offsets for segments --- .../extensions-core/kafka-ingestion.md | 2 +- .../druid/indexing/kafka/KafkaIndexTask.java | 826 ++++++++++++++---- .../indexing/kafka/KafkaIndexTaskClient.java | 30 +- .../indexing/kafka/KafkaTuningConfig.java | 6 +- .../kafka/supervisor/KafkaSupervisor.java | 624 ++++++++++--- .../KafkaSupervisorTuningConfig.java | 4 +- .../kafka/KafkaIndexTaskClientTest.java | 14 +- .../indexing/kafka/KafkaIndexTaskTest.java | 24 +- .../indexing/kafka/KafkaTuningConfigTest.java | 4 +- .../kafka/supervisor/KafkaSupervisorTest.java | 160 +++- .../KafkaSupervisorTuningConfigTest.java | 2 +- .../ActionBasedSegmentAllocator.java | 6 +- .../CheckPointDataSourceMetadataAction.java | 106 +++ .../common/actions/SegmentAllocateAction.java | 33 +- .../indexing/common/actions/TaskAction.java | 3 +- .../druid/indexing/common/task/IndexTask.java | 4 +- .../supervisor/SupervisorManager.java | 25 + .../actions/SegmentAllocateActionTest.java | 6 +- ...TestIndexerMetadataStorageCoordinator.java | 3 +- .../IndexerMetadataStorageCoordinator.java | 5 +- .../supervisor/NoopSupervisorSpec.java | 12 + .../overlord/supervisor/Supervisor.java | 18 + .../IndexerSQLMetadataStorageCoordinator.java | 52 +- .../realtime/appenderator/Appenderator.java | 17 +- .../appenderator/AppenderatorDriver.java | 287 ++++-- .../AppenderatorDriverMetadata.java | 22 +- .../appenderator/AppenderatorImpl.java | 133 ++- .../realtime/appenderator/Committed.java | 8 + .../appenderator/SegmentAllocator.java | 3 +- .../AppenderatorDriverFailTest.java | 6 + .../appenderator/AppenderatorDriverTest.java | 26 +- 31 files changed, 1941 insertions(+), 530 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/actions/CheckPointDataSourceMetadataAction.java diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 5a8d661d648a..f8c64ef823af 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -121,7 +121,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| |`indexSpec`|Object|Tune how data is indexed, see 'IndexSpec' below for more details.|no| |`reportParseExceptions`|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)| -|`handoffConditionTimeout`|Long|Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever. This option is deprecated. Use `completionTimeout` of KafkaSupervisorIOConfig instead.|no (default == 0)| +|`handoffConditionTimeout`|Long|Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever.|no (default == 0)| |`resetOffsetAutomatically`|Boolean|Whether to reset the consumer offset if the next offset that it is trying to fetch is less than the earliest available offset for that particular partition. The consumer offset will be reset to either the earliest or latest offset depending on `useEarliestOffset` property of `KafkaSupervisorIOConfig` (see below). This situation typically occurs when messages in Kafka are no longer available for consumption and therefore won't be ingested into Druid. If set to false then ingestion for that particular partition will halt and manual intervention is required to correct the situation, please see `Reset Supervisor` API below.|no (default == false)| |`workerThreads`|Integer|The number of threads that will be used by the supervisor for asynchronous operations.|no (default == min(10, taskCount))| |`chatThreads`|Integer|The number of threads that will be used for communicating with indexing tasks.|no (default == min(10, taskCount * replicas))| diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index fdbb42c93e5d..0088103f9490 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -23,9 +23,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Preconditions; @@ -33,11 +32,15 @@ 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.collect.Maps; import com.google.common.collect.Sets; import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import com.google.common.util.concurrent.Futures; +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 io.druid.data.input.Committer; import io.druid.data.input.InputRow; @@ -49,6 +52,7 @@ import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.CheckPointDataSourceMetadataAction; import io.druid.indexing.common.actions.ResetDataSourceMetadataAction; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskActionClient; @@ -58,6 +62,7 @@ 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.concurrent.Execs; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.parsers.ParseException; import io.druid.query.DruidMetrics; @@ -106,20 +111,33 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Random; import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Future; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; public class KafkaIndexTask extends AbstractTask implements ChatHandler { @@ -131,7 +149,7 @@ public enum Status STARTING, READING, PAUSED, - PUBLISHING + FINISHING } private static final EmittingLogger log = new EmittingLogger(KafkaIndexTask.class); @@ -140,6 +158,7 @@ public enum Status private static final long POLL_TIMEOUT = 100; private static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; + private static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions"; private final DataSchema dataSchema; private final InputRowParser parser; @@ -150,16 +169,20 @@ public enum Status private final Map endOffsets = new ConcurrentHashMap<>(); private final Map nextOffsets = new ConcurrentHashMap<>(); + private final Map maxEndOffsets = new HashMap<>(); + private final Map lastPersistedOffsets = new ConcurrentHashMap<>(); - private ObjectMapper mapper; + private TaskToolbox toolbox; private volatile Appenderator appenderator = null; + private volatile AppenderatorDriver driver = null; private volatile FireDepartmentMetrics fireDepartmentMetrics = null; private volatile DateTime startTime; private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread) private volatile Thread runThread = null; - private volatile boolean stopRequested = false; - private volatile boolean publishOnStop = false; + private volatile File sequencesPersistFile = null; + private final AtomicBoolean stopRequested = new AtomicBoolean(false); + private final AtomicBoolean publishOnStop = new AtomicBoolean(false); // The pause lock and associated conditions are to support coordination between the Jetty threads and the main // ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully @@ -202,6 +225,16 @@ public enum Status // This value can be tuned in some tests private long pollRetryMs = 30000; + private final Set publishingSequences = Sets.newConcurrentHashSet(); + private final BlockingQueue publishQueue = new LinkedBlockingQueue<>(); + private final List> handOffWaitList = new ArrayList<>(); + private final CountDownLatch waitForPublishes = new CountDownLatch(1); + private final AtomicReference throwableAtomicReference = new AtomicReference<>(); + private final ListeningExecutorService publishExecService; + private final String topic; + + private volatile CopyOnWriteArrayList sequences; + @JsonCreator public KafkaIndexTask( @JsonProperty("id") String id, @@ -228,8 +261,16 @@ public KafkaIndexTask( this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); this.authorizerMapper = authorizerMapper; - this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionOffsetMap()); + this.maxEndOffsets.putAll(endOffsets.entrySet() + .stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + integerLongEntry -> Long.MAX_VALUE + ))); + this.topic = ioConfig.getStartPartitions().getTopic(); + this.sequences = new CopyOnWriteArrayList<>(); + this.publishExecService = MoreExecutors.listeningDecorator(Execs.singleThreaded("publish-driver")); } @VisibleForTesting @@ -277,13 +318,124 @@ public KafkaIOConfig getIOConfig() return ioConfig; } + private void startExecutors() + { + // start publish executor service + publishExecService.submit( + (Runnable) () -> { + while (true) { + try { + final SequenceMetadata sequenceMetadata = publishQueue.take(); + + Preconditions.checkNotNull(driver); + + if (sequenceMetadata.isSentinel()) { + waitForPublishes.countDown(); + break; + } + + log.info("Publishing segments for sequence [%s]", sequenceMetadata); + + final SegmentsAndMetadata result = driver.publish( + sequenceMetadata.getPublisher(toolbox, ioConfig.isUseTransaction()), + sequenceMetadata.getCommitterSupplier(topic, lastPersistedOffsets).get(), + ImmutableList.of(sequenceMetadata.getSequenceName()) + ).get(); + + if (result == null) { + throw new ISE( + "Transaction failure publishing segments for sequence [%s]", + sequenceMetadata + ); + } else { + log.info( + "Published segments[%s] with metadata[%s].", + Joiner.on(", ").join( + result.getSegments().stream().map(DataSegment::getIdentifier).collect(Collectors.toList()) + ), + result.getCommitMetadata() + ); + } + + sequences.remove(sequenceMetadata); + publishingSequences.remove(sequenceMetadata.getSequenceName()); + try { + persistState(toolbox); + } + catch (IOException e) { + log.error(e, "Unable to persist state, dying"); + Throwables.propagate(e); + } + + final ListenableFuture handOffFuture = driver.registerHandoff(result); + handOffWaitList.add(handOffFuture); + } + catch (Throwable t) { + if ((t instanceof InterruptedException || (t instanceof RejectedExecutionException + && t.getCause() instanceof InterruptedException))) { + log.warn("Stopping publish thread as we are interrupted, probably we are shutting down"); + } else { + log.makeAlert(t, "Error in publish thread, dying").emit(); + throwableAtomicReference.set(t); + } + Futures.allAsList(handOffWaitList).cancel(true); + waitForPublishes.countDown(); + break; + } + } + } + ); + } + @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { log.info("Starting up!"); + startTime = DateTimes.nowUtc(); - mapper = toolbox.getObjectMapper(); status = Status.STARTING; + this.toolbox = toolbox; + + if (getContext() != null && getContext().get("checkpoints") != null) { + log.info("Got checkpoints [%s]", (String) getContext().get("checkpoints")); + final TreeMap> checkpoints = toolbox.getObjectMapper().readValue( + (String) getContext().get("checkpoints"), new TypeReference>>() + { + }); + + Iterator>> sequenceOffsets = checkpoints.entrySet().iterator(); + Map.Entry> previous = sequenceOffsets.next(); + while (sequenceOffsets.hasNext()) { + Map.Entry> current = sequenceOffsets.next(); + sequences.add(new SequenceMetadata( + previous.getKey(), + StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), + previous.getValue(), + current.getValue(), + true + )); + previous = current; + } + sequences.add(new SequenceMetadata( + previous.getKey(), + StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), + previous.getValue(), + maxEndOffsets, + false + )); + } else { + sequences.add(new SequenceMetadata( + 0, + StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0), + ioConfig.getStartPartitions().getPartitionOffsetMap(), + maxEndOffsets, + false + )); + } + + sequencesPersistFile = new File(toolbox.getTaskWorkDir(), "sequences.json"); + + restoreState(); if (chatHandlerProvider.isPresent()) { log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); @@ -321,21 +473,32 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ); try ( - final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox); - final AppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics); final KafkaConsumer consumer = newConsumer() ) { toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); - appenderator = appenderator0; + appenderator = newAppenderator(fireDepartmentMetrics, toolbox); + driver = newDriver(appenderator, toolbox, fireDepartmentMetrics); + startExecutors(); final String topic = ioConfig.getStartPartitions().getTopic(); // Start up, set up initial offsets. final Object restoredMetadata = driver.startJob(); if (restoredMetadata == null) { - nextOffsets.putAll(ioConfig.getStartPartitions().getPartitionOffsetMap()); + // Either all sequences are published and we were waiting for handoff + // or no persist has happened so far + Preconditions.checkState(sequences.size() == 0 || sequences.get(0).startOffsets.entrySet().stream().allMatch( + partitionOffsetEntry -> Longs.compare( + partitionOffsetEntry.getValue(), + ioConfig.getStartPartitions() + .getPartitionOffsetMap() + .get(partitionOffsetEntry.getKey()) + ) >= 0 + ), "Sequence offsets are not compatible with start offsets of task"); + log.info("Setting next offsets to [%s]", sequences.size() == 0 ? endOffsets : sequences.get(0).startOffsets); + nextOffsets.putAll(sequences.size() == 0 ? endOffsets : sequences.get(0).startOffsets); } else { final Map restoredMetadataMap = (Map) restoredMetadata; final KafkaPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( @@ -362,42 +525,36 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception } } - // Set up sequenceNames. - final Map sequenceNames = Maps.newHashMap(); - for (Integer partitionNum : nextOffsets.keySet()) { - sequenceNames.put(partitionNum, StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), partitionNum)); - } - // Set up committer. - final Supplier committerSupplier = new Supplier() - { - @Override - public Committer get() - { - final Map snapshot = ImmutableMap.copyOf(nextOffsets); + final Supplier committerSupplier = () -> { + final Map snapshot = ImmutableMap.copyOf(nextOffsets); + lastPersistedOffsets.clear(); + lastPersistedOffsets.putAll(snapshot); - return new Committer() + return new Committer() + { + @Override + public Object getMetadata() { - @Override - public Object getMetadata() - { - return ImmutableMap.of( - METADATA_NEXT_PARTITIONS, new KafkaPartitions( - ioConfig.getStartPartitions().getTopic(), - snapshot - ) - ); - } + return ImmutableMap.of( + METADATA_NEXT_PARTITIONS, new KafkaPartitions( + ioConfig.getStartPartitions().getTopic(), + snapshot + ) + ); + } - @Override - public void run() - { - // Do nothing. - } - }; - } + @Override + public void run() + { + // Do nothing. + } + }; }; + // restart publishing of sequences (if any) + persistAndPublishSequences(committerSupplier); + Set assignment = assignPartitionsAndSeekToNext(consumer, topic); // Main loop. @@ -413,15 +570,27 @@ public void run() if (assignment.isEmpty()) { log.info("All partitions have been fully read"); - publishOnStop = true; - stopRequested = true; + publishOnStop.set(true); + stopRequested.set(true); } } - if (stopRequested) { + // if stop is requested or task's end offset is set by call to setEndOffsets method with finish set to true + if (stopRequested.get() || (sequences.get(sequences.size() - 1).isCheckpointed() + && !ioConfig.isPauseAfterRead())) { + status = Status.FINISHING; + } + + if (stopRequested.get()) { break; } + checkAndMaybeThrowException(); + + if (!ioConfig.isPauseAfterRead()) { + persistAndPublishSequences(committerSupplier); + } + // The retrying business is because the KafkaConsumer throws OffsetOutOfRangeException if the seeked-to // offset is not present in the topic-partition. This can happen if we're asking a task to read from data // that has not been written yet (which is totally legitimate). So let's wait for it to show up. @@ -435,6 +604,7 @@ public void run() stillReading = ioConfig.isPauseAfterRead() || !assignment.isEmpty(); } + SequenceMetadata sequenceToCheckpoint = null; for (ConsumerRecord record : records) { if (log.isTraceEnabled()) { log.trace( @@ -469,18 +639,43 @@ public void run() final InputRow row = valueBytes == null ? null : parser.parse(ByteBuffer.wrap(valueBytes)); if (row != null && withinMinMaxRecordTime(row)) { - final String sequenceName = sequenceNames.get(record.partition()); + SequenceMetadata sequenceToUse = null; + for (SequenceMetadata sequence : sequences) { + if (sequence.canHandle(record)) { + sequenceToUse = sequence; + } + } + + if (sequenceToUse == null) { + throw new ISE( + "WTH?! cannot find any valid sequence for record with partition [%d] and offset [%d]. Current sequences: [%s]", + record.partition(), + record.offset(), + sequences + ); + } + final AppenderatorDriverAddResult addResult = driver.add( row, - sequenceName, - committerSupplier + sequenceToUse.getSequenceName(), + committerSupplier, + true + // skip segment lineage check as there will always be one segment + // for combination of sequence and segment granularity. + // It is necessary to skip it as the task puts messages polled from all the + // assigned Kafka partitions into a single Druid segment, thus ordering of + // messages among replica tasks across assigned partitions is not guaranteed + // which may cause replica tasks to ask for segments with different interval + // in different order which might cause SegmentAllocateAction to fail. ); if (addResult.isOk()) { // If the number of rows in the segment exceeds the threshold after adding a row, // move the segment out from the active segments of AppenderatorDriver to make a new segment. if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) { - driver.moveSegmentOut(sequenceName, ImmutableList.of(addResult.getSegmentIdentifier())); + if (!sequenceToUse.isCheckpointed()) { + sequenceToCheckpoint = sequenceToUse; + } } } else { // Failure to allocate segment puts determinism at risk, bail out to be safe. @@ -519,87 +714,82 @@ public void run() stillReading = ioConfig.isPauseAfterRead() || !assignment.isEmpty(); } } + + if (sequenceToCheckpoint != null && !ioConfig.isPauseAfterRead()) { + Preconditions.checkArgument( + sequences.get(sequences.size() - 1) + .getSequenceName() + .equals(sequenceToCheckpoint.getSequenceName()), + "Cannot checkpoint a sequence [%s] which is not the latest one, sequences [%s]", + sequenceToCheckpoint, + sequences + ); + pause(-1L); + if (!toolbox.getTaskActionClient().submit(new CheckPointDataSourceMetadataAction( + getDataSource(), + ioConfig.getBaseSequenceName(), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, sequenceToCheckpoint.getStartOffsets())), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, nextOffsets)) + ))) { + throw new ISE("Checkpoint request with offsets [%s] failed, dying", nextOffsets); + } + } } } finally { + log.info("Persisting all pending data"); driver.persist(committerSupplier.get()); // persist pending data } synchronized (statusLock) { - if (stopRequested && !publishOnStop) { + if (stopRequested.get() && !publishOnStop.get()) { throw new InterruptedException("Stopping without publishing"); } - status = Status.PUBLISHING; + status = Status.FINISHING; } - final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { - final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue( - ((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS), - KafkaPartitions.class - ); - - // Sanity check, we should only be publishing things that match our desired end state. - if (!endOffsets.equals(finalPartitions.getPartitionOffsetMap())) { - throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); - } - - final SegmentTransactionalInsertAction action; - - if (ioConfig.isUseTransaction()) { - action = new SegmentTransactionalInsertAction( - segments, - new KafkaDataSourceMetadata(ioConfig.getStartPartitions()), - new KafkaDataSourceMetadata(finalPartitions) - ); - } else { - action = new SegmentTransactionalInsertAction(segments, null, null); + for (SequenceMetadata sequenceMetadata : sequences) { + if (!publishingSequences.contains(sequenceMetadata.getSequenceName())) { + // this is done to prevent checks in sequence specific commit supplier from failing + sequenceMetadata.setEndOffsets(nextOffsets); + sequenceMetadata.updateAssignments(nextOffsets); + publishingSequences.add(sequenceMetadata.getSequenceName()); + // persist already done in finally, so directly add to publishQueue + publishQueue.add(sequenceMetadata); } + } - log.info("Publishing with isTransaction[%s].", ioConfig.isUseTransaction()); - - return toolbox.getTaskActionClient().submit(action).isSuccess(); - }; - - // Supervised kafka tasks are killed by KafkaSupervisor if they are stuck during publishing segments or waiting - // for hand off. See KafkaSupervisorIOConfig.completionTimeout. - final SegmentsAndMetadata published = driver.publish( - publisher, - committerSupplier.get(), - sequenceNames.values() - ).get(); + // add Sentinel SequenceMetadata to indicate end of all sequences + publishQueue.add(SequenceMetadata.getSentinelSequenceMetadata()); + waitForPublishes.await(); + checkAndMaybeThrowException(); - final Future handoffFuture = driver.registerHandoff(published); - final SegmentsAndMetadata handedOff; + final List handedOffList; if (tuningConfig.getHandoffConditionTimeout() == 0) { - handedOff = handoffFuture.get(); + handedOffList = Futures.allAsList(handOffWaitList).get(); } else { - handedOff = handoffFuture.get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); + handedOffList = Futures.allAsList(handOffWaitList) + .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); } - if (handedOff == null) { - throw new ISE("Transaction failure publishing segments, aborting"); - } else { - log.info( - "Published segments[%s] with metadata[%s].", - Joiner.on(", ").join( - Iterables.transform( - handedOff.getSegments(), - new Function() - { - @Override - public String apply(DataSegment input) - { - return input.getIdentifier(); - } - } - ) - ), - handedOff.getCommitMetadata() - ); + for (SegmentsAndMetadata handedOff : handedOffList) { + if (handedOff == null) { + log.warn("Handoff failed for segments [%s]", handedOff.getSegments()); + } else { + log.info( + "Handoff completed for segments[%s] with metadata[%s].", + Joiner.on(", ").join( + handedOff.getSegments().stream().map(DataSegment::getIdentifier).collect(Collectors.toList()) + ), + handedOff.getCommitMetadata() + ); + } } } catch (InterruptedException | RejectedExecutionException e) { + publishExecService.shutdownNow(); + appenderator.closeNow(); // handle the InterruptedException that gets wrapped in a RejectedExecutionException if (e instanceof RejectedExecutionException && (e.getCause() == null || !(e.getCause() instanceof InterruptedException))) { @@ -607,7 +797,7 @@ public String apply(DataSegment input) } // if we were interrupted because we were asked to stop, handle the exception and return success, else rethrow - if (!stopRequested) { + if (!stopRequested.get()) { Thread.currentThread().interrupt(); throw e; } @@ -615,6 +805,16 @@ public String apply(DataSegment input) log.info("The task was asked to stop before completing"); } finally { + if (appenderator != null) { + if (throwableAtomicReference.get() != null) { + appenderator.closeNow(); + } else { + appenderator.close(); + } + } + if (driver != null) { + driver.close(); + } if (chatHandlerProvider.isPresent()) { chatHandlerProvider.get().unregister(getId()); } @@ -626,6 +826,60 @@ public String apply(DataSegment input) return success(); } + private void checkAndMaybeThrowException() + { + if (throwableAtomicReference.get() != null) { + Throwables.propagate(throwableAtomicReference.get()); + } + } + + private void persistAndPublishSequences(Supplier committerSupplier) + throws ExecutionException, InterruptedException + { + for (SequenceMetadata sequenceMetadata : sequences) { + sequenceMetadata.updateAssignments(nextOffsets); + if (!sequenceMetadata.isOpen() && !publishingSequences.contains(sequenceMetadata.getSequenceName())) { + publishingSequences.add(sequenceMetadata.getSequenceName()); + try { + Object result = driver.persist(committerSupplier.get()); + log.info("Persist completed with results [%s]", result); + log.info("Adding sequence to publish queue, [%s]", sequenceMetadata); + publishQueue.add(sequenceMetadata); + } + catch (InterruptedException e) { + log.warn("Interrupted while persisting sequence [%s]", sequenceMetadata); + throw e; + } + } + } + } + + private void restoreState() throws IOException + { + Preconditions.checkNotNull(sequencesPersistFile); + if (sequencesPersistFile.exists()) { + sequences = new CopyOnWriteArrayList<>(toolbox.getObjectMapper().>readValue( + sequencesPersistFile, new TypeReference>() + { + })); + log.info("Restored sequences [%s]", sequences); + } + if (sequences.get(sequences.size() - 1).isCheckpointed()) { + this.endOffsets.putAll(sequences.get(sequences.size() - 1).getEndOffsets()); + log.info("End offsets changed, sequences [%s]", sequences); + } + } + + private synchronized void persistState(final TaskToolbox toolbox) throws IOException + { + log.info("Persisting Sequences Metadata [%s]", sequences); + toolbox.getObjectMapper().writerWithType( + new TypeReference>() + { + } + ).writeValue(sequencesPersistFile, sequences); + } + @Override public boolean canRestore() { @@ -652,14 +906,20 @@ private Access authorizationCheck(final HttpServletRequest req, Action action) return access; } + // used for unit tests + Appenderator getAppenderator() + { + return appenderator; + } + @Override public void stopGracefully() { log.info("Stopping gracefully (status: [%s])", status); - stopRequested = true; + stopRequested.set(true); synchronized (statusLock) { - if (status == Status.PUBLISHING) { + if (status == Status.FINISHING) { runThread.interrupt(); return; } @@ -775,16 +1035,19 @@ public Map getEndOffsets() public Response setEndOffsetsHTTP( Map offsets, @QueryParam("resume") @DefaultValue("false") final boolean resume, + @QueryParam("finish") @DefaultValue("true") final boolean finish, + // this field is only for internal purposes, shouldn't be usually set by users @Context final HttpServletRequest req ) throws InterruptedException { authorizationCheck(req, Action.WRITE); - return setEndOffsets(offsets, resume); + return setEndOffsets(offsets, resume, finish); } public Response setEndOffsets( Map offsets, - final boolean resume + final boolean resume, + final boolean finish // this field is only for internal purposes, shouldn't be usually set by users ) throws InterruptedException { if (offsets == null) { @@ -800,35 +1063,74 @@ public Response setEndOffsets( ) ) .build(); - } - - pauseLock.lockInterruptibly(); - try { - if (!isPaused()) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("Task must be paused before changing the end offsets") - .build(); - } - - for (Map.Entry entry : offsets.entrySet()) { - if (entry.getValue().compareTo(nextOffsets.get(entry.getKey())) < 0) { + } else { + try { + pauseLock.lockInterruptibly(); + // Perform all sequence related checks before checking for isPaused() + // and after acquiring pauseLock to correctly guard against duplicate requests + if (sequences.size() == 0) { return Response.status(Response.Status.BAD_REQUEST) - .entity( - StringUtils.format( - "End offset must be >= current offset for partition [%s] (current: %s)", - entry.getKey(), - nextOffsets.get(entry.getKey()) - ) - ) + .entity("No Sequences found to set end offsets") + .build(); + } else if ((sequences.get(sequences.size() - 1).getStartOffsets().equals(offsets) && !finish) || + (sequences.get(sequences.size() - 1).getEndOffsets().equals(offsets) && finish)) { + log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequences); + } else if (sequences.get(sequences.size() - 1).isCheckpointed() && !ioConfig.isPauseAfterRead()) { + return Response.status(Response.Status.BAD_REQUEST) + .entity(StringUtils.format( + "WTH?! Sequence [%s] has already endOffsets set, cannot set to [%s]", + sequences.get(sequences.size() - 1), + offsets + )).build(); + } else if (!isPaused()) { + return Response.status(Response.Status.BAD_REQUEST) + .entity("Task must be paused before changing the end offsets") .build(); } - } - endOffsets.putAll(offsets); - log.info("endOffsets changed to %s", endOffsets); - } - finally { - pauseLock.unlock(); + for (Map.Entry entry : offsets.entrySet()) { + if (entry.getValue().compareTo(nextOffsets.get(entry.getKey())) < 0) { + return Response.status(Response.Status.BAD_REQUEST) + .entity( + StringUtils.format( + "End offset must be >= current offset for partition [%s] (current: %s)", + entry.getKey(), + nextOffsets.get(entry.getKey()) + ) + ) + .build(); + } + } + + final SequenceMetadata sequenceMetadata = sequences.get(sequences.size() - 1); + sequenceMetadata.setEndOffsets(offsets); + + if (finish) { + log.info("Updating endOffsets from [%s] to [%s]", endOffsets, offsets); + endOffsets.putAll(offsets); + } else { + Preconditions.checkState(!ioConfig.isPauseAfterRead()); + // create new sequence + final SequenceMetadata newSequenceMetadata = new SequenceMetadata( + sequenceMetadata.getSequenceId() + 1, + StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), sequenceMetadata.getSequenceId() + 1), + offsets, + maxEndOffsets, + false + ); + sequences.add(newSequenceMetadata); + } + + persistState(toolbox); + } + catch (Exception e) { + log.error(e, "Unable to set end offsets, dying"); + throwableAtomicReference.set(e); + Throwables.propagate(e); + } + finally { + pauseLock.unlock(); + } } if (resume) { @@ -838,6 +1140,24 @@ public Response setEndOffsets( return Response.ok(endOffsets).build(); } + @GET + @Path("/checkpoints") + @Produces(MediaType.APPLICATION_JSON) + public Map> getCheckpointsHTTP(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return getCheckpoints(); + } + + public Map> getCheckpoints() + { + TreeMap> result = new TreeMap<>(); + result.putAll( + sequences.stream().collect(Collectors.toMap(SequenceMetadata::getSequenceId, SequenceMetadata::getStartOffsets)) + ); + return result; + } + /** * Signals the ingestion loop to pause. * @@ -899,7 +1219,7 @@ public Response pause(final long timeout) throws InterruptedException } try { - return Response.ok().entity(mapper.writeValueAsString(getCurrentOffsets())).build(); + return Response.ok().entity(toolbox.getObjectMapper().writeValueAsString(getCurrentOffsets())).build(); } catch (JsonProcessingException e) { throw Throwables.propagate(e); @@ -1022,17 +1342,7 @@ private static void assignPartitions( { consumer.assign( Lists.newArrayList( - Iterables.transform( - partitions, - new Function() - { - @Override - public TopicPartition apply(Integer n) - { - return new TopicPartition(topic, n); - } - } - ) + partitions.stream().map(n -> new TopicPartition(topic, n)).collect(Collectors.toList()) ) ); } @@ -1162,7 +1472,7 @@ private void possiblyResetOffsetsOrWait( pollRetryLock.lockInterruptibly(); try { long nanos = TimeUnit.MILLISECONDS.toNanos(pollRetryMs); - while (nanos > 0L && !pauseRequested && !stopRequested) { + while (nanos > 0L && !pauseRequested && !stopRequested.get()) { nanos = isAwaitingRetry.awaitNanos(nanos); } } @@ -1231,4 +1541,214 @@ private boolean withinMinMaxRecordTime(final InputRow row) return !beforeMinimumMessageTime && !afterMaximumMessageTime; } + + private static class SequenceMetadata + { + private final int sequenceId; + private final String sequenceName; + private final Map startOffsets; + private final Map endOffsets; + private final Set assignments; + private final boolean sentinel; + private volatile boolean checkpointed; + + @JsonCreator + public SequenceMetadata( + @JsonProperty("sequenceId") int sequenceId, + @JsonProperty("sequenceName") String sequenceName, + @JsonProperty("startOffsets") Map startOffsets, + @JsonProperty("endOffsets") Map endOffsets, + @JsonProperty("checkpointed") boolean checkpointed + ) + { + Preconditions.checkNotNull(sequenceName); + Preconditions.checkNotNull(startOffsets); + Preconditions.checkNotNull(endOffsets); + this.sequenceId = sequenceId; + this.sequenceName = sequenceName; + this.startOffsets = startOffsets; + this.endOffsets = Maps.newHashMap(endOffsets); + this.assignments = Sets.newHashSet(startOffsets.keySet()); + this.checkpointed = checkpointed; + this.sentinel = false; + } + + @JsonProperty + public int getSequenceId() + { + return sequenceId; + } + + @JsonProperty + public boolean isCheckpointed() + { + return checkpointed; + } + + @JsonProperty + public String getSequenceName() + { + return sequenceName; + } + + @JsonProperty + public Map getStartOffsets() + { + return startOffsets; + } + + @JsonProperty + public Map getEndOffsets() + { + return endOffsets; + } + + @JsonProperty + public boolean isSentinel() + { + return sentinel; + } + + public void setCheckpointed() + { + checkpointed = true; + } + + public void setEndOffsets(Map newEndOffsets) + { + endOffsets.putAll(newEndOffsets); + setCheckpointed(); + } + + public void updateAssignments(Map nextPartitionOffset) + { + assignments.clear(); + nextPartitionOffset.entrySet().forEach(partitionOffset -> { + if (Longs.compare(endOffsets.get(partitionOffset.getKey()), nextPartitionOffset.get(partitionOffset.getKey())) + > 0) { + assignments.add(partitionOffset.getKey()); + } + }); + } + + public boolean isOpen() + { + return !assignments.isEmpty(); + } + + boolean canHandle(ConsumerRecord record) + { + return isOpen() + && endOffsets.get(record.partition()) != null + && record.offset() >= startOffsets.get(record.partition()) + && record.offset() < endOffsets.get(record.partition()); + } + + private SequenceMetadata() + { + this.sequenceId = -1; + this.sequenceName = null; + this.startOffsets = null; + this.endOffsets = null; + this.assignments = null; + this.checkpointed = true; + this.sentinel = true; + } + + public static SequenceMetadata getSentinelSequenceMetadata() + { + return new SequenceMetadata(); + } + + @Override + public String toString() + { + return "SequenceMetadata{" + + "sequenceName='" + sequenceName + '\'' + + ", sequenceId=" + sequenceId + + ", startOffsets=" + startOffsets + + ", endOffsets=" + endOffsets + + ", assignments=" + assignments + + ", sentinel=" + sentinel + + ", checkpointed=" + checkpointed + + '}'; + } + + + public Supplier getCommitterSupplier(String topic, Map lastPersistedOffsets) + { + // Set up committer. + return () -> + new Committer() + { + @Override + public Object getMetadata() + { + Preconditions.checkState( + assignments.isEmpty(), + "This committer can be used only once all the records till offsets [%s] have been consumed, also make sure to call updateAssignments before using this committer", + endOffsets + ); + + // merge endOffsets for this sequence with globally lastPersistedOffsets + // This is done because this committer would be persisting only sub set of segments + // corresponding to the current sequence. Generally, lastPersistedOffsets should already + // cover endOffsets but just to be sure take max of offsets and persist that + for (Map.Entry partitionOffset : endOffsets.entrySet()) { + lastPersistedOffsets.put(partitionOffset.getKey(), Math.max( + partitionOffset.getValue(), + lastPersistedOffsets.getOrDefault(partitionOffset.getKey(), 0L) + )); + } + + // Publish metadata can be different from persist metadata as we are giong to publish only + // subset of segments + return ImmutableMap.of(METADATA_NEXT_PARTITIONS, new KafkaPartitions(topic, lastPersistedOffsets), + METADATA_PUBLISH_PARTITIONS, new KafkaPartitions(topic, endOffsets) + ); + } + + @Override + public void run() + { + // Do nothing. + } + }; + } + + public TransactionalSegmentPublisher getPublisher(TaskToolbox toolbox, boolean useTransaction) + { + return (segments, commitMetadata) -> { + final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue( + ((Map) commitMetadata).get(METADATA_PUBLISH_PARTITIONS), + KafkaPartitions.class + ); + + // Sanity check, we should only be publishing things that match our desired end state. + if (!getEndOffsets().equals(finalPartitions.getPartitionOffsetMap())) { + throw new ISE( + "WTF?! Driver for sequence [%s], attempted to publish invalid metadata[%s].", + toString(), + commitMetadata + ); + } + + final SegmentTransactionalInsertAction action; + + if (useTransaction) { + action = new SegmentTransactionalInsertAction( + segments, + new KafkaDataSourceMetadata(new KafkaPartitions(finalPartitions.getTopic(), getStartOffsets())), + new KafkaDataSourceMetadata(finalPartitions) + ); + } else { + action = new SegmentTransactionalInsertAction(segments, null, null); + } + + log.info("Publishing with isTransaction[%s].", useTransaction); + + return toolbox.getTaskActionClient().submit(action).isSuccess(); + }; + } + } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java index c0ba9f444355..1d557719a6d8 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -58,6 +58,7 @@ import java.net.Socket; import java.net.URI; import java.util.Map; +import java.util.TreeMap; import java.util.concurrent.Callable; public class KafkaIndexTaskClient @@ -270,6 +271,21 @@ public Map getCurrentOffsets(final String id, final boolean retry } } + public TreeMap> getCheckpoints(final String id, final boolean retry) + { + log.debug("GetCheckpoints task[%s] retry[%s]", id, retry); + try { + final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "checkpoints", null, retry); + return jsonMapper.readValue(response.getContent(), new TypeReference>>() {}); + } + catch (NoTaskLocationException e) { + return new TreeMap<>(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + public Map getEndOffsets(final String id) { log.debug("GetEndOffsets task[%s]", id); @@ -290,19 +306,19 @@ public Map getEndOffsets(final String id) public boolean setEndOffsets(final String id, final Map endOffsets) { - return setEndOffsets(id, endOffsets, false); + return setEndOffsets(id, endOffsets, false, true); } - public boolean setEndOffsets(final String id, final Map endOffsets, final boolean resume) + public boolean setEndOffsets(final String id, final Map endOffsets, final boolean resume, final boolean finalize) { - log.debug("SetEndOffsets task[%s] endOffsets[%s] resume[%s]", id, endOffsets, resume); + log.debug("SetEndOffsets task[%s] endOffsets[%s] resume[%s] finalize[%s]", id, endOffsets, resume, finalize); try { final FullResponseHolder response = submitRequest( id, HttpMethod.POST, "offsets/end", - resume ? "resume=true" : null, + StringUtils.format("resume=%s&finish=%s", resume, finalize), jsonMapper.writeValueAsBytes(endOffsets), true ); @@ -421,11 +437,11 @@ public Map call() throws Exception public ListenableFuture setEndOffsetsAsync(final String id, final Map endOffsets) { - return setEndOffsetsAsync(id, endOffsets, false); + return setEndOffsetsAsync(id, endOffsets, false, true); } public ListenableFuture setEndOffsetsAsync( - final String id, final Map endOffsets, final boolean resume + final String id, final Map endOffsets, final boolean resume, final boolean finalize ) { return executorService.submit( @@ -434,7 +450,7 @@ public ListenableFuture setEndOffsetsAsync( @Override public Boolean call() throws Exception { - return setEndOffsets(id, endOffsets, resume); + return setEndOffsets(id, endOffsets, resume, finalize); } } ); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index a1792c92a5e2..6da2ef6a34ce 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -38,10 +38,10 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig private final int maxRowsPerSegment; private final Period intermediatePersistPeriod; private final File basePersistDirectory; + @Deprecated private final int maxPendingPersists; private final IndexSpec indexSpec; private final boolean reportParseExceptions; - @Deprecated private final long handoffConditionTimeout; private final boolean resetOffsetAutomatically; @@ -69,7 +69,7 @@ public KafkaTuningConfig( ? defaults.getIntermediatePersistPeriod() : intermediatePersistPeriod; this.basePersistDirectory = defaults.getBasePersistDirectory(); - this.maxPendingPersists = maxPendingPersists == null ? defaults.getMaxPendingPersists() : maxPendingPersists; + this.maxPendingPersists = 0; this.indexSpec = indexSpec == null ? defaults.getIndexSpec() : indexSpec; this.reportParseExceptions = reportParseExceptions == null ? defaults.isReportParseExceptions() @@ -127,6 +127,7 @@ public File getBasePersistDirectory() @Override @JsonProperty + @Deprecated public int getMaxPendingPersists() { return maxPendingPersists; @@ -156,7 +157,6 @@ public boolean isReportParseExceptions() return reportParseExceptions; } - @Deprecated @JsonProperty public long getHandoffConditionTimeout() { diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 302077fc15a1..9c74c501685e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -20,6 +20,7 @@ package io.druid.indexing.kafka.supervisor; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; @@ -31,10 +32,12 @@ 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.Maps; import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -42,7 +45,6 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import io.druid.java.util.common.concurrent.Execs; import io.druid.indexing.common.TaskInfoProvider; import io.druid.indexing.common.TaskLocation; import io.druid.indexing.common.TaskStatus; @@ -68,7 +70,9 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.IAE; 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.metadata.EntryExistsException; import io.druid.server.metrics.DruidMonitorSchedulerConfig; import org.apache.commons.codec.digest.DigestUtils; @@ -80,14 +84,18 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Random; import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; @@ -97,6 +105,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -118,6 +127,7 @@ public class KafkaSupervisor implements Supervisor private static final long MINIMUM_GET_OFFSET_PERIOD_MILLIS = 5000; private static final long INITIAL_GET_OFFSET_DELAY_MILLIS = 15000; private static final long INITIAL_EMIT_LAG_METRIC_DELAY_MILLIS = 25000; + private static final CopyOnWriteArrayList EMPTY_LIST = Lists.newCopyOnWriteArrayList(); // Internal data structures // -------------------------------------------------------- @@ -143,12 +153,24 @@ private static class TaskGroup final Optional minimumMessageTime; final Optional maximumMessageTime; DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action + final TreeMap> sequenceOffsets = new TreeMap<>(); - public TaskGroup(ImmutableMap partitionOffsets, Optional minimumMessageTime, Optional maximumMessageTime) + public TaskGroup( + ImmutableMap partitionOffsets, + Optional minimumMessageTime, + Optional maximumMessageTime + ) { this.partitionOffsets = partitionOffsets; this.minimumMessageTime = minimumMessageTime; this.maximumMessageTime = maximumMessageTime; + this.sequenceOffsets.put(0, partitionOffsets); + } + + public int addNewCheckpoint(Map checkpoint) + { + sequenceOffsets.put(sequenceOffsets.lastKey() + 1, checkpoint); + return sequenceOffsets.lastKey(); } Set taskIds() @@ -186,6 +208,9 @@ private static class TaskData private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); // -------------------------------------------------------- + // BaseSequenceName -> TaskGroup + private final ConcurrentHashMap sequenceTaskGroup = new ConcurrentHashMap<>(); + private final TaskStorage taskStorage; private final TaskMaster taskMaster; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; @@ -464,6 +489,32 @@ public void reset(DataSourceMetadata dataSourceMetadata) notices.add(new ResetNotice(dataSourceMetadata)); } + @Override + public void checkpoint( + String sequenceName, + DataSourceMetadata previousCheckpoint, + DataSourceMetadata currentCheckpoint + ) + { + Preconditions.checkNotNull(sequenceName, "Cannot checkpoint without a sequence name"); + Preconditions.checkNotNull(currentCheckpoint, "current checkpoint cannot be null"); + Preconditions.checkArgument( + ioConfig.getTopic() + .equals(((KafkaDataSourceMetadata) currentCheckpoint).getKafkaPartitions() + .getTopic()), + "Supervisor topic [%s] and topic in checkpoint [%s] does not match", + ioConfig.getTopic(), + ((KafkaDataSourceMetadata) currentCheckpoint).getKafkaPartitions().getTopic() + ); + + log.info("Checkpointing [%s] for sequence [%s]", currentCheckpoint, sequenceName); + notices.add(new CheckpointNotice( + sequenceName, + (KafkaDataSourceMetadata) previousCheckpoint, + (KafkaDataSourceMetadata) currentCheckpoint + )); + } + public void possiblyRegisterListener() { // getTaskRunner() sometimes fails if the task queue is still being initialized so retry later until we succeed @@ -503,13 +554,13 @@ public void statusChanged(String taskId, TaskStatus status) private interface Notice { - void handle() throws ExecutionException, InterruptedException, TimeoutException; + void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException; } private class RunNotice implements Notice { @Override - public void handle() throws ExecutionException, InterruptedException, TimeoutException + public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { long nowTime = System.currentTimeMillis(); if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { @@ -557,11 +608,83 @@ private class ResetNotice implements Notice @Override public void handle() { - log.makeAlert("Resetting dataSource [%s]", dataSource).emit(); resetInternal(dataSourceMetadata); } } + private class CheckpointNotice implements Notice + { + final String sequenceName; + final KafkaDataSourceMetadata previousCheckpoint; + final KafkaDataSourceMetadata currentCheckpoint; + + CheckpointNotice( + String sequenceName, + KafkaDataSourceMetadata previousCheckpoint, + KafkaDataSourceMetadata currentCheckpoint + ) + { + this.sequenceName = sequenceName; + this.previousCheckpoint = previousCheckpoint; + this.currentCheckpoint = currentCheckpoint; + } + + @Override + public void handle() throws ExecutionException, InterruptedException, TimeoutException + { + // check for consistency + // if already received request for this sequenceName and dataSourceMetadata combination then return + + Preconditions.checkNotNull( + sequenceTaskGroup.get(sequenceName), + "WTH?! cannot find task group for this sequence [%s], sequencesTaskGroup map [%s], taskGroups [%s]", + sequenceName, + sequenceTaskGroup, + taskGroups + ); + final TreeMap> checkpoints = sequenceTaskGroup.get(sequenceName).sequenceOffsets; + + // check validity of previousCheckpoint if it is not null + if (previousCheckpoint != null) { + int index = checkpoints.size(); + for (int sequenceId : checkpoints.descendingKeySet()) { + Map checkpoint = checkpoints.get(sequenceId); + if (checkpoint.equals(previousCheckpoint.getKafkaPartitions().getPartitionOffsetMap())) { + break; + } + index--; + } + if (index == 0) { + throw new ISE("No such previous checkpoint [%s] found", previousCheckpoint); + } else if (index < checkpoints.size()) { + // if the found checkpoint is not the latest one then already checkpointed by a replica + Preconditions.checkState(index == checkpoints.size() - 1, "checkpoint consistency failure"); + log.info("Already checkpointed with offsets [%s]", checkpoints.lastEntry().getValue()); + return; + } + } else { + // There cannot be more than one checkpoint when previous checkpoint is null + // as when the task starts they are sent existing checkpoints + Preconditions.checkState( + checkpoints.size() <= 1, + "Got checkpoint request with null as previous check point, however found more than one checkpoints in metadata store" + ); + if (checkpoints.size() == 1) { + log.info("Already checkpointed with dataSourceMetadata [%s]", checkpoints.get(0)); + return; + } + } + final int taskGroupId = getTaskGroupIdForPartition(currentCheckpoint.getKafkaPartitions() + .getPartitionOffsetMap() + .keySet() + .iterator() + .next()); + final Map newCheckpoint = checkpointTaskGroup(taskGroupId, false).get(); + sequenceTaskGroup.get(sequenceName).addNewCheckpoint(newCheckpoint); + log.info("Handled checkpoint notice, new checkpoint is [%s] for sequence [%s]", newCheckpoint, sequenceName); + } + } + @VisibleForTesting void resetInternal(DataSourceMetadata dataSourceMetadata) { @@ -569,7 +692,10 @@ void resetInternal(DataSourceMetadata dataSourceMetadata) // Reset everything boolean result = indexerMetadataStorageCoordinator.deleteDataSourceMetadata(dataSource); log.info("Reset dataSource[%s] - dataSource metadata entry deleted? [%s]", dataSource, result); - killTaskGroupForPartitions(taskGroups.keySet()); + taskGroups.values().forEach(this::killTasksInGroup); + taskGroups.clear(); + partitionGroups.clear(); + sequenceTaskGroup.clear(); } else if (!(dataSourceMetadata instanceof KafkaDataSourceMetadata)) { throw new IAE("Expected KafkaDataSourceMetadata but found instance of [%s]", dataSourceMetadata.getClass()); } else { @@ -608,6 +734,7 @@ void resetInternal(DataSourceMetadata dataSourceMetadata) } if (!doReset) { + log.info("Ignoring duplicate reset request [%s]", dataSourceMetadata); return; } @@ -625,7 +752,13 @@ void resetInternal(DataSourceMetadata dataSourceMetadata) } } if (metadataUpdateSuccess) { - killTaskGroupForPartitions(resetKafkaMetadata.getKafkaPartitions().getPartitionOffsetMap().keySet()); + resetKafkaMetadata.getKafkaPartitions().getPartitionOffsetMap().keySet().forEach(partition -> { + final int groupId = getTaskGroupIdForPartition(partition); + killTaskGroupForPartitions(ImmutableSet.of(partition)); + sequenceTaskGroup.remove(generateSequenceName(groupId)); + taskGroups.remove(groupId); + partitionGroups.get(groupId).replaceAll((partitionId, offset) -> NOT_SET); + }); } else { throw new ISE("Unable to reset metadata"); } @@ -642,16 +775,18 @@ void resetInternal(DataSourceMetadata dataSourceMetadata) private void killTaskGroupForPartitions(Set partitions) { for (Integer partition : partitions) { - TaskGroup taskGroup = taskGroups.get(getTaskGroupIdForPartition(partition)); - if (taskGroup != null) { - // kill all tasks in this task group - for (String taskId : taskGroup.tasks.keySet()) { - log.info("Reset dataSource[%s] - killing task [%s]", dataSource, taskId); - killTask(taskId); - } + killTasksInGroup(taskGroups.get(getTaskGroupIdForPartition(partition))); + } + } + + private void killTasksInGroup(TaskGroup taskGroup) + { + if (taskGroup != null) { + // kill all tasks in this task group + for (String taskId : taskGroup.tasks.keySet()) { + log.info("Kill task [%s] in the task group", taskId); + killTask(taskId); } - partitionGroups.remove(getTaskGroupIdForPartition(partition)); - taskGroups.remove(getTaskGroupIdForPartition(partition)); } } @@ -676,7 +811,7 @@ void gracefulShutdownInternal() throws ExecutionException, InterruptedException, } @VisibleForTesting - void runInternal() throws ExecutionException, InterruptedException, TimeoutException + void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { possiblyRegisterListener(); updatePartitionDataFromKafka(); @@ -694,19 +829,19 @@ void runInternal() throws ExecutionException, InterruptedException, TimeoutExcep } } - @VisibleForTesting - String generateSequenceName(int groupId) + String generateSequenceName( + Map startPartitions, + Optional minimumMessageTime, + Optional maximumMessageTime + ) { StringBuilder sb = new StringBuilder(); - Map startPartitions = taskGroups.get(groupId).partitionOffsets; for (Map.Entry entry : startPartitions.entrySet()) { sb.append(StringUtils.format("+%d(%d)", entry.getKey(), entry.getValue())); } String partitionOffsetStr = sb.toString().substring(1); - Optional minimumMessageTime = taskGroups.get(groupId).minimumMessageTime; - Optional maximumMessageTime = taskGroups.get(groupId).maximumMessageTime; String minMsgTimeStr = (minimumMessageTime.isPresent() ? String.valueOf(minimumMessageTime.get().getMillis()) : ""); String maxMsgTimeStr = (maximumMessageTime.isPresent() ? String.valueOf(maximumMessageTime.get().getMillis()) : ""); @@ -719,12 +854,26 @@ String generateSequenceName(int groupId) throw Throwables.propagate(e); } - String hashCode = DigestUtils.sha1Hex(dataSchema + tuningConfig + partitionOffsetStr + minMsgTimeStr + maxMsgTimeStr) + String hashCode = DigestUtils.sha1Hex(dataSchema + + tuningConfig + + partitionOffsetStr + + minMsgTimeStr + + maxMsgTimeStr) .substring(0, 15); return Joiner.on("_").join("index_kafka", dataSource, hashCode); } + @VisibleForTesting + String generateSequenceName(int groupId) + { + return generateSequenceName( + taskGroups.get(groupId).partitionOffsets, + taskGroups.get(groupId).minimumMessageTime, + taskGroups.get(groupId).maximumMessageTime + ); + } + private static String getRandomId() { final StringBuilder suffix = new StringBuilder(8); @@ -813,6 +962,7 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti List futureTaskIds = Lists.newArrayList(); List> futures = Lists.newArrayList(); List tasks = taskStorage.getActiveTasks(); + final Set taskGroupsToVerify = new HashSet<>(); for (Task task : tasks) { if (!(task instanceof KafkaIndexTask) || !dataSource.equals(task.getDataSource())) { @@ -847,44 +997,63 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti @Override public Boolean apply(KafkaIndexTask.Status status) { - if (status == KafkaIndexTask.Status.PUBLISHING) { - addDiscoveredTaskToPendingCompletionTaskGroups( - taskGroupId, - taskId, - kafkaTask.getIOConfig() - .getStartPartitions() - .getPartitionOffsetMap() - ); + try { + log.debug("Task [%s], status [%s]", taskId, status); + if (status == KafkaIndexTask.Status.FINISHING) { + kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap().keySet().forEach( + partition -> addDiscoveredTaskToPendingCompletionTaskGroups( + getTaskGroupIdForPartition(partition), + taskId, + kafkaTask.getIOConfig() + .getStartPartitions() + .getPartitionOffsetMap() + ) + ); - // update partitionGroups with the publishing task's offsets (if they are greater than what is - // existing) so that the next tasks will start reading from where this task left off - Map publishingTaskCurrentOffsets = taskClient.getCurrentOffsets(taskId, true); + // update partitionGroups with the publishing task's offsets (if they are greater than what is + // existing) so that the next tasks will start reading from where this task left off + Map publishingTaskEndOffsets = taskClient.getEndOffsets(taskId); - for (Map.Entry entry : publishingTaskCurrentOffsets.entrySet()) { - Integer partition = entry.getKey(); - Long offset = entry.getValue(); - ConcurrentHashMap partitionOffsets = partitionGroups.get( - getTaskGroupIdForPartition(partition) - ); + for (Map.Entry entry : publishingTaskEndOffsets.entrySet()) { + Integer partition = entry.getKey(); + Long offset = entry.getValue(); + ConcurrentHashMap partitionOffsets = partitionGroups.get( + getTaskGroupIdForPartition(partition) + ); - boolean succeeded; - do { - succeeded = true; - Long previousOffset = partitionOffsets.putIfAbsent(partition, offset); - if (previousOffset != null && previousOffset < offset) { - succeeded = partitionOffsets.replace(partition, previousOffset, offset); + boolean succeeded; + do { + succeeded = true; + Long previousOffset = partitionOffsets.putIfAbsent(partition, offset); + if (previousOffset != null && previousOffset < offset) { + succeeded = partitionOffsets.replace(partition, previousOffset, offset); + } + } while (!succeeded); + } + } else { + for (Integer partition : kafkaTask.getIOConfig() + .getStartPartitions() + .getPartitionOffsetMap() + .keySet()) { + if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { + log.warn( + "Stopping task [%s] which does not match the expected partition allocation", + taskId + ); + try { + stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + catch (InterruptedException | ExecutionException | TimeoutException e) { + log.warn(e, "Exception while stopping task"); + } + return false; } - } while (!succeeded); - } - - } else { - for (Integer partition : kafkaTask.getIOConfig() - .getStartPartitions() - .getPartitionOffsetMap() - .keySet()) { - if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { - log.warn( - "Stopping task [%s] which does not match the expected partition allocation", + } + // make sure the task's io and tuning configs match with the supervisor config + // if it is current then only create corresponding taskGroup if it does not exist + if (!isTaskCurrent(taskGroupId, taskId)) { + log.info( + "Stopping task [%s] which does not match the expected parameters and ingestion spec", taskId ); try { @@ -894,41 +1063,31 @@ public Boolean apply(KafkaIndexTask.Status status) log.warn(e, "Exception while stopping task"); } return false; + } else { + if (taskGroups.putIfAbsent( + taskGroupId, + new TaskGroup( + ImmutableMap.copyOf( + kafkaTask.getIOConfig() + .getStartPartitions() + .getPartitionOffsetMap() + ), kafkaTask.getIOConfig().getMinimumMessageTime(), + kafkaTask.getIOConfig().getMaximumMessageTime() + ) + ) == null) { + sequenceTaskGroup.put(generateSequenceName(taskGroupId), taskGroups.get(taskGroupId)); + log.info("Created new task group [%d]", taskGroupId); + } + taskGroupsToVerify.add(taskGroupId); + taskGroups.get(taskGroupId).tasks.putIfAbsent(taskId, new TaskData()); } } - - if (taskGroups.putIfAbsent( - taskGroupId, - new TaskGroup( - ImmutableMap.copyOf( - kafkaTask.getIOConfig() - .getStartPartitions() - .getPartitionOffsetMap() - ), - kafkaTask.getIOConfig().getMinimumMessageTime(), - kafkaTask.getIOConfig().getMaximumMessageTime() - ) - ) == null) { - log.debug("Created new task group [%d]", taskGroupId); - } - - if (!isTaskCurrent(taskGroupId, taskId)) { - log.info( - "Stopping task [%s] which does not match the expected parameters and ingestion spec", - taskId - ); - try { - stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - catch (InterruptedException | ExecutionException | TimeoutException e) { - log.warn(e, "Exception while stopping task"); - } - return false; - } else { - taskGroups.get(taskGroupId).tasks.putIfAbsent(taskId, new TaskData()); - } + return true; + } + catch (Throwable t) { + log.error(t, "Something bad while discovering task [%s]", taskId); + return null; } - return true; } }, workerExec ) @@ -946,6 +1105,133 @@ public Boolean apply(KafkaIndexTask.Status status) } } log.debug("Found [%d] Kafka indexing tasks for dataSource [%s]", taskCount, dataSource); + + // make sure the checkpoints are consistent with each other and with the metadata store + taskGroupsToVerify.forEach(this::verifyAndMergeCheckpoints); + } + + /** + * This method does two things - + * 1. Makes sure the checkpoints information in the taskGroup is consistent with that of the tasks, if not kill + * inconsistent tasks. + * 2. truncates the checkpoints in the taskGroup corresponding to which segments have been published, so that any newly + * created tasks for the taskGroup, they start indexing from after the latest published offsets. + */ + private void verifyAndMergeCheckpoints(final Integer groupId) + { + final TaskGroup taskGroup = taskGroups.get(groupId); + + // List {SequenceId, Checkpoints}> + final List>>> taskSequences = new ArrayList<>(); + + for (String taskId : taskGroup.taskIds()) { + TreeMap> checkpoints = null; + try { + checkpoints = taskClient.getCheckpoints( + taskId, + true + ); + } + catch (Exception e) { + log.error(e, "Exception while getting checkpoints for task [%s]", taskId); + } + + if (checkpoints == null) { + log.error("Task [%s] returned no checkpoints, killing task", taskId); + killTask(taskId); + taskGroup.tasks.remove(taskId); + } else if (!checkpoints.isEmpty()) { + taskSequences.add(new Pair<>(taskId, checkpoints)); + } // otherwise task not started yet, continue + } + + final KafkaDataSourceMetadata latestDataSourceMetadata = (KafkaDataSourceMetadata) indexerMetadataStorageCoordinator + .getDataSourceMetadata(dataSource); + final Map latestOffsetsFromDb = (latestDataSourceMetadata == null + || latestDataSourceMetadata.getKafkaPartitions() == null) ? null + : latestDataSourceMetadata + .getKafkaPartitions() + .getPartitionOffsetMap(); + + // order tasks of this taskGroup by the latest sequenceId + taskSequences.sort((o1, o2) -> o2.rhs.firstKey().compareTo(o1.rhs.firstKey())); + + final Set tasksToKill = new HashSet<>(); + final AtomicInteger earliestConsistentSequenceId = new AtomicInteger(-1); + int taskIndex = 0; + + while (taskIndex < taskSequences.size()) { + if (earliestConsistentSequenceId.get() == -1) { + // find the first replica task with earliest sequenceId consistent with datasource metadata in the metadata store + if (taskSequences.get(taskIndex).rhs.entrySet().stream().anyMatch( + sequenceCheckpoint -> sequenceCheckpoint.getValue().entrySet().stream().allMatch( + partitionOffset -> Longs.compare( + partitionOffset.getValue(), + latestOffsetsFromDb == null + ? + partitionOffset.getValue() + : latestOffsetsFromDb.getOrDefault(partitionOffset.getKey(), partitionOffset.getValue()) + ) == 0) && earliestConsistentSequenceId.compareAndSet(-1, sequenceCheckpoint.getKey())) || ( + pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() > 0 + && earliestConsistentSequenceId.compareAndSet(-1, taskSequences.get(taskIndex).rhs.firstKey()))) { + final SortedMap> latestCheckpoints = new TreeMap<>(taskSequences.get(taskIndex).rhs + .tailMap( + earliestConsistentSequenceId + .get())); + log.info("Setting taskGroup sequences to [%s] for group [%d]", latestCheckpoints, groupId); + taskGroup.sequenceOffsets.clear(); + taskGroup.sequenceOffsets.putAll(latestCheckpoints); + } else { + // kill task + log.debug( + "Adding task [%s] to kill list, checkpoints[%s], latestoffsets from DB [%s]", + taskSequences.get(taskIndex).lhs, + taskSequences.get(taskIndex).rhs, + latestOffsetsFromDb + ); + tasksToKill.add(taskSequences.get(taskIndex).lhs); + } + } else { + // check consistency with taskGroup sequences + if (taskSequences.get(taskIndex).rhs.get(taskGroup.sequenceOffsets.firstKey()) == null + || !(taskSequences.get(taskIndex).rhs.get(taskGroup.sequenceOffsets.firstKey()) + .equals(taskGroup.sequenceOffsets.firstEntry().getValue())) + || taskSequences.get(taskIndex).rhs.tailMap(taskGroup.sequenceOffsets.firstKey()).size() + != taskGroup.sequenceOffsets.size()) { + log.debug( + "Adding task [%s] to kill list, checkpoints[%s], taskgroup checkpoints [%s]", + taskSequences.get(taskIndex).lhs, + taskSequences.get(taskIndex).rhs, + taskGroup.sequenceOffsets + ); + tasksToKill.add(taskSequences.get(taskIndex).lhs); + } + } + taskIndex++; + } + + if ((tasksToKill.size() > 0 && tasksToKill.size() == taskGroup.tasks.size()) || + (taskGroup.tasks.size() == 0 && pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() == 0)) { + // killing all tasks or no task left in the group ? + // clear state about the taskgroup so that get latest offset information is fetched from metadata store + log.warn("Clearing task group [%d] information as no valid tasks left the group", groupId); + sequenceTaskGroup.remove(generateSequenceName(groupId)); + taskGroups.remove(groupId); + partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET); + } + + taskSequences.stream().filter(taskIdSequences -> tasksToKill.contains(taskIdSequences.lhs)).forEach( + sequenceCheckpoint -> { + log.warn( + "Killing task [%s], as its checkpoints [%s] are not consistent with group checkpoints[%s] or latest persisted offsets in metadata store [%s]", + sequenceCheckpoint.lhs, + sequenceCheckpoint.rhs, + taskGroup.sequenceOffsets, + latestOffsetsFromDb + ); + killTask(sequenceCheckpoint.lhs); + taskGroup.tasks.remove(sequenceCheckpoint.lhs); + }); } private void addDiscoveredTaskToPendingCompletionTaskGroups( @@ -960,17 +1246,21 @@ private void addDiscoveredTaskToPendingCompletionTaskGroups( for (TaskGroup taskGroup : taskGroupList) { if (taskGroup.partitionOffsets.equals(startingPartitions)) { if (taskGroup.tasks.putIfAbsent(taskId, new TaskData()) == null) { - log.info("Added discovered task [%s] to existing pending task group", taskId); + log.info("Added discovered task [%s] to existing pending task group [%s]", taskId, groupId); } return; } } - log.info("Creating new pending completion task group for discovered task [%s]", taskId); + log.info("Creating new pending completion task group [%s] for discovered task [%s]", groupId, taskId); // reading the minimumMessageTime & maximumMessageTime from the publishing task and setting it here is not necessary as this task cannot // change to a state where it will read any more events - TaskGroup newTaskGroup = new TaskGroup(ImmutableMap.copyOf(startingPartitions), Optional.absent(), Optional.absent()); + TaskGroup newTaskGroup = new TaskGroup( + ImmutableMap.copyOf(startingPartitions), + Optional.absent(), + Optional.absent() + ); newTaskGroup.tasks.put(taskId, new TaskData()); newTaskGroup.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout()); @@ -1067,7 +1357,7 @@ private void checkTaskDuration() throws InterruptedException, ExecutionException if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { log.info("Task group [%d] has run for [%s]", groupId, ioConfig.getTaskDuration()); futureGroupIds.add(groupId); - futures.add(signalTasksToFinish(groupId)); + futures.add(checkpointTaskGroup(groupId, true)); } } @@ -1096,47 +1386,53 @@ private void checkTaskDuration() throws InterruptedException, ExecutionException for (String id : group.taskIds()) { killTask(id); } + // clear partitionGroups, so that latest offsets from db is used as start offsets not the stale ones + // if tasks did some successful incremental handoffs + partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET); } + sequenceTaskGroup.remove(generateSequenceName(groupId)); // remove this task group from the list of current task groups now that it has been handled taskGroups.remove(groupId); } } - private ListenableFuture> signalTasksToFinish(final int groupId) + private ListenableFuture> checkpointTaskGroup(final int groupId, final boolean finalize) { final TaskGroup taskGroup = taskGroups.get(groupId); - // 1) Check if any task completed (in which case we're done) and kill unassigned tasks - Iterator> i = taskGroup.tasks.entrySet().iterator(); - while (i.hasNext()) { - Map.Entry taskEntry = i.next(); - String taskId = taskEntry.getKey(); - TaskData task = taskEntry.getValue(); - - if (task.status.isSuccess()) { - // If any task in this group has already completed, stop the rest of the tasks in the group and return. - // This will cause us to create a new set of tasks next cycle that will start from the offsets in - // metadata store (which will have advanced if we succeeded in publishing and will remain the same if publishing - // failed and we need to re-ingest) - return Futures.transform( - stopTasksInGroup(taskGroup), new Function>() - { - @Nullable - @Override - public Map apply(@Nullable Object input) + if (finalize) { + // 1) Check if any task completed (in which case we're done) and kill unassigned tasks + Iterator> i = taskGroup.tasks.entrySet().iterator(); + while (i.hasNext()) { + Map.Entry taskEntry = i.next(); + String taskId = taskEntry.getKey(); + TaskData task = taskEntry.getValue(); + + if (task.status.isSuccess()) { + // If any task in this group has already completed, stop the rest of the tasks in the group and return. + // This will cause us to create a new set of tasks next cycle that will start from the offsets in + // metadata store (which will have advanced if we succeeded in publishing and will remain the same if publishing + // failed and we need to re-ingest) + return Futures.transform( + stopTasksInGroup(taskGroup), new Function>() { - return null; + @Nullable + @Override + public Map apply(@Nullable Object input) + { + return null; + } } - } - ); - } + ); + } - if (task.status.isRunnable()) { - if (taskInfoProvider.getTaskLocation(taskId).equals(TaskLocation.unknown())) { - log.info("Killing task [%s] which hasn't been assigned to a worker", taskId); - killTask(taskId); - i.remove(); + if (task.status.isRunnable()) { + if (taskInfoProvider.getTaskLocation(taskId).equals(TaskLocation.unknown())) { + log.info("Killing task [%s] which hasn't been assigned to a worker", taskId); + killTask(taskId); + i.remove(); + } } } } @@ -1186,12 +1482,22 @@ public Map apply(List> input) return null; } - log.info("Setting endOffsets for tasks in taskGroup [%d] to %s and resuming", groupId, endOffsets); - for (final String taskId : setEndOffsetTaskIds) { - setEndOffsetFutures.add(taskClient.setEndOffsetsAsync(taskId, endOffsets, true)); - } - try { + + if (endOffsets.equals(taskGroup.partitionOffsets)) { + log.warn( + "Not adding checkpoint [%s] as its offsets are same as the start offsets [%s] for the task group [%d]", + endOffsets, + taskGroup.partitionOffsets, + groupId + ); + } + + log.info("Setting endOffsets for tasks in taskGroup [%d] to %s and resuming", groupId, endOffsets); + for (final String taskId : setEndOffsetTaskIds) { + setEndOffsetFutures.add(taskClient.setEndOffsetsAsync(taskId, endOffsets, true, finalize)); + } + List results = Futures.successfulAsList(setEndOffsetFutures) .get(futureTimeoutInSeconds, TimeUnit.SECONDS); for (int i = 0; i < results.size(); i++) { @@ -1204,6 +1510,7 @@ public Map apply(List> input) } } catch (Exception e) { + log.error("Something bad happened [%s]", e.getMessage()); Throwables.propagate(e); } @@ -1284,17 +1591,16 @@ private void checkPendingCompletionTasks() throws ExecutionException, Interrupte } // reset partitions offsets for this task group so that they will be re-read from metadata storage - partitionGroups.remove(groupId); - - // stop all the tasks in this pending completion group - futures.add(stopTasksInGroup(group)); + partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET); + sequenceTaskGroup.remove(generateSequenceName(groupId)); + // kill all the tasks in this pending completion group + killTasksInGroup(group); // set a flag so the other pending completion groups for this set of partitions will also stop stopTasksInTaskGroup = true; - // stop all the tasks in the currently reading task group and remove the bad task group - futures.add(stopTasksInGroup(taskGroups.remove(groupId))); - + // kill all the tasks in the currently reading task group and remove the bad task group + killTasksInGroup(taskGroups.remove(groupId)); toRemove.add(group); } } @@ -1347,6 +1653,7 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep // be recreated with the next set of offsets if (taskData.status.isSuccess()) { futures.add(stopTasksInGroup(taskGroup)); + sequenceTaskGroup.remove(generateSequenceName(groupId)); iTaskGroups.remove(); break; } @@ -1358,8 +1665,14 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); } - void createNewTasks() + void createNewTasks() throws JsonProcessingException { + // update the checkpoints in the taskGroup to latest ones so that new tasks do not read what is already published + taskGroups.entrySet() + .stream() + .filter(taskGroup -> taskGroup.getValue().tasks.size() < ioConfig.getReplicas()) + .forEach(taskGroup -> verifyAndMergeCheckpoints(taskGroup.getKey())); + // check that there is a current task group for each group of partitions in [partitionGroups] for (Integer groupId : partitionGroups.keySet()) { if (!taskGroups.containsKey(groupId)) { @@ -1373,7 +1686,15 @@ void createNewTasks() DateTimes.nowUtc().plus(ioConfig.getTaskDuration()).plus(ioConfig.getEarlyMessageRejectionPeriod().get()) ) : Optional.absent()); - taskGroups.put(groupId, new TaskGroup(generateStartingOffsetsForPartitionGroup(groupId), minimumMessageTime, maximumMessageTime)); + taskGroups.put( + groupId, + new TaskGroup( + generateStartingOffsetsForPartitionGroup(groupId), + minimumMessageTime, + maximumMessageTime + ) + ); + sequenceTaskGroup.put(generateSequenceName(groupId), taskGroups.get(groupId)); } } @@ -1400,10 +1721,11 @@ void createNewTasks() } } - private void createKafkaTasksForGroup(int groupId, int replicas) + private void createKafkaTasksForGroup(int groupId, int replicas) throws JsonProcessingException { Map startPartitions = taskGroups.get(groupId).partitionOffsets; - Map endPartitions = new HashMap<>(); + Map endPartitions = new HashMap<>(); // TODO if endOffsets were already set + for (Integer partition : startPartitions.keySet()) { endPartitions.put(partition, Long.MAX_VALUE); } @@ -1426,6 +1748,15 @@ private void createKafkaTasksForGroup(int groupId, int replicas) ioConfig.isSkipOffsetGaps() ); + final String checkpoints = sortingMapper.writerWithType(new TypeReference>>() + { + }).writeValueAsString(taskGroups.get(groupId).sequenceOffsets); + final Map context = spec.getContext() == null + ? ImmutableMap.of("checkpoints", checkpoints) + : ImmutableMap.builder() + .put("checkpoints", checkpoints) + .putAll(spec.getContext()) + .build(); for (int i = 0; i < replicas; i++) { String taskId = Joiner.on("_").join(sequenceName, getRandomId()); KafkaIndexTask indexTask = new KafkaIndexTask( @@ -1434,7 +1765,7 @@ private void createKafkaTasksForGroup(int groupId, int replicas) spec.getDataSchema(), taskTuningConfig, kafkaIOConfig, - spec.getContext(), + context, null, null ); @@ -1558,8 +1889,17 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) } String taskSequenceName = ((KafkaIndexTask) taskOptional.get()).getIOConfig().getBaseSequenceName(); - - return generateSequenceName(taskGroupId).equals(taskSequenceName); + if (taskGroups.get(taskGroupId) != null) { + return generateSequenceName(taskGroupId).equals(taskSequenceName); + } else { + return generateSequenceName( + ((KafkaIndexTask) taskOptional.get()).getIOConfig() + .getStartPartitions() + .getPartitionOffsetMap(), + ((KafkaIndexTask) taskOptional.get()).getIOConfig().getMinimumMessageTime(), + ((KafkaIndexTask) taskOptional.get()).getIOConfig().getMaximumMessageTime() + ).equals(taskSequenceName); + } } private ListenableFuture stopTasksInGroup(TaskGroup taskGroup) @@ -1607,7 +1947,7 @@ private void killTask(final String id) } } - private int getTaskGroupIdForPartition(int partition) + protected int getTaskGroupIdForPartition(int partition) { return partition % ioConfig.getTaskCount(); } @@ -1698,7 +2038,7 @@ private KafkaSupervisorReport generateReport(boolean includeOffsets) } } - taskReports.stream().forEach(report::addTask); + taskReports.forEach(report::addTask); } catch (Exception e) { log.warn(e, "Failed to generate status report"); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index ef83165e8ebd..587d1a300ddd 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -46,7 +46,7 @@ public KafkaSupervisorTuningConfig( // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. @JsonProperty("buildV9Directly") Boolean buildV9Directly, @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, - @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, // for backward compatibility + @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, @JsonProperty("workerThreads") Integer workerThreads, @JsonProperty("chatThreads") Integer chatThreads, @@ -65,8 +65,6 @@ public KafkaSupervisorTuningConfig( indexSpec, true, reportParseExceptions, - // Supervised kafka tasks should respect KafkaSupervisorIOConfig.completionTimeout instead of - // handoffConditionTimeout handoffConditionTimeout, resetOffsetAutomatically ); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java index 70084eb89836..6464d5eb130c 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -144,7 +144,7 @@ public void testNoTaskLocation() throws Exception Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true)); Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID)); Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of())); - Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of(), true)); + Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of(), true, true)); verifyAll(); } @@ -548,7 +548,7 @@ public void testSetEndOffsets() throws Exception Request request = captured.getValue(); Assert.assertEquals(HttpMethod.POST, request.getMethod()); Assert.assertEquals( - new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end"), + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end?resume=false&finish=true"), request.getUrl() ); Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); @@ -567,13 +567,13 @@ public void testSetEndOffsetsAndResume() throws Exception ); replayAll(); - client.setEndOffsets(TEST_ID, endOffsets, true); + client.setEndOffsets(TEST_ID, endOffsets, true, true); verifyAll(); Request request = captured.getValue(); Assert.assertEquals(HttpMethod.POST, request.getMethod()); Assert.assertEquals( - new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end?resume=true"), + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end?resume=true&finish=true"), request.getUrl() ); Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); @@ -902,7 +902,7 @@ public void testSetEndOffsetsAsync() throws Exception List expectedUrls = Lists.newArrayList(); List> futures = Lists.newArrayList(); for (String testId : TEST_IDS) { - expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "offsets/end"))); + expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, StringUtils.format("offsets/end?resume=%s&finish=%s", false, true)))); futures.add(client.setEndOffsetsAsync(testId, endOffsets)); } @@ -942,11 +942,11 @@ public void testSetEndOffsetsAsyncWithResume() throws Exception TEST_HOST, TEST_PORT, testId, - "offsets/end?resume=true" + "offsets/end?resume=true&finish=true" ) ) ); - futures.add(client.setEndOffsetsAsync(testId, endOffsets, true)); + futures.add(client.setEndOffsetsAsync(testId, endOffsets, true, true)); } List responses = Futures.allAsList(futures).get(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index e7c94624ff34..7e110bc9a9cc 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -121,6 +121,7 @@ import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.SegmentLoaderLocalCacheManager; import io.druid.segment.loading.StorageLocationConfig; +import io.druid.segment.realtime.appenderator.AppenderatorImpl; import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import io.druid.server.DruidNode; @@ -144,6 +145,8 @@ import java.io.File; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -154,6 +157,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +//CHECKSTYLE.OFF: Regexp +//CHECKSTYLE.ON: Regexp + public class KafkaIndexTaskTest { private static final Logger log = new Logger(KafkaIndexTaskTest.class); @@ -1024,7 +1030,7 @@ public void testRunOneTaskTwoPartitions() throws Exception // Check published segments & metadata SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); - SegmentDescriptor desc3 = SD(task, "2011/P1D", 1); + SegmentDescriptor desc3 = SD(task, "2011/P1D", 0); SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals( @@ -1151,6 +1157,8 @@ public void testRestore() throws Exception // Stop without publishing segment task1.stopGracefully(); + unlockAppenderatorBasePersistDirForTask(task1); + Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); // Start a new task @@ -1326,7 +1334,7 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception Assert.assertEquals(ImmutableMap.of(0, 3L), task.getEndOffsets()); Map newEndOffsets = ImmutableMap.of(0, 4L); - task.setEndOffsets(newEndOffsets, false); + task.setEndOffsets(newEndOffsets, false, true); Assert.assertEquals(newEndOffsets, task.getEndOffsets()); Assert.assertEquals(KafkaIndexTask.Status.PAUSED, task.getStatus()); task.resume(); @@ -1341,7 +1349,7 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception // try again but with resume flag == true newEndOffsets = ImmutableMap.of(0, 6L); - task.setEndOffsets(newEndOffsets, true); + task.setEndOffsets(newEndOffsets, true, true); Assert.assertEquals(newEndOffsets, task.getEndOffsets()); Assert.assertNotEquals(KafkaIndexTask.Status.PAUSED, task.getStatus()); @@ -1738,6 +1746,16 @@ public SegmentDescriptor apply(DataSegment input) ).toSet(); } + private void unlockAppenderatorBasePersistDirForTask(KafkaIndexTask task) + throws NoSuchMethodException, InvocationTargetException, IllegalAccessException + { + Method unlockBasePersistDir = ((AppenderatorImpl) task.getAppenderator()).getClass() + .getDeclaredMethod( + "unlockBasePersistDirectory"); + unlockBasePersistDir.setAccessible(true); + unlockBasePersistDir.invoke(task.getAppenderator()); + } + private File getSegmentDirectory() { return new File(directory, "segments"); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java index 81aa36666836..3bd5a3891007 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java @@ -93,7 +93,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(100, config.getMaxRowsInMemory()); Assert.assertEquals(100, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); - Assert.assertEquals(100, config.getMaxPendingPersists()); + Assert.assertEquals(0, config.getMaxPendingPersists()); Assert.assertEquals(true, config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); } @@ -119,7 +119,7 @@ public void testCopyOf() throws Exception Assert.assertEquals(2, copy.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod()); Assert.assertEquals(new File("/tmp/xxx"), copy.getBasePersistDirectory()); - Assert.assertEquals(4, copy.getMaxPendingPersists()); + Assert.assertEquals(0, copy.getMaxPendingPersists()); Assert.assertEquals(new IndexSpec(), copy.getIndexSpec()); Assert.assertEquals(true, copy.isReportParseExceptions()); Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 3fbfc11f726a..fc7e18f0799e 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -96,8 +96,10 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.TreeMap; import java.util.concurrent.Executor; +import static org.easymock.EasyMock.anyBoolean; import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.anyString; import static org.easymock.EasyMock.capture; @@ -620,6 +622,11 @@ public void testKillIncompatibleTasks() throws Exception taskQueue.shutdown("id3"); expect(taskQueue.add(anyObject(Task.class))).andReturn(true); + + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); + expect(taskClient.getCheckpoints(EasyMock.contains("id2"), anyBoolean())).andReturn(checkpoints).times(2); + replayAll(); supervisor.start(); @@ -693,8 +700,8 @@ public void testKillBadPartitionAssignment() throws Exception expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - expect(taskStorage.getTask("id4")).andReturn(Optional.of(id3)).anyTimes(); - expect(taskStorage.getTask("id5")).andReturn(Optional.of(id3)).anyTimes(); + expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); + expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) .anyTimes(); expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); @@ -706,6 +713,14 @@ public void testKillBadPartitionAssignment() throws Exception expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(1, 0L)); + expect(taskClient.getCheckpoints(EasyMock.contains("id1"), anyBoolean())).andReturn(checkpoints1).times(1); + expect(taskClient.getCheckpoints(EasyMock.contains("id2"), anyBoolean())).andReturn(checkpoints2).times(1); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); taskQueue.shutdown("id4"); taskQueue.shutdown("id5"); @@ -736,6 +751,16 @@ public void testRequeueTaskWhenFailed() throws Exception ) ).anyTimes(); expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(1, 0L)); + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + .anyTimes(); + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + .anyTimes(); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); @@ -820,6 +845,11 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception null ) ).anyTimes(); + + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of(0, 0L, 2, 0L)); + expect(taskClient.getCheckpoints(EasyMock.contains("id1"), anyBoolean())).andReturn(checkpoints).times(2); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); @@ -837,6 +867,12 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception reset(taskStorage); reset(taskQueue); reset(taskClient); + + // for the newly created replica task + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints) + .times(2); + expect(taskClient.getCheckpoints(EasyMock.contains("id1"), anyBoolean())).andReturn(checkpoints).times(1); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes(); @@ -888,6 +924,7 @@ public void testQueueNextTasksOnSuccess() throws Exception ).anyTimes(); expect(taskQueue.add(capture(captured))).andReturn(true).times(4); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); supervisor.start(); @@ -897,12 +934,28 @@ public void testQueueNextTasksOnSuccess() throws Exception List tasks = captured.getValues(); reset(taskStorage); + reset(taskClient); + + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(1, 0L)); + // there would be 4 tasks, 2 for each task group + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + .times(2); + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + .times(2); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } replay(taskStorage); + replay(taskClient); supervisor.runInternal(); verifyAll(); @@ -992,11 +1045,21 @@ public void testBeginPublishAndQueueNextTasks() throws Exception taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), + EasyMock.eq(true), EasyMock.eq(true) ) ).andReturn(Futures.immediateFuture(true)).times(2); expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(1, 0L)); + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + .times(2); + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + .times(2); + replay(taskStorage, taskRunner, taskClient, taskQueue); supervisor.runInternal(); @@ -1052,12 +1115,16 @@ public void testDiscoverExistingPublishingTask() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.FINISHING)); expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); - expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); expect(taskQueue.add(capture(captured))).andReturn(true); + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); + expect(taskClient.getCheckpoints(anyString(), anyBoolean())).andReturn(checkpoints).anyTimes(); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); @@ -1143,10 +1210,10 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.FINISHING)); expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); - expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); expect(taskQueue.add(capture(captured))).andReturn(true); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); @@ -1248,16 +1315,22 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.FINISHING)); expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 1L, 1, 2L, 2, 3L))); - expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); expect(taskClient.getCurrentOffsetsAsync("id2", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 4L, 1, 5L, 2, 6L))); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + + // since id1 is publishing, so getCheckpoints wouldn't be called for it + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); + expect(taskClient.getCheckpoints(EasyMock.contains("id2"), anyBoolean())).andReturn(checkpoints).times(1); + replayAll(); supervisor.start(); @@ -1327,6 +1400,16 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception List tasks = captured.getValues(); reset(taskStorage, taskClient, taskQueue); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(1, 0L)); + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + .times(2); + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + .times(2); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); @@ -1376,6 +1459,16 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception } reset(taskStorage, taskRunner, taskClient, taskQueue); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(1, 0L)); + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + .times(2); + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + .times(2); + captured = Capture.newInstance(CaptureType.ALL); expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { @@ -1443,6 +1536,16 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception } reset(taskStorage, taskRunner, taskClient, taskQueue); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(1, 0L)); + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + .times(2); + expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + .times(2); + captured = Capture.newInstance(CaptureType.ALL); expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { @@ -1466,6 +1569,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), + EasyMock.eq(true), EasyMock.eq(true) ) ).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); @@ -1566,12 +1670,18 @@ public void testStopGracefully() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.FINISHING)); expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + + // getCheckpoints will not be called for id1 as it is in publishing state + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getCheckpoints(EasyMock.contains("id2"), anyBoolean())).andReturn(checkpoints).times(1); + expect(taskClient.getCheckpoints(EasyMock.contains("id3"), anyBoolean())).andReturn(checkpoints).times(1); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); @@ -1584,7 +1694,7 @@ public void testStopGracefully() throws Exception expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); expect(taskClient.pauseAsync("id2")) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); - expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) + expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true, true)) .andReturn(Futures.immediateFuture(true)); taskQueue.shutdown("id3"); expectLastCall().times(2); @@ -1660,7 +1770,14 @@ public void testResetDataSourceMetadata() throws Exception )).andReturn(true); replay(indexerMetadataStorageCoordinator); - supervisor.resetInternal(resetMetadata); + try { + supervisor.resetInternal(resetMetadata); + } + catch (NullPointerException npe) { + // Expected as there will be an attempt to reset partitionGroups offsets to NOT_SET + // however there would be no entries in the map as we have not put nay data in kafka + Assert.assertTrue(npe.getCause() == null); + } verifyAll(); Assert.assertEquals(captureDataSource.getValue(), DATASOURCE); @@ -1755,12 +1872,17 @@ public void testResetRunningTasks() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.FINISHING)); expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getCheckpoints(EasyMock.contains("id2"), anyBoolean())).andReturn(checkpoints).times(1); + expect(taskClient.getCheckpoints(EasyMock.contains("id3"), anyBoolean())).andReturn(checkpoints).times(1); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); @@ -1969,5 +2091,15 @@ protected String generateSequenceName(int groupId) { return StringUtils.format("sequenceName-%d", groupId); } + + @Override + protected String generateSequenceName( + Map startPartitions, + Optional minimumMessageTime, + Optional maximumMessageTime + ) + { + return generateSequenceName(getTaskGroupIdForPartition(startPartitions.keySet().iterator().next())); + } } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java index e399dd0bd4a0..34477c143727 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java @@ -107,7 +107,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(100, config.getMaxRowsInMemory()); Assert.assertEquals(100, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); - Assert.assertEquals(100, config.getMaxPendingPersists()); + Assert.assertEquals(0, config.getMaxPendingPersists()); Assert.assertEquals(true, config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); Assert.assertEquals(12, (int) config.getWorkerThreads()); diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java index 2ead3b2c4e45..28a072d14cb5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java @@ -46,7 +46,8 @@ public ActionBasedSegmentAllocator( public SegmentIdentifier allocate( final InputRow row, final String sequenceName, - final String previousSegmentId + final String previousSegmentId, + final boolean skipSegmentLineageCheck ) throws IOException { return taskActionClient.submit( @@ -56,7 +57,8 @@ public SegmentIdentifier allocate( dataSchema.getGranularitySpec().getQueryGranularity(), dataSchema.getGranularitySpec().getSegmentGranularity(), sequenceName, - previousSegmentId + previousSegmentId, + skipSegmentLineageCheck ) ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/CheckPointDataSourceMetadataAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/CheckPointDataSourceMetadataAction.java new file mode 100644 index 000000000000..aca4a1099aef --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/CheckPointDataSourceMetadataAction.java @@ -0,0 +1,106 @@ +/* + * 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.actions; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.DataSourceMetadata; + +import java.io.IOException; + +public class CheckPointDataSourceMetadataAction implements TaskAction +{ + private final String supervisorId; + private final String sequenceName; + private final DataSourceMetadata previousCheckPoint; + private final DataSourceMetadata currentCheckPoint; + + public CheckPointDataSourceMetadataAction( + @JsonProperty("supervisorId") String supervisorId, + @JsonProperty("sequenceName") String sequenceName, + @JsonProperty("previousCheckPoint") DataSourceMetadata previousCheckPoint, + @JsonProperty("currentCheckPoint") DataSourceMetadata currentCheckPoint + ) + { + this.supervisorId = supervisorId; + this.sequenceName = sequenceName; + this.previousCheckPoint = previousCheckPoint; + this.currentCheckPoint = currentCheckPoint; + } + + @JsonProperty + public String getSupervisorId() + { + return supervisorId; + } + + @JsonProperty + public String getSequenceName() + { + return sequenceName; + } + + @JsonProperty + public DataSourceMetadata getPreviousCheckPoint() + { + return previousCheckPoint; + } + + @JsonProperty + public DataSourceMetadata getCurrentCheckPoint() + { + return currentCheckPoint; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + @Override + public Boolean perform( + Task task, TaskActionToolbox toolbox + ) throws IOException + { + return toolbox.getSupervisorManager() + .checkPointDataSourceMetadata(supervisorId, sequenceName, previousCheckPoint, currentCheckPoint); + } + + @Override + public boolean isAudited() + { + return true; + } + + @Override + public String toString() + { + return "CheckPointDataSourceMetadataAction{" + + "supervisorId='" + supervisorId + '\'' + + ", sequenceName='" + sequenceName + '\'' + + ", previousCheckPoint=" + previousCheckPoint + + ", currentCheckPoint=" + currentCheckPoint + + '}'; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java index d340e5b7df6b..ef73da4d9305 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java @@ -66,6 +66,7 @@ public class SegmentAllocateAction implements TaskAction private final Granularity preferredSegmentGranularity; private final String sequenceName; private final String previousSegmentId; + private final boolean skipSegmentLineageCheck; public SegmentAllocateAction( @JsonProperty("dataSource") String dataSource, @@ -73,7 +74,8 @@ public SegmentAllocateAction( @JsonProperty("queryGranularity") Granularity queryGranularity, @JsonProperty("preferredSegmentGranularity") Granularity preferredSegmentGranularity, @JsonProperty("sequenceName") String sequenceName, - @JsonProperty("previousSegmentId") String previousSegmentId + @JsonProperty("previousSegmentId") String previousSegmentId, + @JsonProperty("skipSegmentLineageCheck") boolean skipSegmentLineageCheck ) { this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); @@ -85,6 +87,7 @@ public SegmentAllocateAction( ); this.sequenceName = Preconditions.checkNotNull(sequenceName, "sequenceName"); this.previousSegmentId = previousSegmentId; + this.skipSegmentLineageCheck = skipSegmentLineageCheck; } @JsonProperty @@ -123,6 +126,12 @@ public String getPreviousSegmentId() return previousSegmentId; } + @JsonProperty + public boolean isSkipSegmentLineageCheck() + { + return skipSegmentLineageCheck; + } + @Override public TypeReference getReturnTypeReference() { @@ -157,12 +166,13 @@ public SegmentIdentifier perform( ); final SegmentIdentifier identifier = usedSegmentsForRow.isEmpty() ? - tryAllocateFirstSegment(toolbox, task, rowInterval) : + tryAllocateFirstSegment(toolbox, task, rowInterval, skipSegmentLineageCheck) : tryAllocateSubsequentSegment( toolbox, task, rowInterval, - usedSegmentsForRow.iterator().next() + usedSegmentsForRow.iterator().next(), + skipSegmentLineageCheck ); if (identifier != null) { return identifier; @@ -205,7 +215,8 @@ public SegmentIdentifier perform( private SegmentIdentifier tryAllocateFirstSegment( TaskActionToolbox toolbox, Task task, - Interval rowInterval + Interval rowInterval, + boolean skipSegmentLineageCheck ) throws IOException { // No existing segments for this row, but there might still be nearby ones that conflict with our preferred @@ -216,7 +227,7 @@ private SegmentIdentifier tryAllocateFirstSegment( .collect(Collectors.toList()); for (Interval tryInterval : tryIntervals) { if (tryInterval.contains(rowInterval)) { - final SegmentIdentifier identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false); + final SegmentIdentifier identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false, skipSegmentLineageCheck); if (identifier != null) { return identifier; } @@ -229,7 +240,8 @@ private SegmentIdentifier tryAllocateSubsequentSegment( TaskActionToolbox toolbox, Task task, Interval rowInterval, - DataSegment usedSegment + DataSegment usedSegment, + boolean skipSegmentLineageCheck ) throws IOException { // Existing segment(s) exist for this row; use the interval of the first one. @@ -239,7 +251,7 @@ private SegmentIdentifier tryAllocateSubsequentSegment( } else { // If segment allocation failed here, it is highly likely an unrecoverable error. We log here for easier // debugging. - return tryAllocate(toolbox, task, usedSegment.getInterval(), rowInterval, true); + return tryAllocate(toolbox, task, usedSegment.getInterval(), rowInterval, true, skipSegmentLineageCheck); } } @@ -248,7 +260,8 @@ private SegmentIdentifier tryAllocate( Task task, Interval tryInterval, Interval rowInterval, - boolean logOnFail + boolean logOnFail, + boolean skipSegmentLineageCheck ) throws IOException { log.debug( @@ -268,7 +281,8 @@ private SegmentIdentifier tryAllocate( sequenceName, previousSegmentId, tryInterval, - lockResult.getTaskLock().getVersion() + lockResult.getTaskLock().getVersion(), + skipSegmentLineageCheck ); if (identifier != null) { return identifier; @@ -316,6 +330,7 @@ public String toString() ", preferredSegmentGranularity=" + preferredSegmentGranularity + ", sequenceName='" + sequenceName + '\'' + ", previousSegmentId='" + previousSegmentId + '\'' + + ", skipSegmentLineageCheck='" + skipSegmentLineageCheck + '\'' + '}'; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java index 000ee8c35565..0da85f4803bf 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java @@ -39,7 +39,8 @@ @JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class), @JsonSubTypes.Type(name = "segmentMetadataUpdate", value = SegmentMetadataUpdateAction.class), @JsonSubTypes.Type(name = "segmentAllocate", value = SegmentAllocateAction.class), - @JsonSubTypes.Type(name = "resetDataSourceMetadata", value = ResetDataSourceMetadataAction.class) + @JsonSubTypes.Type(name = "resetDataSourceMetadata", value = ResetDataSourceMetadataAction.class), + @JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class) }) public interface TaskAction { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index cc5c28afa9d6..3d1288303303 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -622,7 +622,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null } } - segmentAllocator = (row, sequenceName, previousSegmentId) -> lookup.get(sequenceName); + segmentAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> lookup.get(sequenceName); } else if (ioConfig.isAppendToExisting()) { // Append mode: Allocate segments as needed using Overlord APIs. segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema); @@ -630,7 +630,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null // Overwrite mode, non-guaranteed rollup: We can make up our own segment ids but we don't know them in advance. final Map counters = new HashMap<>(); - segmentAllocator = (row, sequenceName, previousSegmentId) -> { + segmentAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> { final DateTime timestamp = row.getTimestamp(); Optional maybeInterval = granularitySpec.bucketInterval(timestamp); if (!maybeInterval.isPresent()) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorManager.java index eb7bdaf99c08..1dd4d76d7620 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -156,6 +156,31 @@ public boolean resetSupervisor(String id, @Nullable DataSourceMetadata dataSourc return true; } + public boolean checkPointDataSourceMetadata( + String supervisorId, + @Nullable String sequenceName, + @Nullable DataSourceMetadata previousDataSourceMetadata, + @Nullable DataSourceMetadata currentDataSourceMetadata + ) + { + try { + Preconditions.checkState(started, "SupervisorManager not started"); + Preconditions.checkNotNull(supervisorId, "supervisorId cannot be null"); + + Pair supervisor = supervisors.get(supervisorId); + + Preconditions.checkNotNull(supervisor, "supervisor could not be found"); + + supervisor.lhs.checkpoint(sequenceName, previousDataSourceMetadata, currentDataSourceMetadata); + return true; + } + catch (Exception e) { + log.error(e, "Checkpoint request failed"); + } + return false; + } + + /** * Stops a supervisor with a given id and then removes it from the list. *

diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java index 888ae59dc909..638336eca718 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -690,7 +690,8 @@ public void testSerde() throws Exception Granularities.MINUTE, Granularities.HOUR, "s1", - "prev" + "prev", + false ); final ObjectMapper objectMapper = new DefaultObjectMapper(); @@ -722,7 +723,8 @@ private SegmentIdentifier allocate( queryGranularity, preferredSegmentGranularity, sequenceName, - sequencePreviousId + sequencePreviousId, + false ); return action.perform(task, taskActionTestKit.getTaskActionToolbox()); } diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 03e08728490b..d51957743ef8 100644 --- a/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -116,7 +116,8 @@ public SegmentIdentifier allocatePendingSegment( String sequenceName, String previousSegmentId, Interval interval, - String maxVersion + String maxVersion, + boolean skipSegmentLineageCheck ) throws IOException { throw new UnsupportedOperationException(); diff --git a/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index d3f0d61f7707..00f29f68cec7 100644 --- a/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -82,6 +82,8 @@ List getUsedSegmentsForIntervals(String dataSource, List * @param interval interval for which to allocate a segment * @param maxVersion use this version if we have no better version to use. The returned segment identifier may * have a version lower than this one, but will not have one higher. + * @param skipSegmentLineageCheck if false, perform lineage validation using previousSegmentId for this sequence. + * Should be set to false if replica tasks would index events in same order * * @return the pending segment identifier, or null if it was impossible to allocate a new segment */ @@ -90,7 +92,8 @@ SegmentIdentifier allocatePendingSegment( String sequenceName, String previousSegmentId, Interval interval, - String maxVersion + String maxVersion, + boolean skipSegmentLineageCheck ) throws IOException; /** diff --git a/server/src/main/java/io/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java b/server/src/main/java/io/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java index 508e5e9bffa2..401023fe032a 100644 --- a/server/src/main/java/io/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java +++ b/server/src/main/java/io/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java @@ -20,6 +20,8 @@ package io.druid.indexing.overlord.supervisor; import io.druid.indexing.overlord.DataSourceMetadata; + +import javax.annotation.Nullable; import java.util.List; /** @@ -52,6 +54,16 @@ public SupervisorReport getStatus() @Override public void reset(DataSourceMetadata dataSourceMetadata) {} + + @Override + public void checkpoint( + @Nullable String sequenceName, + @Nullable DataSourceMetadata previousCheckPoint, + @Nullable DataSourceMetadata currentCheckPoint + ) + { + + } }; } diff --git a/server/src/main/java/io/druid/indexing/overlord/supervisor/Supervisor.java b/server/src/main/java/io/druid/indexing/overlord/supervisor/Supervisor.java index cec68c9674e2..c6c411480683 100644 --- a/server/src/main/java/io/druid/indexing/overlord/supervisor/Supervisor.java +++ b/server/src/main/java/io/druid/indexing/overlord/supervisor/Supervisor.java @@ -21,6 +21,8 @@ import io.druid.indexing.overlord.DataSourceMetadata; +import javax.annotation.Nullable; + public interface Supervisor { void start(); @@ -36,4 +38,20 @@ public interface Supervisor SupervisorReport getStatus(); void reset(DataSourceMetadata dataSourceMetadata); + + /** + * The definition of checkpoint is not very strict as currently it does not affect data or control path + * On this call Supervisor can potentially checkpoint data processed so far to some durable storage + * for example - Kafka Supervisor uses this to merge and handoff segments containing at least the data + * represented by dataSourceMetadata + * + * @param sequenceName unique Identifier to figure out for which sequence to do check pointing + * @param previousCheckPoint DataSourceMetadata check pointed in previous call + * @param currentCheckPoint current DataSourceMetadata to be check pointed + */ + void checkpoint( + @Nullable String sequenceName, + @Nullable DataSourceMetadata previousCheckPoint, + @Nullable DataSourceMetadata currentCheckPoint + ); } diff --git a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 53035f941204..b8bb7e7ea8e7 100644 --- a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -385,7 +385,8 @@ public SegmentIdentifier allocatePendingSegment( final String sequenceName, final String previousSegmentId, final Interval interval, - final String maxVersion + final String maxVersion, + final boolean skipSegmentLineageCheck ) throws IOException { Preconditions.checkNotNull(dataSource, "dataSource"); @@ -401,20 +402,40 @@ public SegmentIdentifier allocatePendingSegment( @Override public SegmentIdentifier inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception { - final List existingBytes = handle - .createQuery( - StringUtils.format( - "SELECT payload FROM %s WHERE " - + "dataSource = :dataSource AND " - + "sequence_name = :sequence_name AND " - + "sequence_prev_id = :sequence_prev_id", - dbTables.getPendingSegmentsTable() - ) - ).bind("dataSource", dataSource) - .bind("sequence_name", sequenceName) - .bind("sequence_prev_id", previousSegmentIdNotNull) - .map(ByteArrayMapper.FIRST) - .list(); + final List existingBytes; + if (!skipSegmentLineageCheck) { + existingBytes = handle + .createQuery( + StringUtils.format( + "SELECT payload FROM %s WHERE " + + "dataSource = :dataSource AND " + + "sequence_name = :sequence_name AND " + + "sequence_prev_id = :sequence_prev_id", + dbTables.getPendingSegmentsTable() + ) + ).bind("dataSource", dataSource) + .bind("sequence_name", sequenceName) + .bind("sequence_prev_id", previousSegmentIdNotNull) + .map(ByteArrayMapper.FIRST) + .list(); + } else { + existingBytes = handle + .createQuery( + StringUtils.format( + "SELECT payload FROM %s WHERE " + + "dataSource = :dataSource AND " + + "sequence_name = :sequence_name AND " + + "start = :start AND " + + "%2$send%2$s = :end", + dbTables.getPendingSegmentsTable(), connector.getQuoteString() + ) + ).bind("dataSource", dataSource) + .bind("sequence_name", sequenceName) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .map(ByteArrayMapper.FIRST) + .list(); + } if (!existingBytes.isEmpty()) { final SegmentIdentifier existingIdentifier = jsonMapper.readValue( @@ -744,6 +765,7 @@ protected DataSourceMetadataUpdateResult updateDataSourceMetadataWithHandle( if (!startMetadataMatchesExisting) { // Not in the desired start state. log.info("Not updating metadata, existing state is not the expected start state."); + log.debug("Existing database state [%s], request's start metadata", oldCommitMetadataFromDb, startMetadata); return DataSourceMetadataUpdateResult.FAILURE; } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java index 7faffbdf21b2..79d24c56a48c 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java @@ -104,7 +104,7 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * Drop all in-memory and on-disk data, and forget any previously-remembered commit metadata. This could be useful if, * for some reason, rows have been added that we do not actually want to hand off. Blocks until all data has been * cleared. This may take some time, since all pending persists must finish first. - * + *

* The add, clear, persist, persistAll, and push methods should all be called from the same thread to keep the * metadata committed by Committer in sync. */ @@ -135,7 +135,7 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * * @param identifiers segment identifiers to be persisted * @param committer a committer associated with all data that has been added to segments of the given identifiers so - * far + * far * * @return future that resolves when all pending data to segments of the identifiers has been persisted, contains * commit metadata for this persist @@ -178,9 +178,18 @@ default ListenableFuture persistAll(Committer committer) ListenableFuture push(Collection identifiers, Committer committer); /** - * Stop any currently-running processing and clean up after ourselves. This will not remove any on-disk persisted - * data, but it will drop any data that has not yet been persisted. + * Stop any currently-running processing and clean up after ourselves. This allows currently running persists and pushes + * to finish. This will not remove any on-disk persisted data, but it will drop any data that has not yet been persisted. */ @Override void close(); + + /** + * Stop all processing, abandoning current pushes, currently running persist may be allowed to finish if they persist + * critical metadata otherwise shutdown immediately. This will not remove any on-disk persisted data, + * but it will drop any data that has not yet been persisted. + * Since this does not wait for pushes to finish, implementations have to make sure if any push is still happening + * in background thread then it does not cause any problems. + */ + void closeNow(); } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java index 172011e3a58b..b9a4aaa63b70 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java @@ -19,6 +19,8 @@ package io.druid.segment.realtime.appenderator; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; @@ -52,11 +54,11 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.NavigableMap; -import java.util.Objects; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.atomic.AtomicInteger; @@ -65,11 +67,11 @@ /** * A AppenderatorDriver drives an Appenderator to index a finite stream of data. This class does not help you * index unbounded streams. All handoff is done at the end of indexing. - * + *

* This class helps with doing things that Appenderators don't, including deciding which segments to use (with a * SegmentAllocator), publishing segments to the metadata store (with a SegmentPublisher), and monitoring handoff (with * a SegmentHandoffNotifier). - * + *

* Note that the commit metadata stored by this class via the underlying Appenderator is not the same metadata as * you pass in. It's wrapped in some extra metadata needed by the driver. */ @@ -84,16 +86,56 @@ public class AppenderatorDriver implements Closeable private final ObjectMapper objectMapper; private final FireDepartmentMetrics metrics; - // All access to "activeSegments", "publishPendingSegments", and "lastSegmentId" must be synchronized on - // "activeSegments". + private enum SegmentState + { + ACTIVE, + INACTIVE, + PUBLISHING + } - // sequenceName -> start of segment interval -> segment we're currently adding data to - private final Map> activeSegments = new TreeMap<>(); + static class SegmentWithState + { + private SegmentIdentifier segmentIdentifier; + private SegmentState state; + + @JsonCreator + SegmentWithState( + @JsonProperty("segmentIdentifier") SegmentIdentifier segmentIdentifier, + @JsonProperty("state") SegmentState state + ) + { + this.segmentIdentifier = segmentIdentifier; + this.state = state; + } - // sequenceName -> list of identifiers of segments waiting for being published - // publishPendingSegments is always a super set of activeSegments because there can be some segments to which data - // are not added anymore, but not published yet. - private final Map> publishPendingSegments = new HashMap<>(); + @JsonProperty + public SegmentIdentifier getSegmentIdentifier() + { + return segmentIdentifier; + } + + @JsonProperty + public SegmentState getState() + { + return state; + } + + @Override + public String toString() + { + return "SegmentWithState{" + + "segmentIdentifier=" + segmentIdentifier + + ", state=" + state + + '}'; + } + } + + // sequenceName -> {Interval Start millis -> List of Segments for this interval} + // there might be multiple segments for a start interval, for example one segment + // can be in ACTIVE state and others might be in PUBLISHING state + private final Map>> segments = new TreeMap<>(); + + private final Set publishingSequences = new HashSet<>(); // sequenceName -> most recently allocated segment private final Map lastSegmentIds = Maps.newHashMap(); @@ -103,12 +145,12 @@ public class AppenderatorDriver implements Closeable /** * Create a driver. * - * @param appenderator appenderator - * @param segmentAllocator segment allocator - * @param handoffNotifierFactory handoff notifier factory - * @param usedSegmentChecker used segment checker - * @param objectMapper object mapper, used for serde of commit metadata - * @param metrics Firedepartment metrics + * @param appenderator appenderator + * @param segmentAllocator segment allocator + * @param handoffNotifierFactory handoff notifier factory + * @param usedSegmentChecker used segment checker + * @param objectMapper object mapper, used for serde of commit metadata + * @param metrics Firedepartment metrics */ public AppenderatorDriver( Appenderator appenderator, @@ -130,20 +172,14 @@ public AppenderatorDriver( } @VisibleForTesting - Map> getActiveSegments() - { - return activeSegments; - } - - @VisibleForTesting - Map> getPublishPendingSegments() + Map>> getSegments() { - return publishPendingSegments; + return segments; } /** * Perform any initial setup and return currently persisted commit metadata. - * + *

* Note that this method returns the same metadata you've passed in with your Committers, even though this class * stores extra metadata on disk. * @@ -161,18 +197,29 @@ public Object startJob() log.info("Restored metadata[%s].", metadata); if (metadata != null) { - synchronized (activeSegments) { - for (Map.Entry> entry : metadata.getActiveSegments().entrySet()) { + synchronized (segments) { + for (Map.Entry> entry : metadata.getSegments().entrySet()) { final String sequenceName = entry.getKey(); - final TreeMap segmentMap = Maps.newTreeMap(); + final TreeMap> segmentMap = Maps.newTreeMap(); - activeSegments.put(sequenceName, segmentMap); + segments.put(sequenceName, segmentMap); - for (SegmentIdentifier identifier : entry.getValue()) { - segmentMap.put(identifier.getInterval().getStartMillis(), identifier); + for (SegmentWithState segmentWithState : entry.getValue()) { + segmentMap.computeIfAbsent( + segmentWithState.getSegmentIdentifier().getInterval().getStartMillis(), + k -> new LinkedList<>() + ); + LinkedList segmentList = segmentMap.get(segmentWithState.getSegmentIdentifier() + .getInterval() + .getStartMillis()); + // always keep the ACTIVE segment for an interval start millis in the front + if (segmentWithState.getState() == SegmentState.ACTIVE) { + segmentList.addFirst(segmentWithState); + } else { + segmentList.addLast(segmentWithState); + } } } - publishPendingSegments.putAll(metadata.getPublishPendingSegments()); lastSegmentIds.putAll(metadata.getLastSegmentIds()); } @@ -184,12 +231,10 @@ public Object startJob() private void addSegment(String sequenceName, SegmentIdentifier identifier) { - synchronized (activeSegments) { - activeSegments.computeIfAbsent(sequenceName, k -> new TreeMap<>()) - .putIfAbsent(identifier.getInterval().getStartMillis(), identifier); - - publishPendingSegments.computeIfAbsent(sequenceName, k -> new ArrayList<>()) - .add(identifier); + synchronized (segments) { + segments.computeIfAbsent(sequenceName, k -> new TreeMap<>()) + .computeIfAbsent(identifier.getInterval().getStartMillis(), k -> new LinkedList<>()) + .addFirst(new SegmentWithState(identifier, SegmentState.ACTIVE)); lastSegmentIds.put(sequenceName, identifier.getIdentifierAsString()); } } @@ -199,8 +244,8 @@ private void addSegment(String sequenceName, SegmentIdentifier identifier) */ public void clear() throws InterruptedException { - synchronized (activeSegments) { - activeSegments.clear(); + synchronized (segments) { + segments.clear(); } appenderator.clear(); } @@ -221,12 +266,22 @@ public AppenderatorDriverAddResult add( final String sequenceName, final Supplier committerSupplier ) throws IOException + { + return add(row, sequenceName, committerSupplier, false); + } + + public AppenderatorDriverAddResult add( + final InputRow row, + final String sequenceName, + final Supplier committerSupplier, + final boolean skipSegmentLineageCheck + ) throws IOException { Preconditions.checkNotNull(row, "row"); Preconditions.checkNotNull(sequenceName, "sequenceName"); Preconditions.checkNotNull(committerSupplier, "committerSupplier"); - final SegmentIdentifier identifier = getSegment(row, sequenceName); + final SegmentIdentifier identifier = getSegment(row, sequenceName, skipSegmentLineageCheck); if (identifier != null) { try { @@ -243,7 +298,7 @@ public AppenderatorDriverAddResult add( /** * Persist all data indexed through this driver so far. Blocks until complete. - * + *

* Should be called after all data has been added through {@link #add(InputRow, String, Supplier)}. * * @param committer committer representing all data that has been added so far @@ -275,8 +330,8 @@ public Object persist(final Committer committer) throws InterruptedException * {@link #publish(TransactionalSegmentPublisher, Committer, Collection)} * * @return null if the input segmentsAndMetadata is null. Otherwise, a {@link ListenableFuture} for the submitted task - * which returns {@link SegmentsAndMetadata} containing the segments successfully handed off and the metadata - * of the caller of {@link AppenderatorDriverMetadata} + * which returns {@link SegmentsAndMetadata} containing the segments successfully handed off and the metadata + * of the caller of {@link AppenderatorDriverMetadata} */ public ListenableFuture registerHandoff(SegmentsAndMetadata segmentsAndMetadata) { @@ -324,7 +379,7 @@ public ListenableFuture registerHandoff(SegmentsAndMetadata public void onSuccess(Object result) { if (numRemainingHandoffSegments.decrementAndGet() == 0) { - log.info("All segments handed off."); + log.info("Successfully handed off [%d] segments.", segmentsAndMetadata.getSegments().size()); resultFuture.set( new SegmentsAndMetadata( segmentsAndMetadata.getSegments(), @@ -364,16 +419,18 @@ public void close() private SegmentIdentifier getActiveSegment(final DateTime timestamp, final String sequenceName) { - synchronized (activeSegments) { - final NavigableMap activeSegmentsForSequence = activeSegments.get(sequenceName); + synchronized (segments) { + final NavigableMap> segmentsForSequence = segments.get(sequenceName); - if (activeSegmentsForSequence == null) { + if (segmentsForSequence == null) { return null; } - final Map.Entry candidateEntry = activeSegmentsForSequence.floorEntry(timestamp.getMillis()); - if (candidateEntry != null && candidateEntry.getValue().getInterval().contains(timestamp)) { - return candidateEntry.getValue(); + final Map.Entry> candidateEntry = segmentsForSequence.floorEntry(timestamp.getMillis()); + if (candidateEntry != null + && candidateEntry.getValue().getFirst().getSegmentIdentifier().getInterval().contains(timestamp) + && candidateEntry.getValue().getFirst().getState().equals(SegmentState.ACTIVE)) { + return candidateEntry.getValue().getFirst().getSegmentIdentifier(); } else { return null; } @@ -390,9 +447,13 @@ private SegmentIdentifier getActiveSegment(final DateTime timestamp, final Strin * * @throws IOException if an exception occurs while allocating a segment */ - private SegmentIdentifier getSegment(final InputRow row, final String sequenceName) throws IOException + private SegmentIdentifier getSegment( + final InputRow row, + final String sequenceName, + final boolean skipSegmentLineageCheck + ) throws IOException { - synchronized (activeSegments) { + synchronized (segments) { final DateTime timestamp = row.getTimestamp(); final SegmentIdentifier existing = getActiveSegment(timestamp, sequenceName); if (existing != null) { @@ -402,7 +463,11 @@ private SegmentIdentifier getSegment(final InputRow row, final String sequenceNa final SegmentIdentifier newSegment = segmentAllocator.allocate( row, sequenceName, - lastSegmentIds.get(sequenceName) + lastSegmentIds.get(sequenceName), + // send lastSegmentId irrespective of skipSegmentLineageCheck so that + // unique constraint for sequence_name_prev_id_sha1 does not fail for + // allocatePendingSegment in IndexerSQLMetadataStorageCoordinator + skipSegmentLineageCheck ); if (newSegment != null) { @@ -433,8 +498,8 @@ private SegmentIdentifier getSegment(final InputRow row, final String sequenceNa */ public void moveSegmentOut(final String sequenceName, final List identifiers) { - synchronized (activeSegments) { - final NavigableMap activeSegmentsForSequence = activeSegments.get(sequenceName); + synchronized (segments) { + final NavigableMap> activeSegmentsForSequence = segments.get(sequenceName); if (activeSegmentsForSequence == null) { throw new ISE("WTF?! Asked to remove segments for sequenceName[%s] which doesn't exist...", sequenceName); } @@ -442,7 +507,16 @@ public void moveSegmentOut(final String sequenceName, final List { + if (segmentWithState.getSegmentIdentifier().equals(identifier)) { + segmentWithState.state = SegmentState.INACTIVE; + return true; + } else { + return false; + } + } + )) { throw new ISE("WTF?! Asked to remove segment[%s] that didn't exist...", identifier); } } @@ -456,36 +530,30 @@ public void moveSegmentOut(final String sequenceName, final List publishAll( final TransactionalSegmentPublisher publisher, final Committer committer ) { - final List theSegments; - synchronized (activeSegments) { - final List sequenceNames = ImmutableList.copyOf(publishPendingSegments.keySet()); - theSegments = sequenceNames.stream() - .map(publishPendingSegments::remove) - .filter(Objects::nonNull) - .flatMap(Collection::stream) - .collect(Collectors.toList()); - sequenceNames.forEach(activeSegments::remove); + final List theSequences; + synchronized (segments) { + theSequences = ImmutableList.copyOf(segments.keySet()); } - return publish(publisher, wrapCommitter(committer), theSegments); + return publish(publisher, wrapCommitter(committer), theSequences); } /** * Execute a task in background to publish all segments corresponding to the given sequence names. The task * internally pushes the segments to the deep storage first, and then publishes the metadata to the metadata storage. * - * @param publisher segment publisher - * @param committer committer + * @param publisher segment publisher + * @param committer committer * @param sequenceNames a collection of sequence names to be published * * @return a {@link ListenableFuture} for the submitted task which removes published {@code sequenceNames} from - * {@code activeSegments} and {@code publishPendingSegments} + * {@code activeSegments} and {@code publishPendingSegments} */ public ListenableFuture publish( final TransactionalSegmentPublisher publisher, @@ -493,26 +561,62 @@ public ListenableFuture publish( final Collection sequenceNames ) { - final List theSegments; - synchronized (activeSegments) { - theSegments = sequenceNames.stream() - .map(publishPendingSegments::remove) - .filter(Objects::nonNull) - .flatMap(Collection::stream) - .collect(Collectors.toList()); - sequenceNames.forEach(activeSegments::remove); + final List theSegments = new ArrayList<>(); + synchronized (segments) { + sequenceNames.stream() + .filter(sequenceName -> !publishingSequences.contains(sequenceName)) + .forEach(sequenceName -> { + if (segments.containsKey(sequenceName)) { + segments.get(sequenceName) + .values() + .stream() + .flatMap(Collection::stream) + .forEach(segmentWithState -> { + segmentWithState.state = SegmentState.PUBLISHING; + theSegments.add(segmentWithState.getSegmentIdentifier()); + }); + } + }); } - return publish(publisher, wrapCommitter(committer), theSegments); + final ListenableFuture publishFuture = publish( + publisher, + wrapCommitter(committer), + theSegments + ); + + Futures.addCallback( + publishFuture, + new FutureCallback() + { + @Override + public void onSuccess(SegmentsAndMetadata result) + { + if (result != null) { + publishingSequences.removeAll(sequenceNames); + sequenceNames.forEach(segments::remove); + } + } + + @Override + public void onFailure(Throwable t) + { + // Do nothing, caller should handle the exception + log.error("Error publishing sequences [%s]", sequenceNames); + } + } + ); + + return publishFuture; } /** * Execute a task in background to publish the given segments. The task blocks until complete. * Retries forever on transient failures, but may exit early on permanent failures. - * + *

* Should be called after all data has been added through {@link #add(InputRow, String, Supplier)}. * - * @param publisher publisher to use for this set of segments + * @param publisher publisher to use for this set of segments * @param wrappedCommitter committer representing all data that has been added so far * * @return segments and metadata published if successful, or null if segments could not be handed off due to @@ -603,22 +707,15 @@ private Supplier wrapCommitterSupplier(final Supplier comm private WrappedCommitter wrapCommitter(final Committer committer) { final AppenderatorDriverMetadata wrappedMetadata; - synchronized (activeSegments) { + synchronized (segments) { wrappedMetadata = new AppenderatorDriverMetadata( ImmutableMap.copyOf( Maps.transformValues( - activeSegments, - new Function, List>() - { - @Override - public List apply(NavigableMap input) - { - return ImmutableList.copyOf(input.values()); - } - } + segments, + (Function>, List>) input -> ImmutableList + .copyOf(input.values().stream().flatMap(Collection::stream).collect(Collectors.toList())) ) ), - ImmutableMap.copyOf(publishPendingSegments), ImmutableMap.copyOf(lastSegmentIds), committer.getMetadata() ); diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverMetadata.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverMetadata.java index edff72572d9e..38a907255e10 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverMetadata.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverMetadata.java @@ -27,35 +27,26 @@ public class AppenderatorDriverMetadata { - private final Map> activeSegments; - private final Map> publishPendingSegments; + private final Map> segments; private final Map lastSegmentIds; private final Object callerMetadata; @JsonCreator public AppenderatorDriverMetadata( - @JsonProperty("activeSegments") Map> activeSegments, - @JsonProperty("publishPendingSegments") Map> publishPendingSegments, + @JsonProperty("segments") Map> segments, @JsonProperty("lastSegmentIds") Map lastSegmentIds, @JsonProperty("callerMetadata") Object callerMetadata ) { - this.activeSegments = activeSegments; - this.publishPendingSegments = publishPendingSegments; + this.segments = segments; this.lastSegmentIds = lastSegmentIds; this.callerMetadata = callerMetadata; } @JsonProperty - public Map> getActiveSegments() + public Map> getSegments() { - return activeSegments; - } - - @JsonProperty - public Map> getPublishPendingSegments() - { - return publishPendingSegments; + return segments; } @JsonProperty @@ -74,8 +65,7 @@ public Object getCallerMetadata() public String toString() { return "AppenderatorDriverMetadata{" + - "activeSegments=" + activeSegments + - ", publishPendingSegments=" + publishPendingSegments + + "segments=" + segments + ", lastSegmentIds=" + lastSegmentIds + ", callerMetadata=" + callerMetadata + '}'; diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index cb7f2cc1df7d..24e9c459845d 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -94,7 +94,9 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; /** */ @@ -129,6 +131,7 @@ public class AppenderatorImpl implements Appenderator private volatile long nextFlush; private volatile FileLock basePersistDirLock = null; private volatile FileChannel basePersistDirLockChannel = null; + private AtomicBoolean closed = new AtomicBoolean(false); public AppenderatorImpl( DataSchema schema, @@ -365,7 +368,7 @@ public ListenableFuture drop(final SegmentIdentifier identifier) @Override public ListenableFuture persist(Collection identifiers, Committer committer) { - final Map commitHydrants = Maps.newHashMap(); + final Map currentHydrants = Maps.newHashMap(); final List> indexesToPersist = Lists.newArrayList(); int numPersistedRows = 0; for (SegmentIdentifier identifier : identifiers) { @@ -374,7 +377,7 @@ public ListenableFuture persist(Collection identifier throw new ISE("No sink for identifier: %s", identifier); } final List hydrants = Lists.newArrayList(sink); - commitHydrants.put(identifier, hydrants.size()); + currentHydrants.put(identifier.getIdentifierAsString(), hydrants.size()); numPersistedRows += sink.getNumRowsInMemory(); final int limit = sink.isWritable() ? hydrants.size() - 1 : hydrants.size(); @@ -410,22 +413,28 @@ public Object doCall() log.info( "Committing metadata[%s] for sinks[%s].", commitMetadata, Joiner.on(", ").join( - Iterables.transform( - commitHydrants.entrySet(), - new Function, String>() - { - @Override - public String apply(Map.Entry entry) - { - return StringUtils.format("%s:%d", entry.getKey().getIdentifierAsString(), entry.getValue()); - } - } - ) + currentHydrants.entrySet() + .stream() + .map(entry -> StringUtils.format( + "%s:%d", + entry.getKey(), + entry.getValue() + )) + .collect(Collectors.toList()) ) ); committer.run(); - objectMapper.writeValue(computeCommitFile(), Committed.create(commitHydrants, commitMetadata)); + + final File commitFile = computeCommitFile(); + final Map commitHydrants = Maps.newHashMap(); + if (commitFile.exists()) { + // merge current hydrants with existing hydrants + final Committed oldCommitted = objectMapper.readValue(commitFile, Committed.class); + commitHydrants.putAll(oldCommitted.getHydrants()); + } + commitHydrants.putAll(currentHydrants); + objectMapper.writeValue(commitFile, new Committed(commitHydrants, commitMetadata)); return commitMetadata; } @@ -481,29 +490,24 @@ public ListenableFuture push( return Futures.transform( persist(identifiers, committer), - new Function() - { - @Override - public SegmentsAndMetadata apply(Object commitMetadata) - { - final List dataSegments = Lists.newArrayList(); + (Function) commitMetadata -> { + final List dataSegments = Lists.newArrayList(); - for (Map.Entry entry : theSinks.entrySet()) { - if (droppingSinks.contains(entry.getKey())) { - log.info("Skipping push of currently-dropping sink[%s]", entry.getKey()); - continue; - } - - final DataSegment dataSegment = mergeAndPush(entry.getKey(), entry.getValue()); - if (dataSegment != null) { - dataSegments.add(dataSegment); - } else { - log.warn("mergeAndPush[%s] returned null, skipping.", entry.getKey()); - } + for (Map.Entry entry : theSinks.entrySet()) { + if (droppingSinks.contains(entry.getKey())) { + log.info("Skipping push of currently-dropping sink[%s]", entry.getKey()); + continue; } - return new SegmentsAndMetadata(dataSegments, commitMetadata); + final DataSegment dataSegment = mergeAndPush(entry.getKey(), entry.getValue()); + if (dataSegment != null) { + dataSegments.add(dataSegment); + } else { + log.warn("mergeAndPush[%s] returned null, skipping.", entry.getKey()); + } } + + return new SegmentsAndMetadata(dataSegments, commitMetadata); }, pushExecutor ); @@ -632,6 +636,11 @@ private DataSegment mergeAndPush(final SegmentIdentifier identifier, final Sink @Override public void close() { + if (!closed.compareAndSet(false, true)) { + log.info("Appenderator already closed"); + return; + } + log.info("Shutting down..."); final List> futures = Lists.newArrayList(); @@ -652,8 +661,14 @@ public void close() try { shutdownExecutors(); - Preconditions.checkState(persistExecutor.awaitTermination(365, TimeUnit.DAYS), "persistExecutor not terminated"); - Preconditions.checkState(pushExecutor.awaitTermination(365, TimeUnit.DAYS), "pushExecutor not terminated"); + Preconditions.checkState( + persistExecutor == null || persistExecutor.awaitTermination(365, TimeUnit.DAYS), + "persistExecutor not terminated" + ); + Preconditions.checkState( + pushExecutor == null || pushExecutor.awaitTermination(365, TimeUnit.DAYS), + "pushExecutor not terminated" + ); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -664,6 +679,46 @@ public void close() unlockBasePersistDirectory(); } + /** + * Unannounce the segments and wait for outstanding persists to finish. + * Do not unlock base persist dir as we are not waiting for push executor to shut down + * relying on current JVM to shutdown to not cause any locking problem if the task is restored. + * In case when task is restored and current task is still active because of push executor (which it shouldn't be + * since push executor starts daemon threads) then the locking should fail and new task should fail to start. + * This also means that this method should only be called when task is shutting down. + */ + @Override + public void closeNow() + { + if (!closed.compareAndSet(false, true)) { + log.info("Appenderator already closed"); + return; + } + + log.info("Shutting down immediately..."); + for (Map.Entry entry : sinks.entrySet()) { + try { + segmentAnnouncer.unannounceSegment(entry.getValue().getSegment()); + } + catch (Exception e) { + log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) + .addData("identifier", entry.getKey().getIdentifierAsString()) + .emit(); + } + } + try { + shutdownExecutors(); + Preconditions.checkState( + persistExecutor == null || persistExecutor.awaitTermination(365, TimeUnit.DAYS), + "persistExecutor not terminated" + ); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ISE("Failed to shutdown executors during close()"); + } + } + private void lockBasePersistDirectory() { if (basePersistDirLock == null) { @@ -723,8 +778,12 @@ private void initializeExecutors() private void shutdownExecutors() { - persistExecutor.shutdownNow(); - pushExecutor.shutdownNow(); + if (persistExecutor != null) { + persistExecutor.shutdownNow(); + } + if (pushExecutor != null) { + pushExecutor.shutdownNow(); + } } private void resetNextFlush() diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Committed.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Committed.java index 14f13cdbe6a3..54961b6f59b6 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Committed.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Committed.java @@ -83,6 +83,14 @@ public Committed without(final String identifierAsString) return new Committed(newHydrants, metadata); } + public Committed with(final Map hydrantsToAdd) + { + final Map newHydrants = Maps.newHashMap(); + newHydrants.putAll(hydrants); + newHydrants.putAll(hydrantsToAdd); + return new Committed(newHydrants, metadata); + } + @Override public boolean equals(Object o) { diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java index 15b2934f72fa..f7abbd58d386 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java @@ -37,6 +37,7 @@ public interface SegmentAllocator SegmentIdentifier allocate( InputRow row, String sequenceName, - String previousSegmentId + String previousSegmentId, + boolean skipSegmentLineageCheck ) throws IOException; } diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java index e1237ef16966..44142ce3af69 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java @@ -419,6 +419,12 @@ public void close() } + @Override + public void closeNow() + { + + } + @Override public QueryRunner getQueryRunnerForIntervals( Query query, Iterable intervals diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverTest.java index d6430b6acff6..d9681351c77e 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverTest.java @@ -122,7 +122,7 @@ public void tearDown() throws Exception driver.close(); } - @Test + @Test(timeout = 2000L) public void testSimple() throws Exception { final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); @@ -139,8 +139,11 @@ public void testSimple() throws Exception committerSupplier.get(), ImmutableList.of("dummy") ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); - Assert.assertFalse(driver.getActiveSegments().containsKey("dummy")); - Assert.assertFalse(driver.getPublishPendingSegments().containsKey("dummy")); + + while (driver.getSegments().containsKey("dummy")) { + Thread.sleep(100); + } + final SegmentsAndMetadata segmentsAndMetadata = driver.registerHandoff(published) .get(HANDOFF_CONDITION_TIMEOUT, TimeUnit.MILLISECONDS); @@ -186,8 +189,11 @@ public void testMaxRowsPerSegment() throws Exception committerSupplier.get(), ImmutableList.of("dummy") ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); - Assert.assertFalse(driver.getActiveSegments().containsKey("dummy")); - Assert.assertFalse(driver.getPublishPendingSegments().containsKey("dummy")); + + while (driver.getSegments().containsKey("dummy")) { + Thread.sleep(100); + } + final SegmentsAndMetadata segmentsAndMetadata = driver.registerHandoff(published) .get(HANDOFF_CONDITION_TIMEOUT, TimeUnit.MILLISECONDS); Assert.assertEquals(numSegments, segmentsAndMetadata.getSegments().size()); @@ -212,8 +218,11 @@ public void testHandoffTimeout() throws Exception committerSupplier.get(), ImmutableList.of("dummy") ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); - Assert.assertFalse(driver.getActiveSegments().containsKey("dummy")); - Assert.assertFalse(driver.getPublishPendingSegments().containsKey("dummy")); + + while (driver.getSegments().containsKey("dummy")) { + Thread.sleep(100); + } + driver.registerHandoff(published).get(HANDOFF_CONDITION_TIMEOUT, TimeUnit.MILLISECONDS); } @@ -414,7 +423,8 @@ public TestSegmentAllocator(String dataSource, Granularity granularity) public SegmentIdentifier allocate( final InputRow row, final String sequenceName, - final String previousSegmentId + final String previousSegmentId, + final boolean skipSegmentLineageCheck ) throws IOException { synchronized (counters) { From 9ecd5d1f8c01ad39e8da939e14f5970060969b22 Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Sat, 21 Oct 2017 14:40:34 -0500 Subject: [PATCH 02/14] take care of review comments --- .../druid/indexing/kafka/KafkaIndexTask.java | 114 +++++++++--------- .../indexing/kafka/KafkaIndexTaskClient.java | 10 -- .../kafka/supervisor/KafkaSupervisor.java | 15 ++- .../kafka/KafkaIndexTaskClientTest.java | 6 +- .../indexing/kafka/KafkaIndexTaskTest.java | 5 +- 5 files changed, 70 insertions(+), 80 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 0088103f9490..2f608fdc9ecf 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -132,6 +132,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Condition; @@ -230,10 +231,10 @@ public enum Status private final List> handOffWaitList = new ArrayList<>(); private final CountDownLatch waitForPublishes = new CountDownLatch(1); private final AtomicReference throwableAtomicReference = new AtomicReference<>(); - private final ListeningExecutorService publishExecService; private final String topic; private volatile CopyOnWriteArrayList sequences; + private ListeningExecutorService publishExecService; @JsonCreator public KafkaIndexTask( @@ -270,7 +271,6 @@ public KafkaIndexTask( ))); this.topic = ioConfig.getStartPartitions().getTopic(); this.sequences = new CopyOnWriteArrayList<>(); - this.publishExecService = MoreExecutors.listeningDecorator(Execs.singleThreaded("publish-driver")); } @VisibleForTesting @@ -318,9 +318,9 @@ public KafkaIOConfig getIOConfig() return ioConfig; } - private void startExecutors() + private void createAndStartPublishExecutor() { - // start publish executor service + publishExecService = MoreExecutors.listeningDecorator(Execs.singleThreaded("publish-driver")); publishExecService.submit( (Runnable) () -> { while (true) { @@ -360,7 +360,7 @@ private void startExecutors() sequences.remove(sequenceMetadata); publishingSequences.remove(sequenceMetadata.getSequenceName()); try { - persistState(toolbox); + persistSequences(); } catch (IOException e) { log.error(e, "Unable to persist state, dying"); @@ -434,8 +434,8 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception } sequencesPersistFile = new File(toolbox.getTaskWorkDir(), "sequences.json"); - - restoreState(); + restoreSequences(); + log.info("Starting with sequences: %s", sequences); if (chatHandlerProvider.isPresent()) { log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); @@ -480,25 +480,23 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception appenderator = newAppenderator(fireDepartmentMetrics, toolbox); driver = newDriver(appenderator, toolbox, fireDepartmentMetrics); - startExecutors(); + createAndStartPublishExecutor(); final String topic = ioConfig.getStartPartitions().getTopic(); // Start up, set up initial offsets. final Object restoredMetadata = driver.startJob(); if (restoredMetadata == null) { - // Either all sequences are published and we were waiting for handoff - // or no persist has happened so far - Preconditions.checkState(sequences.size() == 0 || sequences.get(0).startOffsets.entrySet().stream().allMatch( + // no persist has happened so far + Preconditions.checkState(sequences.get(0).startOffsets.entrySet().stream().allMatch( partitionOffsetEntry -> Longs.compare( partitionOffsetEntry.getValue(), ioConfig.getStartPartitions() .getPartitionOffsetMap() .get(partitionOffsetEntry.getKey()) - ) >= 0 + ) == 0 ), "Sequence offsets are not compatible with start offsets of task"); - log.info("Setting next offsets to [%s]", sequences.size() == 0 ? endOffsets : sequences.get(0).startOffsets); - nextOffsets.putAll(sequences.size() == 0 ? endOffsets : sequences.get(0).startOffsets); + nextOffsets.putAll(sequences.get(0).startOffsets); } else { final Map restoredMetadataMap = (Map) restoredMetadata; final KafkaPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( @@ -523,6 +521,14 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ioConfig.getStartPartitions().getPartitionOffsetMap().keySet() ); } + // sequences size can 0 only when all sequences got published and task stopped before it could finish + // which is super rare + if (sequences.size() == 0 || sequences.get(sequences.size() - 1).isCheckpointed()) { + this.endOffsets.putAll(sequences.size() == 0 + ? nextOffsets + : sequences.get(sequences.size() - 1).getEndOffsets()); + log.info("End offsets changed to [%s]", endOffsets); + } } // Set up committer. @@ -553,7 +559,7 @@ public void run() }; // restart publishing of sequences (if any) - persistAndPublishSequences(committerSupplier); + mayBePersistAndPublishSequences(committerSupplier); Set assignment = assignPartitionsAndSeekToNext(consumer, topic); @@ -588,7 +594,7 @@ public void run() checkAndMaybeThrowException(); if (!ioConfig.isPauseAfterRead()) { - persistAndPublishSequences(committerSupplier); + mayBePersistAndPublishSequences(committerSupplier); } // The retrying business is because the KafkaConsumer throws OffsetOutOfRangeException if the seeked-to @@ -648,7 +654,7 @@ public void run() if (sequenceToUse == null) { throw new ISE( - "WTH?! cannot find any valid sequence for record with partition [%d] and offset [%d]. Current sequences: [%s]", + "WTH?! cannot find any valid sequence for record with partition [%d] and offset [%d]. Current sequences: %s", record.partition(), record.offset(), sequences @@ -720,11 +726,11 @@ public void run() sequences.get(sequences.size() - 1) .getSequenceName() .equals(sequenceToCheckpoint.getSequenceName()), - "Cannot checkpoint a sequence [%s] which is not the latest one, sequences [%s]", + "Cannot checkpoint a sequence [%s] which is not the latest one, sequences %s", sequenceToCheckpoint, sequences ); - pause(-1L); + pause(PAUSE_FOREVER); if (!toolbox.getTaskActionClient().submit(new CheckPointDataSourceMetadataAction( getDataSource(), ioConfig.getBaseSequenceName(), @@ -765,17 +771,24 @@ public void run() waitForPublishes.await(); checkAndMaybeThrowException(); - final List handedOffList; + List handedOffList = Lists.newArrayList(); if (tuningConfig.getHandoffConditionTimeout() == 0) { handedOffList = Futures.allAsList(handOffWaitList).get(); } else { - handedOffList = Futures.allAsList(handOffWaitList) - .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); + try { + handedOffList = Futures.allAsList(handOffWaitList) + .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); + } + catch (TimeoutException e) { + log.makeAlert("Timed out after [%d] millis waiting for handoffs", tuningConfig.getHandoffConditionTimeout()) + .addData("TaskId", this.getId()) + .emit(); + } } for (SegmentsAndMetadata handedOff : handedOffList) { if (handedOff == null) { - log.warn("Handoff failed for segments [%s]", handedOff.getSegments()); + log.warn("Handoff failed for segments %s", handedOff.getSegments()); } else { log.info( "Handoff completed for segments[%s] with metadata[%s].", @@ -788,7 +801,6 @@ public void run() } } catch (InterruptedException | RejectedExecutionException e) { - publishExecService.shutdownNow(); appenderator.closeNow(); // handle the InterruptedException that gets wrapped in a RejectedExecutionException if (e instanceof RejectedExecutionException @@ -818,6 +830,7 @@ public void run() if (chatHandlerProvider.isPresent()) { chatHandlerProvider.get().unregister(getId()); } + publishExecService.shutdownNow(); toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); toolbox.getDataSegmentServerAnnouncer().unannounce(); @@ -833,7 +846,7 @@ private void checkAndMaybeThrowException() } } - private void persistAndPublishSequences(Supplier committerSupplier) + private void mayBePersistAndPublishSequences(Supplier committerSupplier) throws ExecutionException, InterruptedException { for (SequenceMetadata sequenceMetadata : sequences) { @@ -842,8 +855,11 @@ private void persistAndPublishSequences(Supplier committerSupplier) publishingSequences.add(sequenceMetadata.getSequenceName()); try { Object result = driver.persist(committerSupplier.get()); - log.info("Persist completed with results [%s]", result); - log.info("Adding sequence to publish queue, [%s]", sequenceMetadata); + log.info( + "Persist completed with results: [%s], adding sequence [%s] to publish queue", + result, + sequenceMetadata + ); publishQueue.add(sequenceMetadata); } catch (InterruptedException e) { @@ -854,7 +870,7 @@ private void persistAndPublishSequences(Supplier committerSupplier) } } - private void restoreState() throws IOException + private void restoreSequences() throws IOException { Preconditions.checkNotNull(sequencesPersistFile); if (sequencesPersistFile.exists()) { @@ -862,15 +878,10 @@ private void restoreState() throws IOException sequencesPersistFile, new TypeReference>() { })); - log.info("Restored sequences [%s]", sequences); - } - if (sequences.get(sequences.size() - 1).isCheckpointed()) { - this.endOffsets.putAll(sequences.get(sequences.size() - 1).getEndOffsets()); - log.info("End offsets changed, sequences [%s]", sequences); } } - private synchronized void persistState(final TaskToolbox toolbox) throws IOException + private synchronized void persistSequences() throws IOException { log.info("Persisting Sequences Metadata [%s]", sequences); toolbox.getObjectMapper().writerWithType( @@ -1068,18 +1079,17 @@ public Response setEndOffsets( pauseLock.lockInterruptibly(); // Perform all sequence related checks before checking for isPaused() // and after acquiring pauseLock to correctly guard against duplicate requests - if (sequences.size() == 0) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("No Sequences found to set end offsets") - .build(); - } else if ((sequences.get(sequences.size() - 1).getStartOffsets().equals(offsets) && !finish) || - (sequences.get(sequences.size() - 1).getEndOffsets().equals(offsets) && finish)) { + Preconditions.checkState(sequences.size() > 0, "WTH?! No Sequences found to set end offsets"); + + final SequenceMetadata latestSequence = sequences.get(sequences.size() - 1); + if ((latestSequence.getStartOffsets().equals(offsets) && !finish) || + (latestSequence.getEndOffsets().equals(offsets) && finish)) { log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequences); - } else if (sequences.get(sequences.size() - 1).isCheckpointed() && !ioConfig.isPauseAfterRead()) { + } else if (latestSequence.isCheckpointed() && !ioConfig.isPauseAfterRead()) { return Response.status(Response.Status.BAD_REQUEST) .entity(StringUtils.format( "WTH?! Sequence [%s] has already endOffsets set, cannot set to [%s]", - sequences.get(sequences.size() - 1), + latestSequence, offsets )).build(); } else if (!isPaused()) { @@ -1102,8 +1112,7 @@ public Response setEndOffsets( } } - final SequenceMetadata sequenceMetadata = sequences.get(sequences.size() - 1); - sequenceMetadata.setEndOffsets(offsets); + latestSequence.setEndOffsets(offsets); if (finish) { log.info("Updating endOffsets from [%s] to [%s]", endOffsets, offsets); @@ -1111,17 +1120,17 @@ public Response setEndOffsets( } else { Preconditions.checkState(!ioConfig.isPauseAfterRead()); // create new sequence - final SequenceMetadata newSequenceMetadata = new SequenceMetadata( - sequenceMetadata.getSequenceId() + 1, - StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), sequenceMetadata.getSequenceId() + 1), + final SequenceMetadata newSequence = new SequenceMetadata( + latestSequence.getSequenceId() + 1, + StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), latestSequence.getSequenceId() + 1), offsets, maxEndOffsets, false ); - sequences.add(newSequenceMetadata); + sequences.add(newSequence); } - persistState(toolbox); + persistSequences(); } catch (Exception e) { log.error(e, "Unable to set end offsets, dying"); @@ -1609,15 +1618,10 @@ public boolean isSentinel() return sentinel; } - public void setCheckpointed() - { - checkpointed = true; - } - public void setEndOffsets(Map newEndOffsets) { endOffsets.putAll(newEndOffsets); - setCheckpointed(); + checkpointed = true; } public void updateAssignments(Map nextPartitionOffset) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java index 1d557719a6d8..f31ce542ce8b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -304,11 +304,6 @@ public Map getEndOffsets(final String id) } } - public boolean setEndOffsets(final String id, final Map endOffsets) - { - return setEndOffsets(id, endOffsets, false, true); - } - public boolean setEndOffsets(final String id, final Map endOffsets, final boolean resume, final boolean finalize) { log.debug("SetEndOffsets task[%s] endOffsets[%s] resume[%s] finalize[%s]", id, endOffsets, resume, finalize); @@ -435,11 +430,6 @@ public Map call() throws Exception ); } - public ListenableFuture setEndOffsetsAsync(final String id, final Map endOffsets) - { - return setEndOffsetsAsync(id, endOffsets, false, true); - } - public ListenableFuture setEndOffsetsAsync( final String id, final Map endOffsets, final boolean resume, final boolean finalize ) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 9c74c501685e..29859768ac18 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -782,9 +782,8 @@ private void killTaskGroupForPartitions(Set partitions) private void killTasksInGroup(TaskGroup taskGroup) { if (taskGroup != null) { - // kill all tasks in this task group for (String taskId : taskGroup.tasks.keySet()) { - log.info("Kill task [%s] in the task group", taskId); + log.info("Killing task [%s] in the task group", taskId); killTask(taskId); } } @@ -1115,7 +1114,7 @@ public Boolean apply(KafkaIndexTask.Status status) * 1. Makes sure the checkpoints information in the taskGroup is consistent with that of the tasks, if not kill * inconsistent tasks. * 2. truncates the checkpoints in the taskGroup corresponding to which segments have been published, so that any newly - * created tasks for the taskGroup, they start indexing from after the latest published offsets. + * created tasks for the taskGroup start indexing from after the latest published offsets. */ private void verifyAndMergeCheckpoints(final Integer groupId) { @@ -1182,7 +1181,6 @@ private void verifyAndMergeCheckpoints(final Integer groupId) taskGroup.sequenceOffsets.clear(); taskGroup.sequenceOffsets.putAll(latestCheckpoints); } else { - // kill task log.debug( "Adding task [%s] to kill list, checkpoints[%s], latestoffsets from DB [%s]", taskSequences.get(taskIndex).lhs, @@ -1484,13 +1482,14 @@ public Map apply(List> input) try { - if (endOffsets.equals(taskGroup.partitionOffsets)) { + if (endOffsets.equals(taskGroup.sequenceOffsets.lastEntry().getValue())) { log.warn( - "Not adding checkpoint [%s] as its offsets are same as the start offsets [%s] for the task group [%d]", + "Not adding checkpoint [%s] as its same as the start offsets [%s] of latest sequence for the task group [%d]", endOffsets, - taskGroup.partitionOffsets, + taskGroup.sequenceOffsets.lastEntry().getValue(), groupId ); + return endOffsets; } log.info("Setting endOffsets for tasks in taskGroup [%d] to %s and resuming", groupId, endOffsets); @@ -1724,7 +1723,7 @@ void createNewTasks() throws JsonProcessingException private void createKafkaTasksForGroup(int groupId, int replicas) throws JsonProcessingException { Map startPartitions = taskGroups.get(groupId).partitionOffsets; - Map endPartitions = new HashMap<>(); // TODO if endOffsets were already set + Map endPartitions = new HashMap<>(); for (Integer partition : startPartitions.keySet()) { endPartitions.put(partition, Long.MAX_VALUE); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java index 6464d5eb130c..d33cd50bc37a 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -143,7 +143,7 @@ public void testNoTaskLocation() throws Exception Assert.assertEquals(null, client.getStartTime(TEST_ID)); Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true)); Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID)); - Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of())); + Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of(), false, true)); Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of(), true, true)); verifyAll(); @@ -542,7 +542,7 @@ public void testSetEndOffsets() throws Exception ); replayAll(); - client.setEndOffsets(TEST_ID, endOffsets); + client.setEndOffsets(TEST_ID, endOffsets, false, true); verifyAll(); Request request = captured.getValue(); @@ -903,7 +903,7 @@ public void testSetEndOffsetsAsync() throws Exception List> futures = Lists.newArrayList(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, StringUtils.format("offsets/end?resume=%s&finish=%s", false, true)))); - futures.add(client.setEndOffsetsAsync(testId, endOffsets)); + futures.add(client.setEndOffsetsAsync(testId, endOffsets, false, true)); } List responses = Futures.allAsList(futures).get(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 7e110bc9a9cc..74ed173a657a 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -157,9 +157,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -//CHECKSTYLE.OFF: Regexp -//CHECKSTYLE.ON: Regexp - public class KafkaIndexTaskTest { private static final Logger log = new Logger(KafkaIndexTaskTest.class); @@ -709,7 +706,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio final ListenableFuture future = runTask(task); // Wait for task to exit - Assert.assertEquals(TaskStatus.Status.FAILED, future.get().getStatusCode()); + Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); // Check metrics Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); From 1597a6a50379ac47be75067a27b72fa9ce7ed874 Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Wed, 25 Oct 2017 11:03:56 -0500 Subject: [PATCH 03/14] make getCurrentOffsets call async, keep track of publishing sequence, review comments --- .../druid/indexing/kafka/KafkaIndexTask.java | 19 ++++--- .../indexing/kafka/KafkaIndexTaskClient.java | 11 +++- .../kafka/supervisor/KafkaSupervisor.java | 50 ++++++++++++------- .../appenderator/AppenderatorDriver.java | 1 + 4 files changed, 52 insertions(+), 29 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 2f608fdc9ecf..5ed3ef394628 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -114,7 +114,6 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -228,7 +227,7 @@ public enum Status private final Set publishingSequences = Sets.newConcurrentHashSet(); private final BlockingQueue publishQueue = new LinkedBlockingQueue<>(); - private final List> handOffWaitList = new ArrayList<>(); + private final List> handOffWaitList = new CopyOnWriteArrayList<>(); // to prevent concurrency visibility issue private final CountDownLatch waitForPublishes = new CountDownLatch(1); private final AtomicReference throwableAtomicReference = new AtomicReference<>(); private final String topic; @@ -730,7 +729,7 @@ public void run() sequenceToCheckpoint, sequences ); - pause(PAUSE_FOREVER); + requestPause(PAUSE_FOREVER); if (!toolbox.getTaskActionClient().submit(new CheckPointDataSourceMetadataAction( getDataSource(), ioConfig.getBaseSequenceName(), @@ -917,7 +916,7 @@ private Access authorizationCheck(final HttpServletRequest req, Action action) return access; } - // used for unit tests + @VisibleForTesting Appenderator getAppenderator() { return appenderator; @@ -1284,6 +1283,11 @@ private boolean isPaused() return status == Status.PAUSED; } + private void requestPause(long pauseMillis) { + this.pauseMillis = pauseMillis; + pauseRequested = true; + } + private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) { return Appenderators.createRealtime( @@ -1513,12 +1517,7 @@ private void sendResetRequestAndWait(Map outOfRangePartiti .addData("partitions", partitionOffsetMap.keySet()) .emit(); // wait for being killed by supervisor - try { - pause(-1); - } - catch (InterruptedException e) { - throw new RuntimeException("Got interrupted while pausing task"); - } + requestPause(PAUSE_FOREVER); } else { log.makeAlert("Failed to send reset request for partitions [%s]", partitionOffsetMap.keySet()).emit(); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java index f31ce542ce8b..3cfef84e2173 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -34,7 +34,6 @@ import com.metamx.http.client.Request; import com.metamx.http.client.response.FullResponseHandler; import com.metamx.http.client.response.FullResponseHolder; -import io.druid.java.util.common.concurrent.Execs; import io.druid.indexing.common.RetryPolicy; import io.druid.indexing.common.RetryPolicyConfig; import io.druid.indexing.common.RetryPolicyFactory; @@ -45,6 +44,7 @@ import io.druid.java.util.common.IOE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.segment.realtime.firehose.ChatHandlerResource; import org.jboss.netty.channel.ChannelException; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -85,6 +85,7 @@ public TaskNotRunnableException(String message) private static final EmittingLogger log = new EmittingLogger(KafkaIndexTaskClient.class); private static final String BASE_PATH = "/druid/worker/v1/chat"; private static final int TASK_MISMATCH_RETRY_DELAY_SECONDS = 5; + private static final TreeMap EMPTY_TREE_MAP = new TreeMap(); private final HttpClient httpClient; private final ObjectMapper jsonMapper; @@ -279,13 +280,19 @@ public TreeMap> getCheckpoints(final String id, fina return jsonMapper.readValue(response.getContent(), new TypeReference>>() {}); } catch (NoTaskLocationException e) { - return new TreeMap<>(); + return EMPTY_TREE_MAP; } catch (IOException e) { throw Throwables.propagate(e); } } + public ListenableFuture>> getCheckpointsAsync(final String id, final boolean retry) { + return executorService.submit( + () -> getCheckpoints(id, retry) + ); + } + public Map getEndOffsets(final String id) { log.debug("GetEndOffsets task[%s]", id); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 29859768ac18..ea53eabc0b79 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -38,6 +38,7 @@ import com.google.common.collect.Maps; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; +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.ListeningExecutorService; @@ -1122,26 +1123,41 @@ private void verifyAndMergeCheckpoints(final Integer groupId) // List {SequenceId, Checkpoints}> final List>>> taskSequences = new ArrayList<>(); + final List>>> futures = new ArrayList<>(); for (String taskId : taskGroup.taskIds()) { - TreeMap> checkpoints = null; - try { - checkpoints = taskClient.getCheckpoints( - taskId, - true - ); - } - catch (Exception e) { - log.error(e, "Exception while getting checkpoints for task [%s]", taskId); - } + final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync(taskId, true); + futures.add(checkpointsFuture); + Futures.addCallback( + checkpointsFuture, + new FutureCallback>>() + { + @Override + public void onSuccess(TreeMap> checkpoints) + { + if (!checkpoints.isEmpty()) { + taskSequences.add(new Pair<>(taskId, checkpoints)); + } else { + log.warn("Ignoring task [%s], as probably it is not started running yet", taskId); + } + } - if (checkpoints == null) { - log.error("Task [%s] returned no checkpoints, killing task", taskId); - killTask(taskId); - taskGroup.tasks.remove(taskId); - } else if (!checkpoints.isEmpty()) { - taskSequences.add(new Pair<>(taskId, checkpoints)); - } // otherwise task not started yet, continue + @Override + public void onFailure(Throwable t) + { + log.error(t, "Problem while getting checkpoints for task [%s], killing the task", taskId); + killTask(taskId); + taskGroup.tasks.remove(taskId); + } + } + ); + } + + try { + Futures.allAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + catch (Exception e) { + Throwables.propagate(e); } final KafkaDataSourceMetadata latestDataSourceMetadata = (KafkaDataSourceMetadata) indexerMetadataStorageCoordinator diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java index b9a4aaa63b70..f716985c8f2a 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java @@ -567,6 +567,7 @@ public ListenableFuture publish( .filter(sequenceName -> !publishingSequences.contains(sequenceName)) .forEach(sequenceName -> { if (segments.containsKey(sequenceName)) { + publishingSequences.add(sequenceName); segments.get(sequenceName) .values() .stream() From 04bc4515347d2a4f93735d2c56fbed3747c14106 Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Wed, 25 Oct 2017 11:27:04 -0500 Subject: [PATCH 04/14] fix setEndoffset duplicate request handling, formatting --- .../druid/indexing/kafka/KafkaIndexTask.java | 6 +++-- .../indexing/kafka/KafkaIndexTaskClient.java | 22 +++++++++++++++---- .../kafka/supervisor/KafkaSupervisor.java | 5 ++++- .../kafka/KafkaIndexTaskClientTest.java | 14 +++++++++--- 4 files changed, 37 insertions(+), 10 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 5ed3ef394628..63eed58cbf56 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -1084,6 +1084,7 @@ public Response setEndOffsets( if ((latestSequence.getStartOffsets().equals(offsets) && !finish) || (latestSequence.getEndOffsets().equals(offsets) && finish)) { log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequences); + return Response.ok(offsets).build(); } else if (latestSequence.isCheckpointed() && !ioConfig.isPauseAfterRead()) { return Response.status(Response.Status.BAD_REQUEST) .entity(StringUtils.format( @@ -1145,7 +1146,7 @@ public Response setEndOffsets( resume(); } - return Response.ok(endOffsets).build(); + return Response.ok(offsets).build(); } @GET @@ -1283,7 +1284,8 @@ private boolean isPaused() return status == Status.PAUSED; } - private void requestPause(long pauseMillis) { + private void requestPause(long pauseMillis) + { this.pauseMillis = pauseMillis; pauseRequested = true; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java index 3cfef84e2173..8e39da6b8f26 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -277,7 +277,9 @@ public TreeMap> getCheckpoints(final String id, fina log.debug("GetCheckpoints task[%s] retry[%s]", id, retry); try { final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "checkpoints", null, retry); - return jsonMapper.readValue(response.getContent(), new TypeReference>>() {}); + return jsonMapper.readValue(response.getContent(), new TypeReference>>() + { + }); } catch (NoTaskLocationException e) { return EMPTY_TREE_MAP; @@ -287,7 +289,11 @@ public TreeMap> getCheckpoints(final String id, fina } } - public ListenableFuture>> getCheckpointsAsync(final String id, final boolean retry) { + public ListenableFuture>> getCheckpointsAsync( + final String id, + final boolean retry + ) + { return executorService.submit( () -> getCheckpoints(id, retry) ); @@ -311,7 +317,12 @@ public Map getEndOffsets(final String id) } } - public boolean setEndOffsets(final String id, final Map endOffsets, final boolean resume, final boolean finalize) + public boolean setEndOffsets( + final String id, + final Map endOffsets, + final boolean resume, + final boolean finalize + ) { log.debug("SetEndOffsets task[%s] endOffsets[%s] resume[%s] finalize[%s]", id, endOffsets, resume, finalize); @@ -496,7 +507,10 @@ private FullResponseHolder submitRequest( Optional status = taskInfoProvider.getTaskStatus(id); if (!status.isPresent() || !status.get().isRunnable()) { - throw new TaskNotRunnableException(StringUtils.format("Aborting request because task [%s] is not runnable", id)); + throw new TaskNotRunnableException(StringUtils.format( + "Aborting request because task [%s] is not runnable", + id + )); } String host = location.getHost(); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index ea53eabc0b79..bed878edce3a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -1126,7 +1126,10 @@ private void verifyAndMergeCheckpoints(final Integer groupId) final List>>> futures = new ArrayList<>(); for (String taskId : taskGroup.taskIds()) { - final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync(taskId, true); + final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync( + taskId, + true + ); futures.add(checkpointsFuture); Futures.addCallback( checkpointsFuture, diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java index d33cd50bc37a..c4838a84a289 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -108,7 +108,8 @@ public void setUp() throws Exception headers = createMock(HttpHeaders.class); client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider); - expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)).anyTimes(); + expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); for (String testId : TEST_IDS) { @@ -153,7 +154,8 @@ public void testNoTaskLocation() throws Exception public void testTaskNotRunnableException() throws Exception { reset(taskInfoProvider); - expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)).anyTimes(); + expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.failure(TEST_ID))).anyTimes(); replayAll(); @@ -902,7 +904,13 @@ public void testSetEndOffsetsAsync() throws Exception List expectedUrls = Lists.newArrayList(); List> futures = Lists.newArrayList(); for (String testId : TEST_IDS) { - expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, StringUtils.format("offsets/end?resume=%s&finish=%s", false, true)))); + expectedUrls.add(new URL(StringUtils.format( + URL_FORMATTER, + TEST_HOST, + TEST_PORT, + testId, + StringUtils.format("offsets/end?resume=%s&finish=%s", false, true) + ))); futures.add(client.setEndOffsetsAsync(testId, endOffsets, false, true)); } From 2320517aa49de849d84b57d8742145b276467411 Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Wed, 25 Oct 2017 12:21:55 -0500 Subject: [PATCH 05/14] fix unit test --- .../kafka/supervisor/KafkaSupervisorTest.java | 46 +++++++++---------- 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index fc7e18f0799e..c754a740e83c 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -625,7 +625,7 @@ public void testKillIncompatibleTasks() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); - expect(taskClient.getCheckpoints(EasyMock.contains("id2"), anyBoolean())).andReturn(checkpoints).times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(2); replayAll(); @@ -718,8 +718,8 @@ public void testKillBadPartitionAssignment() throws Exception checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpoints(EasyMock.contains("id1"), anyBoolean())).andReturn(checkpoints1).times(1); - expect(taskClient.getCheckpoints(EasyMock.contains("id2"), anyBoolean())).andReturn(checkpoints2).times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1)).times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2)).times(1); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); taskQueue.shutdown("id4"); @@ -756,9 +756,9 @@ public void testRequeueTaskWhenFailed() throws Exception checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1)) .anyTimes(); - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2)) .anyTimes(); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); @@ -848,7 +848,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L, 2, 0L)); - expect(taskClient.getCheckpoints(EasyMock.contains("id1"), anyBoolean())).andReturn(checkpoints).times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(2); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); @@ -869,9 +869,9 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception reset(taskClient); // for the newly created replica task - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)) .times(2); - expect(taskClient.getCheckpoints(EasyMock.contains("id1"), anyBoolean())).andReturn(checkpoints).times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1); expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); @@ -944,9 +944,9 @@ public void testQueueNextTasksOnSuccess() throws Exception TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); // there would be 4 tasks, 2 for each task group - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1)) .times(2); - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2)) .times(2); expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); @@ -1055,9 +1055,9 @@ public void testBeginPublishAndQueueNextTasks() throws Exception checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1)) .times(2); - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2)) .times(2); replay(taskStorage, taskRunner, taskClient, taskQueue); @@ -1329,7 +1329,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception // since id1 is publishing, so getCheckpoints wouldn't be called for it TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); - expect(taskClient.getCheckpoints(EasyMock.contains("id2"), anyBoolean())).andReturn(checkpoints).times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1); replayAll(); @@ -1405,9 +1405,9 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1)) .times(2); - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2)) .times(2); expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); @@ -1464,9 +1464,9 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1)) .times(2); - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2)) .times(2); captured = Capture.newInstance(CaptureType.ALL); @@ -1541,9 +1541,9 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(checkpoints1) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1)) .times(2); - expect(taskClient.getCheckpoints(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(checkpoints2) + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2)) .times(2); captured = Capture.newInstance(CaptureType.ALL); @@ -1680,8 +1680,8 @@ public void testStopGracefully() throws Exception // getCheckpoints will not be called for id1 as it is in publishing state TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - expect(taskClient.getCheckpoints(EasyMock.contains("id2"), anyBoolean())).andReturn(checkpoints).times(1); - expect(taskClient.getCheckpoints(EasyMock.contains("id3"), anyBoolean())).andReturn(checkpoints).times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); @@ -1881,8 +1881,8 @@ public void testResetRunningTasks() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - expect(taskClient.getCheckpoints(EasyMock.contains("id2"), anyBoolean())).andReturn(checkpoints).times(1); - expect(taskClient.getCheckpoints(EasyMock.contains("id3"), anyBoolean())).andReturn(checkpoints).times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); From 53ca5dcf39c0ad428dad132868ec8a52bdddeb9f Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Wed, 1 Nov 2017 10:45:48 -0500 Subject: [PATCH 06/14] backward compatibility --- .../druid/indexing/kafka/KafkaIndexTask.java | 438 +++++++++++++++++- .../kafka/supervisor/KafkaSupervisor.java | 7 +- .../indexing/kafka/KafkaIndexTaskTest.java | 32 +- .../kafka/supervisor/KafkaSupervisorTest.java | 10 +- 4 files changed, 471 insertions(+), 16 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 63eed58cbf56..c89c7404db1d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Preconditions; @@ -32,6 +33,7 @@ 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.collect.Maps; import com.google.common.collect.Sets; @@ -59,6 +61,7 @@ import io.druid.indexing.common.task.AbstractTask; import io.druid.indexing.common.task.RealtimeIndexTask; import io.druid.indexing.common.task.TaskResource; +import io.druid.indexing.kafka.supervisor.KafkaSupervisor; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; @@ -128,6 +131,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; @@ -149,7 +153,9 @@ public enum Status STARTING, READING, PAUSED, - FINISHING + PUBLISHING + // ideally this should be called FINISHING now as the task does incremental publishes + // through out its lifetime } private static final EmittingLogger log = new EmittingLogger(KafkaIndexTask.class); @@ -234,6 +240,7 @@ public enum Status private volatile CopyOnWriteArrayList sequences; private ListeningExecutorService publishExecService; + private final boolean useLegacy; @JsonCreator public KafkaIndexTask( @@ -270,6 +277,13 @@ public KafkaIndexTask( ))); this.topic = ioConfig.getStartPartitions().getTopic(); this.sequences = new CopyOnWriteArrayList<>(); + + if (context != null && context.get(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED) != null + && ((boolean) context.get(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) { + useLegacy = false; + } else { + useLegacy = true; + } } @VisibleForTesting @@ -389,6 +403,11 @@ private void createAndStartPublishExecutor() @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { + // for backwards compatibility, should be remove from versions greater than 0.11.1 + if (useLegacy) { + return runLegacy(toolbox); + } + log.info("Starting up!"); startTime = DateTimes.nowUtc(); @@ -583,7 +602,7 @@ public void run() // if stop is requested or task's end offset is set by call to setEndOffsets method with finish set to true if (stopRequested.get() || (sequences.get(sequences.size() - 1).isCheckpointed() && !ioConfig.isPauseAfterRead())) { - status = Status.FINISHING; + status = Status.PUBLISHING; } if (stopRequested.get()) { @@ -751,7 +770,7 @@ public void run() throw new InterruptedException("Stopping without publishing"); } - status = Status.FINISHING; + status = Status.PUBLISHING; } for (SequenceMetadata sequenceMetadata : sequences) { @@ -838,6 +857,354 @@ public void run() return success(); } + private TaskStatus runLegacy(final TaskToolbox toolbox) throws Exception + { + log.info("Starting up!"); + startTime = DateTimes.nowUtc(); + status = Status.STARTING; + this.toolbox = toolbox; + + if (chatHandlerProvider.isPresent()) { + log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); + chatHandlerProvider.get().register(getId(), this, false); + } else { + log.warn("No chat handler detected"); + } + + runThread = Thread.currentThread(); + + // Set up FireDepartmentMetrics + final FireDepartment fireDepartmentForMetrics = new FireDepartment( + dataSchema, + new RealtimeIOConfig(null, null, null), + null + ); + fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + toolbox.getMonitorScheduler().addMonitor( + new RealtimeMetricsMonitor( + ImmutableList.of(fireDepartmentForMetrics), + ImmutableMap.of(DruidMetrics.TASK_ID, new String[]{getId()}) + ) + ); + + LookupNodeService lookupNodeService = getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER) == null ? + toolbox.getLookupNodeService() : + new LookupNodeService((String) getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER)); + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( + toolbox.getDruidNode(), + DruidNodeDiscoveryProvider.NODE_TYPE_PEON, + ImmutableMap.of( + toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), + lookupNodeService.getName(), lookupNodeService + ) + ); + + try ( + final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox); + final AppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics); + final KafkaConsumer consumer = newConsumer() + ) { + toolbox.getDataSegmentServerAnnouncer().announce(); + toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); + + appenderator = appenderator0; + + final String topic = ioConfig.getStartPartitions().getTopic(); + + // Start up, set up initial offsets. + final Object restoredMetadata = driver.startJob(); + if (restoredMetadata == null) { + nextOffsets.putAll(ioConfig.getStartPartitions().getPartitionOffsetMap()); + } else { + final Map restoredMetadataMap = (Map) restoredMetadata; + final KafkaPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( + restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), + KafkaPartitions.class + ); + nextOffsets.putAll(restoredNextPartitions.getPartitionOffsetMap()); + + // Sanity checks. + if (!restoredNextPartitions.getTopic().equals(ioConfig.getStartPartitions().getTopic())) { + throw new ISE( + "WTF?! Restored topic[%s] but expected topic[%s]", + restoredNextPartitions.getTopic(), + ioConfig.getStartPartitions().getTopic() + ); + } + + if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionOffsetMap().keySet())) { + throw new ISE( + "WTF?! Restored partitions[%s] but expected partitions[%s]", + nextOffsets.keySet(), + ioConfig.getStartPartitions().getPartitionOffsetMap().keySet() + ); + } + } + + // Set up sequenceNames. + final Map sequenceNames = Maps.newHashMap(); + for (Integer partitionNum : nextOffsets.keySet()) { + sequenceNames.put(partitionNum, StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), partitionNum)); + } + + // Set up committer. + final Supplier committerSupplier = new Supplier() + { + @Override + public Committer get() + { + final Map snapshot = ImmutableMap.copyOf(nextOffsets); + + return new Committer() + { + @Override + public Object getMetadata() + { + return ImmutableMap.of( + METADATA_NEXT_PARTITIONS, new KafkaPartitions( + ioConfig.getStartPartitions().getTopic(), + snapshot + ) + ); + } + + @Override + public void run() + { + // Do nothing. + } + }; + } + }; + + Set assignment = assignPartitionsAndSeekToNext(consumer, topic); + + // Main loop. + // Could eventually support leader/follower mode (for keeping replicas more in sync) + boolean stillReading = !assignment.isEmpty(); + status = Status.READING; + try { + while (stillReading) { + if (possiblyPause(assignment)) { + // The partition assignments may have changed while paused by a call to setEndOffsets() so reassign + // partitions upon resuming. This is safe even if the end offsets have not been modified. + assignment = assignPartitionsAndSeekToNext(consumer, topic); + + if (assignment.isEmpty()) { + log.info("All partitions have been fully read"); + publishOnStop.set(true); + stopRequested.set(true); + } + } + + if (stopRequested.get()) { + break; + } + + // The retrying business is because the KafkaConsumer throws OffsetOutOfRangeException if the seeked-to + // offset is not present in the topic-partition. This can happen if we're asking a task to read from data + // that has not been written yet (which is totally legitimate). So let's wait for it to show up. + ConsumerRecords records = ConsumerRecords.empty(); + try { + records = consumer.poll(POLL_TIMEOUT); + } + catch (OffsetOutOfRangeException e) { + log.warn("OffsetOutOfRangeException with message [%s]", e.getMessage()); + possiblyResetOffsetsOrWait(e.offsetOutOfRangePartitions(), consumer, toolbox); + stillReading = ioConfig.isPauseAfterRead() || !assignment.isEmpty(); + } + + for (ConsumerRecord record : records) { + if (log.isTraceEnabled()) { + log.trace( + "Got topic[%s] partition[%d] offset[%,d].", + record.topic(), + record.partition(), + record.offset() + ); + } + + if (record.offset() < endOffsets.get(record.partition())) { + if (record.offset() != nextOffsets.get(record.partition())) { + if (ioConfig.isSkipOffsetGaps()) { + log.warn( + "Skipped to offset[%,d] after offset[%,d] in partition[%d].", + record.offset(), + nextOffsets.get(record.partition()), + record.partition() + ); + } else { + throw new ISE( + "WTF?! Got offset[%,d] after offset[%,d] in partition[%d].", + record.offset(), + nextOffsets.get(record.partition()), + record.partition() + ); + } + } + + try { + final byte[] valueBytes = record.value(); + final InputRow row = valueBytes == null ? null : parser.parse(ByteBuffer.wrap(valueBytes)); + + if (row != null && withinMinMaxRecordTime(row)) { + final String sequenceName = sequenceNames.get(record.partition()); + final AppenderatorDriverAddResult addResult = driver.add( + row, + sequenceName, + committerSupplier + ); + + if (addResult.isOk()) { + // If the number of rows in the segment exceeds the threshold after adding a row, + // move the segment out from the active segments of AppenderatorDriver to make a new segment. + if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) { + driver.moveSegmentOut(sequenceName, ImmutableList.of(addResult.getSegmentIdentifier())); + } + } 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]", row.getTimestamp()); + } + + fireDepartmentMetrics.incrementProcessed(); + } else { + fireDepartmentMetrics.incrementThrownAway(); + } + } + catch (ParseException e) { + if (tuningConfig.isReportParseExceptions()) { + throw e; + } else { + log.debug( + e, + "Dropping unparseable row from partition[%d] offset[%,d].", + record.partition(), + record.offset() + ); + + fireDepartmentMetrics.incrementUnparseable(); + } + } + + nextOffsets.put(record.partition(), record.offset() + 1); + } + + if (nextOffsets.get(record.partition()).equals(endOffsets.get(record.partition())) + && assignment.remove(record.partition())) { + log.info("Finished reading topic[%s], partition[%,d].", record.topic(), record.partition()); + assignPartitions(consumer, topic, assignment); + stillReading = ioConfig.isPauseAfterRead() || !assignment.isEmpty(); + } + } + } + } + finally { + driver.persist(committerSupplier.get()); // persist pending data + } + + synchronized (statusLock) { + if (stopRequested.get() && !publishOnStop.get()) { + throw new InterruptedException("Stopping without publishing"); + } + + status = Status.PUBLISHING; + } + + final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { + final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue( + ((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS), + KafkaPartitions.class + ); + + // Sanity check, we should only be publishing things that match our desired end state. + if (!endOffsets.equals(finalPartitions.getPartitionOffsetMap())) { + throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); + } + + final SegmentTransactionalInsertAction action; + + if (ioConfig.isUseTransaction()) { + action = new SegmentTransactionalInsertAction( + segments, + new KafkaDataSourceMetadata(ioConfig.getStartPartitions()), + new KafkaDataSourceMetadata(finalPartitions) + ); + } else { + action = new SegmentTransactionalInsertAction(segments, null, null); + } + + log.info("Publishing with isTransaction[%s].", ioConfig.isUseTransaction()); + + return toolbox.getTaskActionClient().submit(action).isSuccess(); + }; + + // Supervised kafka tasks are killed by KafkaSupervisor if they are stuck during publishing segments or waiting + // for hand off. See KafkaSupervisorIOConfig.completionTimeout. + final SegmentsAndMetadata published = driver.publish( + publisher, + committerSupplier.get(), + sequenceNames.values() + ).get(); + + final Future handoffFuture = driver.registerHandoff(published); + final SegmentsAndMetadata handedOff; + if (tuningConfig.getHandoffConditionTimeout() == 0) { + handedOff = handoffFuture.get(); + } else { + handedOff = handoffFuture.get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); + } + + if (handedOff == null) { + throw new ISE("Transaction failure publishing segments, aborting"); + } else { + log.info( + "Published segments[%s] with metadata[%s].", + Joiner.on(", ").join( + Iterables.transform( + handedOff.getSegments(), + new Function() + { + @Override + public String apply(DataSegment input) + { + return input.getIdentifier(); + } + } + ) + ), + handedOff.getCommitMetadata() + ); + } + } + catch (InterruptedException | RejectedExecutionException e) { + // handle the InterruptedException that gets wrapped in a RejectedExecutionException + if (e instanceof RejectedExecutionException + && (e.getCause() == null || !(e.getCause() instanceof InterruptedException))) { + throw e; + } + + // if we were interrupted because we were asked to stop, handle the exception and return success, else rethrow + if (!stopRequested.get()) { + Thread.currentThread().interrupt(); + throw e; + } + + log.info("The task was asked to stop before completing"); + } + finally { + if (chatHandlerProvider.isPresent()) { + chatHandlerProvider.get().unregister(getId()); + } + + toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); + toolbox.getDataSegmentServerAnnouncer().unannounce(); + } + + return success(); + } + private void checkAndMaybeThrowException() { if (throwableAtomicReference.get() != null) { @@ -929,7 +1296,7 @@ public void stopGracefully() stopRequested.set(true); synchronized (statusLock) { - if (status == Status.FINISHING) { + if (status == Status.PUBLISHING) { runThread.interrupt(); return; } @@ -1060,6 +1427,11 @@ public Response setEndOffsets( final boolean finish // this field is only for internal purposes, shouldn't be usually set by users ) throws InterruptedException { + // for backwards compatibility, should be remove from versions greater than 0.11.1 + if (useLegacy) { + return setEndOffsetsLegacy(offsets, resume); + } + if (offsets == null) { return Response.status(Response.Status.BAD_REQUEST) .entity("Request body must contain a map of { partition:endOffset }") @@ -1149,6 +1521,62 @@ public Response setEndOffsets( return Response.ok(offsets).build(); } + private Response setEndOffsetsLegacy( + Map offsets, + final boolean resume + ) throws InterruptedException + { + if (offsets == null) { + return Response.status(Response.Status.BAD_REQUEST) + .entity("Request body must contain a map of { partition:endOffset }") + .build(); + } else if (!endOffsets.keySet().containsAll(offsets.keySet())) { + return Response.status(Response.Status.BAD_REQUEST) + .entity( + StringUtils.format( + "Request contains partitions not being handled by this task, my partitions: %s", + endOffsets.keySet() + ) + ) + .build(); + } + + pauseLock.lockInterruptibly(); + try { + if (!isPaused()) { + return Response.status(Response.Status.BAD_REQUEST) + .entity("Task must be paused before changing the end offsets") + .build(); + } + + for (Map.Entry entry : offsets.entrySet()) { + if (entry.getValue().compareTo(nextOffsets.get(entry.getKey())) < 0) { + return Response.status(Response.Status.BAD_REQUEST) + .entity( + StringUtils.format( + "End offset must be >= current offset for partition [%s] (current: %s)", + entry.getKey(), + nextOffsets.get(entry.getKey()) + ) + ) + .build(); + } + } + + endOffsets.putAll(offsets); + log.info("endOffsets changed to %s", endOffsets); + } + finally { + pauseLock.unlock(); + } + + if (resume) { + resume(); + } + + return Response.ok(endOffsets).build(); + } + @GET @Path("/checkpoints") @Produces(MediaType.APPLICATION_JSON) @@ -1706,7 +2134,7 @@ public Object getMetadata() )); } - // Publish metadata can be different from persist metadata as we are giong to publish only + // Publish metadata can be different from persist metadata as we are going to publish only // subset of segments return ImmutableMap.of(METADATA_NEXT_PARTITIONS, new KafkaPartitions(topic, lastPersistedOffsets), METADATA_PUBLISH_PARTITIONS, new KafkaPartitions(topic, endOffsets) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index bed878edce3a..1d6f7a90d58a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -130,6 +130,8 @@ public class KafkaSupervisor implements Supervisor private static final long INITIAL_EMIT_LAG_METRIC_DELAY_MILLIS = 25000; private static final CopyOnWriteArrayList EMPTY_LIST = Lists.newCopyOnWriteArrayList(); + public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED"; + // Internal data structures // -------------------------------------------------------- @@ -999,7 +1001,7 @@ public Boolean apply(KafkaIndexTask.Status status) { try { log.debug("Task [%s], status [%s]", taskId, status); - if (status == KafkaIndexTask.Status.FINISHING) { + if (status == KafkaIndexTask.Status.PUBLISHING) { kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap().keySet().forEach( partition -> addDiscoveredTaskToPendingCompletionTaskGroups( getTaskGroupIdForPartition(partition), @@ -1770,9 +1772,10 @@ private void createKafkaTasksForGroup(int groupId, int replicas) throws JsonProc { }).writeValueAsString(taskGroups.get(groupId).sequenceOffsets); final Map context = spec.getContext() == null - ? ImmutableMap.of("checkpoints", checkpoints) + ? ImmutableMap.of("checkpoints", checkpoints, IS_INCREMENTAL_HANDOFF_SUPPORTED, true) : ImmutableMap.builder() .put("checkpoints", checkpoints) + .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true) .putAll(spec.getContext()) .build(); for (int i = 0; i < replicas; i++) { diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 74ed173a657a..962b354fc470 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -62,6 +62,7 @@ import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskStorageConfig; import io.druid.indexing.common.task.Task; +import io.druid.indexing.kafka.supervisor.KafkaSupervisor; import io.druid.indexing.kafka.test.TestBroker; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import io.druid.indexing.overlord.MetadataTaskStorage; @@ -142,6 +143,8 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.File; import java.io.IOException; @@ -157,6 +160,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +@RunWith(Parameterized.class) public class KafkaIndexTaskTest { private static final Logger log = new Logger(KafkaIndexTaskTest.class); @@ -182,6 +186,18 @@ public class KafkaIndexTaskTest private File directory; private String topic; private List> records; + private final boolean isIncrementalHandoffSupported; + + @Parameterized.Parameters(name = "isIncrementalHandoffSupported = {0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of(new Object[]{true}, new Object[]{false}); + } + + public KafkaIndexTaskTest(boolean isIncrementalHandoffSupported) + { + this.isIncrementalHandoffSupported = isIncrementalHandoffSupported; + } private static final DataSchema DATA_SCHEMA = new DataSchema( "test_ds", @@ -706,7 +722,10 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio final ListenableFuture future = runTask(task); // Wait for task to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals( + isIncrementalHandoffSupported ? TaskStatus.Status.SUCCESS : TaskStatus.Status.FAILED, + future.get().getStatusCode() + ); // Check metrics Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); @@ -1027,7 +1046,7 @@ public void testRunOneTaskTwoPartitions() throws Exception // Check published segments & metadata SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); - SegmentDescriptor desc3 = SD(task, "2011/P1D", 0); + SegmentDescriptor desc3 = SD(task, "2011/P1D", isIncrementalHandoffSupported ? 0 : 1); SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals( @@ -1041,7 +1060,9 @@ public void testRunOneTaskTwoPartitions() throws Exception // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically Assert.assertEquals( - ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), + isIncrementalHandoffSupported + ? ImmutableSet.of(ImmutableList.of("d", "e", "h")) + : ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) ); } @@ -1531,13 +1552,16 @@ private KafkaIndexTask createTask( handoffConditionTimeout, resetOffsetAutomatically ); + final Map context = isIncrementalHandoffSupported + ? ImmutableMap.of(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true) + : null; final KafkaIndexTask task = new KafkaIndexTask( taskId, null, cloneDataSchema(dataSchema), tuningConfig, ioConfig, - null, + context, null, null ); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index c754a740e83c..347813858c65 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -1115,7 +1115,7 @@ public void testDiscoverExistingPublishingTask() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.FINISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); @@ -1210,7 +1210,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.FINISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); @@ -1315,7 +1315,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.FINISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getCurrentOffsetsAsync("id1", false)) @@ -1670,7 +1670,7 @@ public void testStopGracefully() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.FINISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); @@ -1872,7 +1872,7 @@ public void testResetRunningTasks() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.FINISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); From 06c86ab289db0d7a2b99195a20af5f782cc6f306 Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Thu, 2 Nov 2017 14:53:53 -0500 Subject: [PATCH 07/14] make AppenderatorDriverMetadata backwards compatible --- .../appenderator/AppenderatorDriver.java | 2 +- .../AppenderatorDriverMetadata.java | 68 ++++++++++++++++++- 2 files changed, 67 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java index f716985c8f2a..b09380360003 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java @@ -86,7 +86,7 @@ public class AppenderatorDriver implements Closeable private final ObjectMapper objectMapper; private final FireDepartmentMetrics metrics; - private enum SegmentState + enum SegmentState { ACTIVE, INACTIVE, diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverMetadata.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverMetadata.java index 38a907255e10..7a8688ada8d2 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverMetadata.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverMetadata.java @@ -21,9 +21,15 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; public class AppenderatorDriverMetadata { @@ -35,14 +41,72 @@ public class AppenderatorDriverMetadata public AppenderatorDriverMetadata( @JsonProperty("segments") Map> segments, @JsonProperty("lastSegmentIds") Map lastSegmentIds, - @JsonProperty("callerMetadata") Object callerMetadata + @JsonProperty("callerMetadata") Object callerMetadata, + // Next two properties are for backwards compatibility, should be removed on versions greater than 0.11.1 + @JsonProperty("activeSegments") Map> activeSegments, + @JsonProperty("publishPendingSegments") Map> publishPendingSegments ) { - this.segments = segments; + Preconditions.checkState( + segments != null || (activeSegments != null && publishPendingSegments != null), + "Metadata should either have segments with state information or both active segments and publish pending segments information. " + + "segments [%s], activeSegments [%s], publishPendingSegments [%s]", + segments, + activeSegments, + publishPendingSegments + ); + if (segments == null) { + // convert old metadata to new one + final Map> newMetadata = Maps.newHashMap(); + final Set activeSegmentsAlreadySeen = Sets.newHashSet(); // temp data structure + + activeSegments.entrySet() + .forEach(sequenceSegments -> newMetadata.put( + sequenceSegments.getKey(), + sequenceSegments.getValue() + .stream() + .map(segmentIdentifier -> { + activeSegmentsAlreadySeen.add(segmentIdentifier.toString()); + return new AppenderatorDriver.SegmentWithState( + segmentIdentifier, + AppenderatorDriver.SegmentState.ACTIVE + ); + }) + .collect(Collectors.toList()) + )); + // publishPendingSegments is a superset of activeSegments + publishPendingSegments.entrySet() + .forEach(sequenceSegments -> newMetadata.computeIfAbsent( + sequenceSegments.getKey(), + k -> new ArrayList<>() + ).addAll( + sequenceSegments.getValue() + .stream() + .filter(segmentIdentifier -> !activeSegmentsAlreadySeen.contains( + segmentIdentifier.toString())) + .map(segmentIdentifier -> new AppenderatorDriver.SegmentWithState( + segmentIdentifier, + AppenderatorDriver.SegmentState.INACTIVE + )) + .collect(Collectors.toList()) + )); + this.segments = newMetadata; + } else { + this.segments = segments; + } this.lastSegmentIds = lastSegmentIds; this.callerMetadata = callerMetadata; } + public AppenderatorDriverMetadata( + Map> segments, + Map lastSegmentIds, + Object callerMetadata + ) + { + this(segments, lastSegmentIds, callerMetadata, null, null); + } + @JsonProperty public Map> getSegments() { From 4d9194eb5be61a255c9fdc8bb69d40e5ee6e1612 Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Mon, 6 Nov 2017 16:34:10 -0600 Subject: [PATCH 08/14] add unit test --- .../druid/indexing/kafka/KafkaIndexTask.java | 2 +- .../indexing/kafka/KafkaIndexTaskTest.java | 201 +++++++++++++----- .../appenderator/AppenderatorImpl.java | 2 +- 3 files changed, 146 insertions(+), 59 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index c89c7404db1d..c83add3720f5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -1427,7 +1427,7 @@ public Response setEndOffsets( final boolean finish // this field is only for internal purposes, shouldn't be usually set by users ) throws InterruptedException { - // for backwards compatibility, should be remove from versions greater than 0.11.1 + // for backwards compatibility, should be removed from versions greater than 0.11.1 if (useLegacy) { return setEndOffsetsLegacy(offsets, resume); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 962b354fc470..7e1acce1a990 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -33,6 +33,7 @@ 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.io.Files; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -64,6 +65,7 @@ import io.druid.indexing.common.task.Task; import io.druid.indexing.kafka.supervisor.KafkaSupervisor; import io.druid.indexing.kafka.test.TestBroker; +import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import io.druid.indexing.overlord.MetadataTaskStorage; import io.druid.indexing.overlord.TaskLockbox; @@ -146,6 +148,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.lang.reflect.InvocationTargetException; @@ -153,6 +156,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.Executor; @@ -177,7 +181,9 @@ public class KafkaIndexTaskTest private long handoffConditionTimeout = 0; private boolean reportParseExceptions = false; + private boolean resetOffsetAutomatically = false; private boolean doHandoff = true; + private Integer maxRowsPerSegment = null; private TaskToolboxFactory toolboxFactory; private IndexerMetadataStorageCoordinator metadataStorageCoordinator; @@ -187,7 +193,10 @@ public class KafkaIndexTaskTest private String topic; private List> records; private final boolean isIncrementalHandoffSupported; + private final Set checkpointRequestsHash = Sets.newHashSet(); + // This should be removed in versions greater that 0.11.1 + // isIncrementalHandoffSupported should always be set to true in those later versions @Parameterized.Parameters(name = "isIncrementalHandoffSupported = {0}") public static Iterable constructorFeeder() { @@ -341,8 +350,7 @@ public void testRunAfterDataInserted() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -384,8 +392,7 @@ public void testRunBeforeDataInserted() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -424,6 +431,86 @@ public void testRunBeforeDataInserted() throws Exception Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } + @Test(timeout = 60_000L) + public void testIncrementalHandOff() throws Exception + { + if (!isIncrementalHandoffSupported) { + return; + } + final String baseSequenceName = "sequence0"; + maxRowsPerSegment = 1; + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + } + Map consumerProps = kafkaServer.consumerProperties(); + consumerProps.put("max.poll.records", "1"); + + final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L)); + final KafkaPartitions checkpointPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 0L)); + final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 8L, 1, 2L)); + final KafkaIndexTask task = createTask( + null, + new KafkaIOConfig( + baseSequenceName, + startPartitions, + endPartitions, + consumerProps, + true, + false, + null, + null, + false + ) + ); + final ListenableFuture future = runTask(task); + while (task.getStatus() != KafkaIndexTask.Status.PAUSED) { + Thread.sleep(10); + } + task.setEndOffsets(checkpointPartitions.getPartitionOffsetMap(), true, false); + Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + + Assert.assertEquals(1, checkpointRequestsHash.size()); + Assert.assertTrue(checkpointRequestsHash.contains( + Objects.hash(DATA_SCHEMA.getDataSource(), + baseSequenceName, + new KafkaDataSourceMetadata(startPartitions), + new KafkaDataSourceMetadata(checkpointPartitions) + ) + )); + + // Check metrics + Assert.assertEquals(8, task.getFireDepartmentMetrics().processed()); + Assert.assertEquals(1, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(1, task.getFireDepartmentMetrics().thrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); + SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc4 = SD(task, "2011/P1D", 0); + SegmentDescriptor desc5 = SD(task, "2011/P1D", 1); + SegmentDescriptor desc6 = SD(task, "2012/P1D", 0); + SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 8L, 1, 2L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc4)); + Assert.assertEquals(ImmutableList.of("h"), readSegmentColumn("dim1", desc5)); + Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); + } + @Test(timeout = 60_000L) public void testRunWithMinimumMessageTime() throws Exception { @@ -439,8 +526,7 @@ public void testRunWithMinimumMessageTime() throws Exception DateTimes.of("2010"), null, false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -494,8 +580,7 @@ public void testRunWithMaximumMessageTime() throws Exception null, DateTimes.of("2010"), false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -559,8 +644,7 @@ public void testRunWithTransformSpec() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -620,8 +704,7 @@ public void testRunOnNothing() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -662,8 +745,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -715,8 +797,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio null, null, false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -770,8 +851,7 @@ public void testReportParseExceptions() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -804,8 +884,7 @@ public void testRunReplicas() throws Exception null, null, false - ), - null + ) ); final KafkaIndexTask task2 = createTask( null, @@ -819,8 +898,7 @@ public void testRunReplicas() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future1 = runTask(task1); @@ -874,8 +952,7 @@ public void testRunConflicting() throws Exception null, null, false - ), - null + ) ); final KafkaIndexTask task2 = createTask( null, @@ -889,8 +966,7 @@ public void testRunConflicting() throws Exception null, null, false - ), - null + ) ); // Insert data @@ -945,8 +1021,7 @@ public void testRunConflictingWithoutTransactions() throws Exception null, null, false - ), - null + ) ); final KafkaIndexTask task2 = createTask( null, @@ -960,8 +1035,7 @@ public void testRunConflictingWithoutTransactions() throws Exception null, null, false - ), - null + ) ); // Insert data @@ -1021,8 +1095,7 @@ public void testRunOneTaskTwoPartitions() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -1046,9 +1119,12 @@ public void testRunOneTaskTwoPartitions() throws Exception // Check published segments & metadata SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); - SegmentDescriptor desc3 = SD(task, "2011/P1D", isIncrementalHandoffSupported ? 0 : 1); + // desc3 will not be created in KafkaIndexTask (0.11.1) as it does not create per Kafka partition Druid segments + SegmentDescriptor desc3 = SD(task, "2011/P1D", 1); SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + Assert.assertEquals(isIncrementalHandoffSupported + ? ImmutableSet.of(desc1, desc2, desc4) + : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals( new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1063,7 +1139,9 @@ public void testRunOneTaskTwoPartitions() throws Exception isIncrementalHandoffSupported ? ImmutableSet.of(ImmutableList.of("d", "e", "h")) : ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), - ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) + isIncrementalHandoffSupported + ? ImmutableSet.of(readSegmentColumn("dim1", desc2)) + : ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) ); } @@ -1082,8 +1160,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, null, false - ), - null + ) ); final KafkaIndexTask task2 = createTask( null, @@ -1097,8 +1174,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future1 = runTask(task1); @@ -1154,8 +1230,7 @@ public void testRestore() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future1 = runTask(task1); @@ -1192,8 +1267,7 @@ public void testRestore() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future2 = runTask(task2); @@ -1245,8 +1319,7 @@ public void testRunWithPauseAndResume() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -1330,8 +1403,7 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception null, null, false - ), - null + ) ); final ListenableFuture future = runTask(task); @@ -1418,8 +1490,7 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception null, null, false - ), - null + ) ); runTask(task); @@ -1438,6 +1509,7 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception @Test(timeout = 30_000L) public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAvailable() throws Exception { + resetOffsetAutomatically = true; // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { for (ProducerRecord record : records) { @@ -1457,8 +1529,7 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva null, null, false - ), - true + ) ); runTask(task); @@ -1526,23 +1597,21 @@ public boolean apply(TaskLock lock) private KafkaIndexTask createTask( final String taskId, - final KafkaIOConfig ioConfig, - final Boolean resetOffsetAutomatically + final KafkaIOConfig ioConfig ) { - return createTask(taskId, DATA_SCHEMA, ioConfig, resetOffsetAutomatically); + return createTask(taskId, DATA_SCHEMA, ioConfig); } private KafkaIndexTask createTask( final String taskId, final DataSchema dataSchema, - final KafkaIOConfig ioConfig, - final Boolean resetOffsetAutomatically + final KafkaIOConfig ioConfig ) { final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( 1000, - null, + maxRowsPerSegment, new Period("P1Y"), null, null, @@ -1662,6 +1731,24 @@ private void makeToolboxFactory() throws IOException metadataStorageCoordinator, emitter, new SupervisorManager(null) + { + @Override + public boolean checkPointDataSourceMetadata( + String supervisorId, + @Nullable String sequenceName, + @Nullable DataSourceMetadata previousDataSourceMetadata, + @Nullable DataSourceMetadata currentDataSourceMetadata + ) + { + checkpointRequestsHash.add(Objects.hash( + supervisorId, + sequenceName, + previousDataSourceMetadata, + currentDataSourceMetadata + )); + return true; + } + } ); final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( taskStorage, diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 24e9c459845d..bba149668655 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -42,7 +42,6 @@ import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; import io.druid.common.guava.ThreadRenamingCallable; -import io.druid.java.util.common.concurrent.Execs; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; import io.druid.java.util.common.DateTimes; @@ -51,6 +50,7 @@ import io.druid.java.util.common.Pair; import io.druid.java.util.common.RetryUtils; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.io.Closer; import io.druid.query.Query; import io.druid.query.QueryRunner; From c3b68cfd72913c40e0214301f60945b10b0888d7 Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Tue, 7 Nov 2017 20:06:19 -0600 Subject: [PATCH 09/14] fix deadlock between persist and push executors in AppenderatorImpl --- .../druid/indexing/kafka/KafkaIndexTask.java | 2 +- .../indexing/kafka/KafkaIndexTaskTest.java | 27 +++++--- .../IndexerSQLMetadataStorageCoordinator.java | 2 +- .../appenderator/AppenderatorImpl.java | 65 +++++++++++++++---- 4 files changed, 75 insertions(+), 21 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index c83add3720f5..b7fe37c2a680 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -2004,7 +2004,7 @@ public SequenceMetadata( Preconditions.checkNotNull(endOffsets); this.sequenceId = sequenceId; this.sequenceName = sequenceName; - this.startOffsets = startOffsets; + this.startOffsets = ImmutableMap.copyOf(startOffsets); this.endOffsets = Maps.newHashMap(endOffsets); this.assignments = Sets.newHashSet(startOffsets.keySet()); this.checkpointed = checkpointed; diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 7e1acce1a990..e33405bc652f 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -438,6 +438,7 @@ public void testIncrementalHandOff() throws Exception return; } final String baseSequenceName = "sequence0"; + // as soon as any segment has more than one record, incremental publishing should happen maxRowsPerSegment = 1; // Insert data @@ -450,7 +451,10 @@ public void testIncrementalHandOff() throws Exception consumerProps.put("max.poll.records", "1"); final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L)); - final KafkaPartitions checkpointPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 0L)); + // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering + // of events fetched across two partitions from Kafka + final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 0L)); + final KafkaPartitions checkpoint2 = new KafkaPartitions(topic, ImmutableMap.of(0, 4L, 1, 2L)); final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 8L, 1, 2L)); final KafkaIndexTask task = createTask( null, @@ -470,15 +474,19 @@ public void testIncrementalHandOff() throws Exception while (task.getStatus() != KafkaIndexTask.Status.PAUSED) { Thread.sleep(10); } - task.setEndOffsets(checkpointPartitions.getPartitionOffsetMap(), true, false); + final Map currentOffsets = ImmutableMap.copyOf(task.getCurrentOffsets()); + Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets) || checkpoint2.getPartitionOffsetMap() + .equals(currentOffsets)); + task.setEndOffsets(currentOffsets, true, false); Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); Assert.assertEquals(1, checkpointRequestsHash.size()); Assert.assertTrue(checkpointRequestsHash.contains( - Objects.hash(DATA_SCHEMA.getDataSource(), - baseSequenceName, - new KafkaDataSourceMetadata(startPartitions), - new KafkaDataSourceMetadata(checkpointPartitions) + Objects.hash( + DATA_SCHEMA.getDataSource(), + baseSequenceName, + new KafkaDataSourceMetadata(startPartitions), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets)) ) )); @@ -505,8 +513,10 @@ public void testIncrementalHandOff() throws Exception Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc4)); - Assert.assertEquals(ImmutableList.of("h"), readSegmentColumn("dim1", desc5)); + Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4)) + && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) || + (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4)) + && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5)))); Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6)); Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); } @@ -1740,6 +1750,7 @@ public boolean checkPointDataSourceMetadata( @Nullable DataSourceMetadata currentDataSourceMetadata ) { + log.info("Adding checkpoint hash to the set"); checkpointRequestsHash.add(Objects.hash( supervisorId, sequenceName, diff --git a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index b8bb7e7ea8e7..ca6a4e8a1a8a 100644 --- a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -765,7 +765,7 @@ protected DataSourceMetadataUpdateResult updateDataSourceMetadataWithHandle( if (!startMetadataMatchesExisting) { // Not in the desired start state. log.info("Not updating metadata, existing state is not the expected start state."); - log.debug("Existing database state [%s], request's start metadata", oldCommitMetadataFromDb, startMetadata); + log.debug("Existing database state [%s], request's start metadata [%s]", oldCommitMetadataFromDb, startMetadata); return DataSourceMetadataUpdateResult.FAILURE; } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index bba149668655..ef40c3c3e4f9 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -96,6 +96,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; /** @@ -125,9 +127,13 @@ public class AppenderatorImpl implements Appenderator // This variable updated in add(), persist(), and drop() private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger(); private final AtomicInteger totalRows = new AtomicInteger(); + // Synchronize persisting commitMetadata so that multiple persist threads (if present) + // and abandon threads do not step over each other + private final Lock commitLock = new ReentrantLock(); private volatile ListeningExecutorService persistExecutor = null; private volatile ListeningExecutorService pushExecutor = null; + private volatile ListeningExecutorService abandonExecutor = null; private volatile long nextFlush; private volatile FileLock basePersistDirLock = null; private volatile FileChannel basePersistDirLockChannel = null; @@ -331,7 +337,12 @@ public void clear() throws InterruptedException @Override public Object call() throws Exception { - objectMapper.writeValue(computeCommitFile(), Committed.nil()); + try { + commitLock.lock(); + objectMapper.writeValue(computeCommitFile(), Committed.nil()); + } finally { + commitLock.unlock(); + } return null; } } @@ -426,15 +437,20 @@ public Object doCall() committer.run(); - final File commitFile = computeCommitFile(); - final Map commitHydrants = Maps.newHashMap(); - if (commitFile.exists()) { - // merge current hydrants with existing hydrants - final Committed oldCommitted = objectMapper.readValue(commitFile, Committed.class); - commitHydrants.putAll(oldCommitted.getHydrants()); + try { + commitLock.lock(); + final File commitFile = computeCommitFile(); + final Map commitHydrants = Maps.newHashMap(); + if (commitFile.exists()) { + // merge current hydrants with existing hydrants + final Committed oldCommitted = objectMapper.readValue(commitFile, Committed.class); + commitHydrants.putAll(oldCommitted.getHydrants()); + } + commitHydrants.putAll(currentHydrants); + objectMapper.writeValue(commitFile, new Committed(commitHydrants, commitMetadata)); + } finally { + commitLock.unlock(); } - commitHydrants.putAll(currentHydrants); - objectMapper.writeValue(commitFile, new Committed(commitHydrants, commitMetadata)); return commitMetadata; } @@ -669,6 +685,10 @@ public void close() pushExecutor == null || pushExecutor.awaitTermination(365, TimeUnit.DAYS), "pushExecutor not terminated" ); + Preconditions.checkState( + abandonExecutor == null || abandonExecutor.awaitTermination(365, TimeUnit.DAYS), + "abandonExecutor not terminated" + ); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -712,6 +732,10 @@ public void closeNow() persistExecutor == null || persistExecutor.awaitTermination(365, TimeUnit.DAYS), "persistExecutor not terminated" ); + Preconditions.checkState( + abandonExecutor == null || abandonExecutor.awaitTermination(365, TimeUnit.DAYS), + "abandonExecutor not terminated" + ); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -774,6 +798,14 @@ private void initializeExecutors() ) ); } + if (abandonExecutor== null) { + // use single threaded executor with SynchronousQueue so that all abandon operations occur sequentially + abandonExecutor = MoreExecutors.listeningDecorator( + Execs.newBlockingSingleThreaded( + "appenderator_abandon_%d", 0 + ) + ); + } } private void shutdownExecutors() @@ -784,6 +816,9 @@ private void shutdownExecutors() if (pushExecutor != null) { pushExecutor.shutdownNow(); } + if (abandonExecutor != null) { + abandonExecutor.shutdownNow(); + } } private void resetNextFlush() @@ -810,9 +845,12 @@ private Object bootstrapSinksFromDisk() return null; } - final File commitFile = computeCommitFile(); + final Committed committed; + File commitFile = null; try { + commitLock.lock(); + commitFile = computeCommitFile(); if (commitFile.exists()) { committed = objectMapper.readValue(commitFile, Committed.class); } else { @@ -821,6 +859,8 @@ private Object bootstrapSinksFromDisk() } catch (Exception e) { throw new ISE(e, "Failed to read commitFile: %s", commitFile); + } finally { + commitLock.unlock(); } log.info("Loading sinks from[%s]: %s", baseDir, committed.getHydrants().keySet()); @@ -982,6 +1022,7 @@ public Object apply(@Nullable Object input) // Remove this segment from the committed list. This must be done from the persist thread. log.info("Removing commit metadata for segment[%s].", identifier); try { + commitLock.lock(); final File commitFile = computeCommitFile(); if (commitFile.exists()) { final Committed oldCommitted = objectMapper.readValue(commitFile, Committed.class); @@ -993,6 +1034,8 @@ public Object apply(@Nullable Object input) .addData("identifier", identifier.getIdentifierAsString()) .emit(); throw Throwables.propagate(e); + } finally { + commitLock.unlock(); } } @@ -1029,7 +1072,7 @@ public Object apply(@Nullable Object input) return null; } }, - persistExecutor + abandonExecutor ); } From 8295673fd6c3c9e14e95e0738ca692fa9de03a8d Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Tue, 7 Nov 2017 22:09:42 -0600 Subject: [PATCH 10/14] fix formatting --- .../realtime/appenderator/AppenderatorImpl.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index ef40c3c3e4f9..1d126d216830 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -340,7 +340,8 @@ public Object call() throws Exception try { commitLock.lock(); objectMapper.writeValue(computeCommitFile(), Committed.nil()); - } finally { + } + finally { commitLock.unlock(); } return null; @@ -448,7 +449,8 @@ public Object doCall() } commitHydrants.putAll(currentHydrants); objectMapper.writeValue(commitFile, new Committed(commitHydrants, commitMetadata)); - } finally { + } + finally { commitLock.unlock(); } @@ -798,7 +800,7 @@ private void initializeExecutors() ) ); } - if (abandonExecutor== null) { + if (abandonExecutor == null) { // use single threaded executor with SynchronousQueue so that all abandon operations occur sequentially abandonExecutor = MoreExecutors.listeningDecorator( Execs.newBlockingSingleThreaded( @@ -859,7 +861,8 @@ private Object bootstrapSinksFromDisk() } catch (Exception e) { throw new ISE(e, "Failed to read commitFile: %s", commitFile); - } finally { + } + finally { commitLock.unlock(); } @@ -1034,7 +1037,8 @@ public Object apply(@Nullable Object input) .addData("identifier", identifier.getIdentifierAsString()) .emit(); throw Throwables.propagate(e); - } finally { + } + finally { commitLock.unlock(); } } From 80cc3bb3f6b7f33862e6f04ecb2b57ed7b919759 Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Mon, 13 Nov 2017 15:42:23 -0600 Subject: [PATCH 11/14] use persist dir instead of work dir --- .../src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index b7fe37c2a680..4a519df16283 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -451,7 +451,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception )); } - sequencesPersistFile = new File(toolbox.getTaskWorkDir(), "sequences.json"); + sequencesPersistFile = new File(toolbox.getPersistDir(), "sequences.json"); restoreSequences(); log.info("Starting with sequences: %s", sequences); From 32da00b1930d79da00fb798ecfee933e677c00b5 Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Thu, 16 Nov 2017 14:34:31 -0600 Subject: [PATCH 12/14] review comments --- .../main/java/io/druid/indexing/kafka/KafkaIndexTask.java | 6 +++--- .../io/druid/indexing/kafka/supervisor/KafkaSupervisor.java | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 4a519df16283..0b24cbe126b7 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -577,7 +577,7 @@ public void run() }; // restart publishing of sequences (if any) - mayBePersistAndPublishSequences(committerSupplier); + maybePersistAndPublishSequences(committerSupplier); Set assignment = assignPartitionsAndSeekToNext(consumer, topic); @@ -612,7 +612,7 @@ public void run() checkAndMaybeThrowException(); if (!ioConfig.isPauseAfterRead()) { - mayBePersistAndPublishSequences(committerSupplier); + maybePersistAndPublishSequences(committerSupplier); } // The retrying business is because the KafkaConsumer throws OffsetOutOfRangeException if the seeked-to @@ -1212,7 +1212,7 @@ private void checkAndMaybeThrowException() } } - private void mayBePersistAndPublishSequences(Supplier committerSupplier) + private void maybePersistAndPublishSequences(Supplier committerSupplier) throws ExecutionException, InterruptedException { for (SequenceMetadata sequenceMetadata : sequences) { diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 1d6f7a90d58a..76468e4f50a3 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -1124,7 +1124,7 @@ private void verifyAndMergeCheckpoints(final Integer groupId) final TaskGroup taskGroup = taskGroups.get(groupId); // List {SequenceId, Checkpoints}> - final List>>> taskSequences = new ArrayList<>(); + final List>>> taskSequences = new CopyOnWriteArrayList<>(); final List>>> futures = new ArrayList<>(); for (String taskId : taskGroup.taskIds()) { From 6a161efc988aea34137bfaa4a03b884c6dde5559 Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Fri, 17 Nov 2017 11:57:39 -0600 Subject: [PATCH 13/14] fix deadlock --- .../appenderator/AppenderatorImpl.java | 44 ++++++++++++++----- 1 file changed, 33 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 1d126d216830..5f21f9c4f9f8 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -133,7 +133,10 @@ public class AppenderatorImpl implements Appenderator private volatile ListeningExecutorService persistExecutor = null; private volatile ListeningExecutorService pushExecutor = null; - private volatile ListeningExecutorService abandonExecutor = null; + // use intermediate executor so that deadlock conditions can be prevented + // where persist and push Executor try to put tasks in each other queues + // thus creating circular dependency + private volatile ListeningExecutorService intermediateTempExecutor = null; private volatile long nextFlush; private volatile FileLock basePersistDirLock = null; private volatile FileChannel basePersistDirLockChannel = null; @@ -688,8 +691,8 @@ public void close() "pushExecutor not terminated" ); Preconditions.checkState( - abandonExecutor == null || abandonExecutor.awaitTermination(365, TimeUnit.DAYS), - "abandonExecutor not terminated" + intermediateTempExecutor == null || intermediateTempExecutor.awaitTermination(365, TimeUnit.DAYS), + "intermediateTempExecutor not terminated" ); } catch (InterruptedException e) { @@ -735,8 +738,8 @@ public void closeNow() "persistExecutor not terminated" ); Preconditions.checkState( - abandonExecutor == null || abandonExecutor.awaitTermination(365, TimeUnit.DAYS), - "abandonExecutor not terminated" + intermediateTempExecutor == null || intermediateTempExecutor.awaitTermination(365, TimeUnit.DAYS), + "intermediateTempExecutor not terminated" ); } catch (InterruptedException e) { @@ -800,9 +803,9 @@ private void initializeExecutors() ) ); } - if (abandonExecutor == null) { + if (intermediateTempExecutor == null) { // use single threaded executor with SynchronousQueue so that all abandon operations occur sequentially - abandonExecutor = MoreExecutors.listeningDecorator( + intermediateTempExecutor = MoreExecutors.listeningDecorator( Execs.newBlockingSingleThreaded( "appenderator_abandon_%d", 0 ) @@ -818,8 +821,8 @@ private void shutdownExecutors() if (pushExecutor != null) { pushExecutor.shutdownNow(); } - if (abandonExecutor != null) { - abandonExecutor.shutdownNow(); + if (intermediateTempExecutor != null) { + intermediateTempExecutor.shutdownNow(); } } @@ -1008,7 +1011,24 @@ private ListenableFuture abandonSegment( // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. return Futures.transform( - pushBarrier(), + Futures.transform( + pushBarrier(), + new Function() + { + @Nullable + @Override + public Object apply(@Nullable Object input) + { + // do nothing + return null; + } + }, + // use temp executor so that there persistExecutor and pushExecutor do + // not try to put tasks in each other queues which can lead to deadlocks + // after the above function is done, intermediateTempExecutor will push the + // below function to persistExecutor's queue + intermediateTempExecutor + ), new Function() { @Nullable @@ -1076,7 +1096,9 @@ public Object apply(@Nullable Object input) return null; } }, - abandonExecutor + // use persistExecutor to make sure that all the pending persists completes before + // starting to abandon segments + persistExecutor ); } From 4335dc622edb15df85bd7f98d41c085936eafafa Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Fri, 17 Nov 2017 13:08:01 -0600 Subject: [PATCH 14/14] actually fix deadlock --- .../appenderator/AppenderatorImpl.java | 30 ++----------------- 1 file changed, 3 insertions(+), 27 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 5f21f9c4f9f8..83dddae834e4 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -541,15 +541,8 @@ public ListenableFuture push( */ private ListenableFuture pushBarrier() { - return pushExecutor.submit( - new Runnable() - { - @Override - public void run() - { - // Do nothing - } - } + return intermediateTempExecutor.submit( + (Runnable) () -> pushExecutor.submit(() -> {}) ); } @@ -1011,24 +1004,7 @@ private ListenableFuture abandonSegment( // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. return Futures.transform( - Futures.transform( - pushBarrier(), - new Function() - { - @Nullable - @Override - public Object apply(@Nullable Object input) - { - // do nothing - return null; - } - }, - // use temp executor so that there persistExecutor and pushExecutor do - // not try to put tasks in each other queues which can lead to deadlocks - // after the above function is done, intermediateTempExecutor will push the - // below function to persistExecutor's queue - intermediateTempExecutor - ), + pushBarrier(), new Function() { @Nullable