From b35c2b17a9070e64d32c310a519f9815865279ee Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 22 Mar 2019 11:26:31 -0700 Subject: [PATCH 01/22] Add state and error tracking for seekable stream supervisors --- .../druid/guice/DruidSecondaryModule.java | 2 - .../druid/utils/CircularBufferTest.java | 53 +++ .../extensions-core/kafka-ingestion.md | 35 ++ .../extensions-core/kinesis-ingestion.md | 48 ++- .../indexing/kafka/KafkaRecordSupplier.java | 47 ++- .../kafka/supervisor/KafkaSupervisor.java | 11 +- .../KafkaSupervisorReportPayload.java | 13 +- .../kafka/supervisor/KafkaSupervisorSpec.java | 14 +- .../kafka/supervisor/KafkaSupervisorTest.java | 32 +- .../kinesis/KinesisRecordSupplier.java | 39 +- .../kinesis/supervisor/KinesisSupervisor.java | 11 +- .../KinesisSupervisorReportPayload.java | 10 +- .../supervisor/KinesisSupervisorSpec.java | 13 +- .../supervisor/KinesisSupervisorTest.java | 45 ++- .../guice/SeekableStreamSupervisorModule.java | 34 ++ .../druid/indexing/common/task/IndexTask.java | 4 +- .../SeekableStreamSupervisorConfig.java | 91 +++++ .../NonTransientStreamException.java | 33 ++ .../PossiblyTransientStreamException.java | 34 ++ .../exceptions/TransientStreamException.java | 33 ++ .../supervisor/SeekableStreamSupervisor.java | 70 +++- ...SeekableStreamSupervisorReportPayload.java | 21 +- .../SeekableStreamSupervisorSpec.java | 6 +- .../SeekableStreamSupervisorStateManager.java | 335 ++++++++++++++++++ ...kableStreamSupervisorStateManagerTest.java | 307 ++++++++++++++++ .../org/apache/druid/cli/CliOverlord.java | 4 +- 26 files changed, 1248 insertions(+), 97 deletions(-) create mode 100644 core/src/test/java/org/apache/druid/utils/CircularBufferTest.java create mode 100644 indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorConfig.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/NonTransientStreamException.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/PossiblyTransientStreamException.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/TransientStreamException.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java diff --git a/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java b/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java index cded7dda87c1..319c0d1f1dc8 100644 --- a/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java +++ b/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java @@ -34,8 +34,6 @@ import javax.validation.Validator; import java.util.Properties; -/** - */ public class DruidSecondaryModule implements Module { private final Properties properties; diff --git a/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java b/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java new file mode 100644 index 000000000000..5d9d77534e07 --- /dev/null +++ b/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.utils; + +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +public class CircularBufferTest +{ + private static CircularBuffer buff = new CircularBuffer(4); + + @BeforeClass + public static void setup() + { + for (int i = 1; i <= 9; i++) { + buff.add(i); // buffer should contain [9, 6, 7, 8] + } + } + + @Test + public void testCircularBufferGetLatest() + { + for (int i = 0; i < 4; i++) { + Assert.assertEquals((Integer) (9 - i), buff.getLatest(i)); + } + } + + @Test + public void testCircularBufferGet() + { + for (int i = 0; i < 4; i++) { + Assert.assertEquals((Integer) (i + 6), buff.get(i)); + } + } +} diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 47e40d5ec101..d49b2a28397b 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -214,6 +214,30 @@ offsets as reported by Kafka, the consumer lag per partition, as well as the agg consumer lag per partition may be reported as negative values if the supervisor has not received a recent latest offset response from Kafka. The aggregate lag value will always be >= 0. +The status report also contains the supervisor's state and a list of recently thrown exceptions (whose max size can be +controlled using the `druid.supervisor.stream.maxStoredExceptionEvents` config parameter). The list of states is as +follows: + +|State|Description|Priority| +|-----|-----------|--------| +|UNHEALTHY_SUPERVISOR|The supervisor has encountered non-transient errors on the past `druid.supervisor.stream.unhealthinessThreshold` iterations|1| +|UNHEALTHY_TASKS|The last `druid.supervisor.stream.taskUnhealthinessThreshold` tasks have all failed|2| +|UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kinesis and has not successfully connected in the past|3| +|LOST_CONTACT_WITH_STREAM|The supervisor is encountering transient connectivity issues with Kinesis but has successfully connected in the past|4| +|WAITING_TO_RUN (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream.|5| +|CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data|5| +|DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks|5| +|CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state|5| +|RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse|5| +|SUSPENDED|The supervisor has been suspended|5| +|SHUTTING_DOWN|Shutdown has been called but the supervisor hasn’t fully shutdown yet|5| + +Notes about states: + +- Since it's possible that 2+ states can apply to a supervisor at the same time, each state is given a priority. The +active state with the highest priority will be returned in the status report. +- States marked with "first iteration only" only occur on the supervisor's first iteration at startup or after suspension. + ### Getting Supervisor Ingestion Stats Report `GET /druid/indexer/v1/supervisor//stats` returns a snapshot of the current ingestion row counters for each task being managed by the supervisor, along with moving averages for the row counters. @@ -344,3 +368,14 @@ one can schedule re-indexing tasks be run to merge segments together into new se Details on how to optimize the segment size can be found on [Segment size optimization](../../operations/segment-optimization.html). There is also ongoing work to support automatic segment compaction of sharded segments as well as compaction not requiring Hadoop (see [here](https://github.com/apache/incubator-druid/pull/5102)). + +## Configuration Properties + +|property|description|values|default| +|--------|-----------|------|-------| +|druid.supervisor.stream.healthinessThreshold|The number of successful iterations before the supervisor flips from an UNHEALTHY to a RUNNING state|An integer in [3,2147483647]|3| +|druid.supervisor.stream.unhealthinessThreshold|The number of iterations failed before the supervisor flips from a RUNNING to an UNHEALTHY state|An integer in [3,2147483647]|3| +|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state|An integer in [3,2147483647]|3| +|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state|An integer in [3,2147483647]|3| +|druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|true/false|false| +|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|An integer in [`max(healthinessThreshold, unhealthinessThreshold)`, 2147483647]|`max(healthinessThreshold, unhealthinessThreshold)`| \ No newline at end of file diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md index 38e383020acc..a3579ae3d929 100644 --- a/docs/content/development/extensions-core/kinesis-ingestion.md +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -113,7 +113,7 @@ A sample supervisor spec is shown below: } ``` -## Supervisor Configuration +## Supervisor Spec |Field|Description|Required| |--------|-----------|---------| @@ -218,12 +218,39 @@ To authenticate with AWS, you must provide your AWS access key and AWS secret ke ``` -Ddruid.kinesis.accessKey=123 -Ddruid.kinesis.secretKey=456 ``` -The AWS access key ID and secret access key are used for Kinesis API requests. If this is not provided, the service will look for credentials set in environment variables, in the default profile configuration file, and from the EC2 instance profile provider (in this order). +The AWS access key ID and secret access key are used for Kinesis API requests. If this is not provided, the service will +look for credentials set in environment variables, in the default profile configuration file, and from the EC2 instance +profile provider (in this order). ### Getting Supervisor Status Report -`GET /druid/indexer/v1/supervisor//status` returns a snapshot report of the current state of the tasks managed by the given supervisor. This includes the latest -sequence numbers as reported by Kinesis. Unlike the Kafka Indexing Service, stats about lag is not yet supported. +`GET /druid/indexer/v1/supervisor//status` returns a snapshot report of the current state of the tasks +managed by the given supervisor. This includes the latest sequence numbers as reported by Kinesis. Unlike the Kafka +Indexing Service, stats about lag are not yet supported. + +The status report also contains the supervisor's state and a list of recently thrown exceptions (whose max size can be +controlled using the `druid.supervisor.stream.maxStoredExceptionEvents` config parameter). The list of states is as +follows: + +|State|Description|Priority| +|-----|-----------|--------| +|UNHEALTHY_SUPERVISOR|The supervisor has encountered non-transient errors on the past `druid.supervisor.stream.unhealthinessThreshold` iterations|1| +|UNHEALTHY_TASKS|The last `druid.supervisor.stream.taskUnhealthinessThreshold` tasks have all failed|2| +|UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kinesis and has not successfully connected in the past|3| +|LOST_CONTACT_WITH_STREAM|The supervisor is encountering transient connectivity issues with Kinesis but has successfully connected in the past|4| +|WAITING_TO_RUN (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream.|5| +|CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data|5| +|DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks|5| +|CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state|5| +|RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse|5| +|SUSPENDED|The supervisor has been suspended|5| +|SHUTTING_DOWN|Shutdown has been called but the supervisor hasn’t fully shutdown yet|5| + +Notes about states: + +- Since it's possible that 2+ states can apply to a supervisor at the same time, each state is given a priority. The +active state with the highest priority will be returned in the status report. +- States marked with "first iteration only" only occur on the supervisor's first iteration at startup or after suspension. ### Updating Existing Supervisors @@ -390,4 +417,15 @@ requires the user to manually provide the Kinesis Client Library on the classpat compatible with Apache projects. To enable this feature, add the `amazon-kinesis-client` (tested on version `1.9.2`) jar file ([link](https://mvnrepository.com/artifact/com.amazonaws/amazon-kinesis-client/1.9.2)) under `dist/druid/extensions/druid-kinesis-indexing-service/`. -Then when submitting a supervisor-spec, set `deaggregate` to true. \ No newline at end of file +Then when submitting a supervisor-spec, set `deaggregate` to true. + +## Configuration Properties + +|property|description|values|default| +|--------|-----------|------|-------| +|druid.supervisor.stream.healthinessThreshold|The number of successful iterations before the supervisor flips from an UNHEALTHY to a RUNNING state|An integer in [3,2147483647]|3| +|druid.supervisor.stream.unhealthinessThreshold|The number of iterations failed before the supervisor flips from a RUNNING to an UNHEALTHY state|An integer in [3,2147483647]|3| +|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state|An integer in [3,2147483647]|3| +|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state|An integer in [3,2147483647]|3| +|druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|true/false|false| +|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|An integer in [`max(healthinessThreshold, unhealthinessThreshold)`, 2147483647]|`max(healthinessThreshold, unhealthinessThreshold)`| \ No newline at end of file diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index fcd167374011..b7522f6061a3 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -25,12 +25,19 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.exceptions.NonTransientStreamException; +import org.apache.druid.indexing.seekablestream.exceptions.PossiblyTransientStreamException; +import org.apache.druid.indexing.seekablestream.exceptions.TransientStreamException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.metadata.PasswordProvider; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import javax.annotation.Nonnull; @@ -122,9 +129,9 @@ public List> poll(long timeout) @Override public Long getLatestSequenceNumber(StreamPartition partition) { - Long currPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); + Long currPos = getPosition(partition); seekToLatest(Collections.singleton(partition)); - Long nextPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); + Long nextPos = getPosition(partition); seek(partition, currPos); return nextPos; } @@ -132,9 +139,9 @@ public Long getLatestSequenceNumber(StreamPartition partition) @Override public Long getEarliestSequenceNumber(StreamPartition partition) { - Long currPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); + Long currPos = getPosition(partition); seekToEarliest(Collections.singleton(partition)); - Long nextPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); + Long nextPos = getPosition(partition); seek(partition, currPos); return nextPos; } @@ -142,17 +149,39 @@ public Long getEarliestSequenceNumber(StreamPartition partition) @Override public Long getPosition(StreamPartition partition) { - return consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); + try { + return consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); + } + catch (TimeoutException e) { + throw new PossiblyTransientStreamException(e); + } + catch (WakeupException | InterruptException e) { + throw new TransientStreamException(e); + } + catch (KafkaException e) { + throw new NonTransientStreamException(e); + } } @Override public Set getPartitionIds(String stream) { - List partitions = consumer.partitionsFor(stream); - if (partitions == null) { - throw new ISE("Topic [%s] is not found in KafkaConsumer's list of topics", stream); + try { + List partitions = consumer.partitionsFor(stream); + if (partitions == null) { + throw new ISE("Topic [%s] is not found in KafkaConsumer's list of topics", stream); + } + return partitions.stream().map(PartitionInfo::partition).collect(Collectors.toSet()); + } + catch (TimeoutException e) { + throw new PossiblyTransientStreamException(e); + } + catch (WakeupException | InterruptException e) { + throw new TransientStreamException(e); + } + catch (KafkaException e) { + throw new NonTransientStreamException(e); } - return partitions.stream().map(PartitionInfo::partition).collect(Collectors.toSet()); } @Override diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index b2924ea2fdff..56db89db6921 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -43,6 +43,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -105,7 +106,8 @@ public KafkaSupervisor( final KafkaIndexTaskClientFactory taskClientFactory, final ObjectMapper mapper, final KafkaSupervisorSpec spec, - final RowIngestionMetersFactory rowIngestionMetersFactory + final RowIngestionMetersFactory rowIngestionMetersFactory, + final SeekableStreamSupervisorConfig supervisorConfig ) { super( @@ -117,7 +119,8 @@ public KafkaSupervisor( mapper, spec, rowIngestionMetersFactory, - false + false, + supervisorConfig ); this.spec = spec; @@ -191,7 +194,9 @@ protected SeekableStreamSupervisorReportPayload createReportPaylo includeOffsets ? partitionLag : null, includeOffsets ? partitionLag.values().stream().mapToLong(x -> Math.max(x, 0)).sum() : null, includeOffsets ? sequenceLastUpdated : null, - spec.isSuspended() + spec.isSuspended(), + stateManager.getSupervisorState(), + stateManager.getExceptionEvents() ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index d5f4efa02a20..c3d6e4807e8e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -20,14 +20,15 @@ package org.apache.druid.indexing.kafka.supervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Map; +import java.util.Queue; public class KafkaSupervisorReportPayload extends SeekableStreamSupervisorReportPayload { - public KafkaSupervisorReportPayload( String dataSource, String topic, @@ -38,7 +39,9 @@ public KafkaSupervisorReportPayload( @Nullable Map minimumLag, @Nullable Long aggregateLag, @Nullable DateTime offsetsLastUpdated, - boolean suspended + boolean suspended, + SeekableStreamSupervisorStateManager.State state, + Queue recentErrors ) { super( @@ -51,11 +54,12 @@ public KafkaSupervisorReportPayload( minimumLag, aggregateLag, offsetsLastUpdated, - suspended + suspended, + state, + recentErrors ); } - @Override public String toString() { @@ -74,5 +78,4 @@ public String toString() ", suspended=" + getSuspended() + '}'; } - } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index 33ad3e11bee7..d7cb41326525 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -30,6 +30,7 @@ import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; @@ -39,7 +40,6 @@ public class KafkaSupervisorSpec extends SeekableStreamSupervisorSpec { - @JsonCreator public KafkaSupervisorSpec( @JsonProperty("dataSchema") DataSchema dataSchema, @@ -54,7 +54,8 @@ public KafkaSupervisorSpec( @JacksonInject @Json ObjectMapper mapper, @JacksonInject ServiceEmitter emitter, @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, - @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, + @JacksonInject SeekableStreamSupervisorConfig supervisorConfig ) { super( @@ -96,7 +97,8 @@ public KafkaSupervisorSpec( mapper, emitter, monitorSchedulerConfig, - rowIngestionMetersFactory + rowIngestionMetersFactory, + supervisorConfig ); } @@ -110,7 +112,8 @@ public Supervisor createSupervisor() (KafkaIndexTaskClientFactory) indexTaskClientFactory, mapper, this, - rowIngestionMetersFactory + rowIngestionMetersFactory, + supervisorConfig ); } @@ -144,7 +147,8 @@ protected KafkaSupervisorSpec toggleSuspend(boolean suspend) mapper, emitter, monitorSchedulerConfig, - rowIngestionMetersFactory + rowIngestionMetersFactory, + supervisorConfig ); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index b30b75ee7886..4ab94dc03a76 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -63,6 +63,8 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -1288,12 +1290,14 @@ public void testDiscoverExistingPublishingTask() throws Exception KafkaSupervisorReportPayload payload = report.getPayload(); Assert.assertEquals(DATASOURCE, payload.getDataSource()); - Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); - Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); - Assert.assertEquals(1, (int) payload.getReplicas()); + Assert.assertEquals(3600L, payload.getDurationSeconds()); + Assert.assertEquals(NUM_PARTITIONS, payload.getPartitions()); + Assert.assertEquals(1, payload.getReplicas()); Assert.assertEquals(topic, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData publishingReport = payload.getPublishingTasks().get(0); @@ -1394,12 +1398,14 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() KafkaSupervisorReportPayload payload = report.getPayload(); Assert.assertEquals(DATASOURCE, payload.getDataSource()); - Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); - Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); - Assert.assertEquals(1, (int) payload.getReplicas()); + Assert.assertEquals(3600L, payload.getDurationSeconds()); + Assert.assertEquals(NUM_PARTITIONS, payload.getPartitions()); + Assert.assertEquals(1, payload.getReplicas()); Assert.assertEquals(topic, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData publishingReport = payload.getPublishingTasks().get(0); @@ -1531,12 +1537,14 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception KafkaSupervisorReportPayload payload = report.getPayload(); Assert.assertEquals(DATASOURCE, payload.getDataSource()); - Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); - Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); - Assert.assertEquals(1, (int) payload.getReplicas()); + Assert.assertEquals(3600L, payload.getDurationSeconds()); + Assert.assertEquals(NUM_PARTITIONS, payload.getPartitions()); + Assert.assertEquals(1, payload.getReplicas()); Assert.assertEquals(topic, payload.getStream()); Assert.assertEquals(1, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData activeReport = payload.getActiveTasks().get(0); TaskReportData publishingReport = payload.getPublishingTasks().get(0); @@ -2948,7 +2956,8 @@ public KafkaIndexTaskClient build( objectMapper, new NoopServiceEmitter(), new DruidMonitorSchedulerConfig(), - rowIngestionMetersFactory + rowIngestionMetersFactory, + new SeekableStreamSupervisorConfig() ), rowIngestionMetersFactory ); @@ -3076,7 +3085,8 @@ public TestableKafkaSupervisor( taskClientFactory, mapper, spec, - rowIngestionMetersFactory + rowIngestionMetersFactory, + new SeekableStreamSupervisorConfig() ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index a05eade5f4cd..f6d85f54bb99 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -26,12 +26,15 @@ import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; +import com.amazonaws.services.kinesis.model.AmazonKinesisException; import com.amazonaws.services.kinesis.model.ExpiredIteratorException; import com.amazonaws.services.kinesis.model.GetRecordsRequest; import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.amazonaws.services.kinesis.model.InvalidArgumentException; +import com.amazonaws.services.kinesis.model.LimitExceededException; import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.ResourceInUseException; import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.model.ShardIteratorType; @@ -46,6 +49,9 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.exceptions.NonTransientStreamException; +import org.apache.druid.indexing.seekablestream.exceptions.PossiblyTransientStreamException; +import org.apache.druid.indexing.seekablestream.exceptions.TransientStreamException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; @@ -579,12 +585,23 @@ public String getPosition(StreamPartition partition) @Override public Set getPartitionIds(String stream) { - checkIfClosed(); - return kinesis.describeStream(stream) - .getStreamDescription() - .getShards() - .stream() - .map(Shard::getShardId).collect(Collectors.toSet()); + try { + checkIfClosed(); + return kinesis.describeStream(stream) + .getStreamDescription() + .getShards() + .stream() + .map(Shard::getShardId).collect(Collectors.toSet()); + } + catch (LimitExceededException | ProvisionedThroughputExceededException | ResourceInUseException e) { + throw new TransientStreamException(e); + } + catch (ResourceNotFoundException e) { + throw new PossiblyTransientStreamException(e); + } + catch (AmazonKinesisException e) { + throw new NonTransientStreamException(e); + } } @Override @@ -671,7 +688,7 @@ private void filterBufferAndResetFetchRunnable(Set> part private String getSequenceNumberInternal(StreamPartition partition, ShardIteratorType iteratorEnum) { - String shardIterator = null; + String shardIterator; try { shardIterator = kinesis.getShardIterator( partition.getStream(), @@ -679,8 +696,14 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar iteratorEnum.toString() ).getShardIterator(); } + catch (LimitExceededException | ProvisionedThroughputExceededException | ResourceInUseException e) { + throw new TransientStreamException(e); + } catch (ResourceNotFoundException e) { - log.warn(e, "Caught ResourceNotFoundException while getting shardIterator"); + throw new PossiblyTransientStreamException(e); + } + catch (AmazonKinesisException e) { + throw new NonTransientStreamException(e); } return getSequenceNumberInternal(partition, shardIterator); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 870d7ecb7d38..af232a71e237 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -45,6 +45,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -92,7 +93,8 @@ public KinesisSupervisor( final ObjectMapper mapper, final KinesisSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, - final AWSCredentialsConfig awsCredentialsConfig + final AWSCredentialsConfig awsCredentialsConfig, + final SeekableStreamSupervisorConfig supervisorConfig ) { super( @@ -104,7 +106,8 @@ public KinesisSupervisor( mapper, spec, rowIngestionMetersFactory, - true + true, + supervisorConfig ); this.spec = spec; @@ -247,7 +250,9 @@ protected SeekableStreamSupervisorReportPayload createReportPayl numPartitions, ioConfig.getReplicas(), ioConfig.getTaskDuration().getMillis() / 1000, - spec.isSuspended() + spec.isSuspended(), + stateManager.getSupervisorState(), + stateManager.getExceptionEvents() ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java index fb08337b4642..09ace4698092 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java @@ -20,8 +20,10 @@ package org.apache.druid.indexing.kinesis.supervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import java.util.Collections; +import java.util.Queue; public class KinesisSupervisorReportPayload extends SeekableStreamSupervisorReportPayload { @@ -31,7 +33,9 @@ public KinesisSupervisorReportPayload( Integer partitions, Integer replicas, Long durationSeconds, - boolean suspended + boolean suspended, + SeekableStreamSupervisorStateManager.State state, + Queue recentErrors ) { super( @@ -44,7 +48,9 @@ public KinesisSupervisorReportPayload( Collections.emptyMap(), null, null, - suspended + suspended, + state, + recentErrors ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index 479851f8eb8c..fa4e862aa91d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -32,6 +32,7 @@ import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; @@ -58,7 +59,8 @@ public KinesisSupervisorSpec( @JacksonInject ServiceEmitter emitter, @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, - @JacksonInject @Named("kinesis") AWSCredentialsConfig awsCredentialsConfig + @JacksonInject @Named("kinesis") AWSCredentialsConfig awsCredentialsConfig, + @JacksonInject SeekableStreamSupervisorConfig supervisorConfig ) { super( @@ -106,7 +108,8 @@ public KinesisSupervisorSpec( mapper, emitter, monitorSchedulerConfig, - rowIngestionMetersFactory + rowIngestionMetersFactory, + supervisorConfig ); this.awsCredentialsConfig = awsCredentialsConfig; } @@ -123,7 +126,8 @@ public Supervisor createSupervisor() mapper, this, rowIngestionMetersFactory, - awsCredentialsConfig + awsCredentialsConfig, + supervisorConfig ); } @@ -170,7 +174,8 @@ protected KinesisSupervisorSpec toggleSuspend(boolean suspend) emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - awsCredentialsConfig + awsCredentialsConfig, + supervisorConfig ); } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 73c1d4692faa..3d760af89412 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -58,8 +58,10 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -111,8 +113,6 @@ public class KinesisSupervisorTest extends EasyMockSupport { - - private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); private static final String DATASOURCE = "testDS"; private static final int TEST_CHAT_THREADS = 3; @@ -120,10 +120,10 @@ public class KinesisSupervisorTest extends EasyMockSupport private static final Period TEST_HTTP_TIMEOUT = new Period("PT10S"); private static final Period TEST_SHUTDOWN_TIMEOUT = new Period("PT80S"); private static final String stream = "stream"; - private static String shardId1 = "1"; - private static String shardId0 = "0"; - private static StreamPartition shard1Partition = StreamPartition.of(stream, shardId1); - private static StreamPartition shard0Partition = StreamPartition.of(stream, shardId0); + private static final String shardId1 = "1"; + private static final String shardId0 = "0"; + private static final StreamPartition shard1Partition = StreamPartition.of(stream, shardId1); + private static final StreamPartition shard0Partition = StreamPartition.of(stream, shardId0); private static DataSchema dataSchema; private KinesisRecordSupplier supervisorRecordSupplier; @@ -1422,12 +1422,14 @@ public void testDiscoverExistingPublishingTask() throws Exception KinesisSupervisorReportPayload payload = report.getPayload(); Assert.assertEquals(DATASOURCE, payload.getDataSource()); - Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); - Assert.assertEquals(2, (int) payload.getPartitions()); - Assert.assertEquals(1, (int) payload.getReplicas()); + Assert.assertEquals(3600L, payload.getDurationSeconds()); + Assert.assertEquals(2, payload.getPartitions()); + Assert.assertEquals(1, payload.getReplicas()); Assert.assertEquals(stream, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData publishingReport = payload.getPublishingTasks().get(0); @@ -1559,12 +1561,14 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() KinesisSupervisorReportPayload payload = report.getPayload(); Assert.assertEquals(DATASOURCE, payload.getDataSource()); - Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); - Assert.assertEquals(2, (int) payload.getPartitions()); - Assert.assertEquals(1, (int) payload.getReplicas()); + Assert.assertEquals(3600L, payload.getDurationSeconds()); + Assert.assertEquals(2, payload.getPartitions()); + Assert.assertEquals(1, payload.getReplicas()); Assert.assertEquals(stream, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData publishingReport = payload.getPublishingTasks().get(0); @@ -1747,12 +1751,14 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception KinesisSupervisorReportPayload payload = report.getPayload(); Assert.assertEquals(DATASOURCE, payload.getDataSource()); - Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); - Assert.assertEquals(2, (int) payload.getPartitions()); - Assert.assertEquals(1, (int) payload.getReplicas()); + Assert.assertEquals(3600L, payload.getDurationSeconds()); + Assert.assertEquals(2, payload.getPartitions()); + Assert.assertEquals(1, payload.getReplicas()); Assert.assertEquals(stream, payload.getStream()); Assert.assertEquals(1, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData activeReport = payload.getActiveTasks().get(0); TaskReportData publishingReport = payload.getPublishingTasks().get(0); @@ -3467,7 +3473,8 @@ public KinesisIndexTaskClient build( new NoopServiceEmitter(), new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, - null + null, + new SeekableStreamSupervisorConfig() ), rowIngestionMetersFactory ); @@ -3643,7 +3650,8 @@ public KinesisIndexTaskClient build( new NoopServiceEmitter(), new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, - null + null, + new SeekableStreamSupervisorConfig() ), rowIngestionMetersFactory ); @@ -3705,7 +3713,8 @@ public TestableKinesisSupervisor( mapper, spec, rowIngestionMetersFactory, - null + null, + new SeekableStreamSupervisorConfig() ); this.spec = spec; } diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java new file mode 100644 index 000000000000..bb56610f73f3 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; + +public class SeekableStreamSupervisorModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.supervisor.stream", SeekableStreamSupervisorConfig.class); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 5161f9bcba93..3514e436dbab 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -223,11 +223,11 @@ public IndexTask( this.authorizerMapper = authorizerMapper; this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() > 0) { - determinePartitionsSavedParseExceptions = new CircularBuffer( + determinePartitionsSavedParseExceptions = new CircularBuffer<>( ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() ); - buildSegmentsSavedParseExceptions = new CircularBuffer( + buildSegmentsSavedParseExceptions = new CircularBuffer<>( ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorConfig.java new file mode 100644 index 000000000000..f5041d1ecafc --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorConfig.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.seekablestream; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; + +import javax.validation.constraints.Min; + +public class SeekableStreamSupervisorConfig +{ + @JsonProperty + private boolean storingStackTraces = false; + + // The number of runs failed before the supervisor flips from a RUNNING to an UNHEALTHY state + @JsonProperty + @Min(3) + private int unhealthinessThreshold = 3; + + // The number of successful before the supervisor flips from an UNHEALTHY to a RUNNING state + @JsonProperty + @Min(3) + private int healthinessThreshold = 3; + + // The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state + @JsonProperty + @Min(3) + private int taskUnhealthinessThreshold = 3; + + // The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state + @JsonProperty + @Min(3) + private int taskHealthinessThreshold = 3; + + // The maximum number of exception events that can be returned through the supervisor status endpoint + @JsonProperty + private int maxStoredExceptionEvents = Math.max(unhealthinessThreshold, healthinessThreshold); + + public boolean isStoringStackTraces() + { + return storingStackTraces; + } + + public int getUnhealthinessThreshold() + { + return unhealthinessThreshold; + } + + public int getHealthinessThreshold() + { + return healthinessThreshold; + } + + public int getTaskUnhealthinessThreshold() + { + return taskUnhealthinessThreshold; + } + + public int getTaskHealthinessThreshold() + { + return taskHealthinessThreshold; + } + + public int getMaxStoredExceptionEvents() + { + return maxStoredExceptionEvents; + } + + @VisibleForTesting + public void setMaxStoredExceptionEvents(int maxStoredExceptionEvents) + { + this.maxStoredExceptionEvents = maxStoredExceptionEvents; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/NonTransientStreamException.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/NonTransientStreamException.java new file mode 100644 index 000000000000..546bbed3775c --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/NonTransientStreamException.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.seekablestream.exceptions; + +/** + * An exception wrapper for stream (Kafka/Kinesis) exceptions that are not expected to resolve over time. An example + * of such is some sort of auth exception. + */ + +public class NonTransientStreamException extends RuntimeException +{ + public NonTransientStreamException(Throwable t) + { + super(t); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/PossiblyTransientStreamException.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/PossiblyTransientStreamException.java new file mode 100644 index 000000000000..d0b0627e19c1 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/PossiblyTransientStreamException.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.seekablestream.exceptions; + +/** + * An exception wrapper for stream (Kafka/Kinesis) exceptions that might resolve over time. An example + * of such is some sort of connectivity exception that could signify that the resource doesn't exist + * or that there's some temporary network issue. + */ + +public class PossiblyTransientStreamException extends RuntimeException +{ + public PossiblyTransientStreamException(Throwable t) + { + super(t); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/TransientStreamException.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/TransientStreamException.java new file mode 100644 index 000000000000..c0514cf3631a --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/TransientStreamException.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.seekablestream.exceptions; + +/** + * An exception wrapper for stream (Kafka/Kinesis) exceptions that will resolve over time. An example + * of such is some sort of rate limit exceeded exception. + */ + +public class TransientStreamException extends RuntimeException +{ + public TransientStreamException(Throwable t) + { + super(t); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 265b3a329360..9548aef6a5ff 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -38,6 +38,7 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.commons.codec.digest.DigestUtils; import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IndexTaskClient; import org.apache.druid.indexing.common.TaskInfoProvider; @@ -61,6 +62,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -422,7 +424,6 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) } } - // Map<{group RandomIdUtils}, {actively reading task group}>; see documentation for TaskGroup class private final ConcurrentHashMap activelyReadingTaskGroups = new ConcurrentHashMap<>(); @@ -446,6 +447,7 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) protected final ObjectMapper sortingMapper; protected final List partitionIds = new CopyOnWriteArrayList<>(); + protected final SeekableStreamSupervisorStateManager stateManager; protected volatile DateTime sequenceLastUpdated; @@ -482,7 +484,6 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) private volatile boolean stopped = false; private volatile boolean lifecycleStarted = false; - public SeekableStreamSupervisor( final String supervisorId, final TaskStorage taskStorage, @@ -492,7 +493,8 @@ public SeekableStreamSupervisor( final ObjectMapper mapper, final SeekableStreamSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, - final boolean useExclusiveStartingSequence + final boolean useExclusiveStartingSequence, + final SeekableStreamSupervisorConfig streamSupervisorConfig ) { this.taskStorage = taskStorage; @@ -511,6 +513,10 @@ public SeekableStreamSupervisor( this.exec = Execs.singleThreaded(supervisorId); this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); this.reportingExec = Execs.scheduledSingleThreaded(supervisorId + "-Reporting-%d"); + this.stateManager = new SeekableStreamSupervisorStateManager( + SeekableStreamSupervisorStateManager.State.WAITING_TO_RUN, + streamSupervisorConfig + ); int workerThreads = (this.tuningConfig.getWorkerThreads() != null ? this.tuningConfig.getWorkerThreads() @@ -678,6 +684,7 @@ public void stop(boolean stopGracefully) log.info("[%s] has stopped", supervisorId); } catch (Exception e) { + stateManager.storeThrowableEvent(e); log.makeAlert(e, "Exception stopping [%s]", supervisorId) .emit(); } @@ -723,6 +730,7 @@ public void tryInit() notice.handle(); } catch (Throwable e) { + stateManager.storeThrowableEvent(e); log.makeAlert(e, "SeekableStreamSupervisor[%s] failed to handle notice", dataSource) .addData("noticeClass", notice.getClass().getSimpleName()) .emit(); @@ -730,6 +738,7 @@ public void tryInit() } } catch (InterruptedException e) { + stateManager.storeThrowableEvent(e); log.info("SeekableStreamSupervisor[%s] interrupted, exiting", dataSource); } } @@ -753,6 +762,7 @@ public void tryInit() ); } catch (Exception e) { + stateManager.storeThrowableEvent(e); if (recordSupplier != null) { recordSupplier.close(); } @@ -854,6 +864,7 @@ private SupervisorReport> getStats() return getCurrentTotalStats(); } catch (InterruptedException ie) { + stateManager.storeThrowableEvent(ie); Thread.currentThread().interrupt(); log.error(ie, "getStats() interrupted."); throw new RuntimeException(ie); } catch (ExecutionException | TimeoutException eete) { + stateManager.storeThrowableEvent(eete); throw new RuntimeException(eete); } } @@ -999,7 +1012,9 @@ public void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { possiblyRegisterListener(); + stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); updatePartitionDataFromStream(); + stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); discoverTasks(); updateTaskStatus(); checkTaskDuration(); @@ -1009,10 +1024,13 @@ public void runInternal() // if suspended, ensure tasks have been requested to gracefully stop if (!spec.isSuspended()) { log.info("[%s] supervisor is running.", dataSource); + stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); createNewTasks(); + stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.RUNNING); } else { log.info("[%s] supervisor is suspended.", dataSource); gracefulShutdownInternal(); + stateManager.setState(SeekableStreamSupervisorStateManager.State.SUSPENDED); } if (log.isDebugEnabled()) { @@ -1020,6 +1038,7 @@ public void runInternal() } else { log.info(generateReport(false).toString()); } + stateManager.markRunFinishedAndEvaluateHealth(); } private void possiblyRegisterListener() @@ -1148,17 +1167,24 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata(dataSource, newMetadata); } catch (IOException e) { + stateManager.storeThrowableEvent(e); log.error("Resetting DataSourceMetadata failed [%s]", e.getMessage()); throw new RuntimeException(e); } } if (metadataUpdateSuccess) { - resetMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap().keySet().forEach(partition -> { - final int groupId = getTaskGroupIdForPartition(partition); - killTaskGroupForPartitions(ImmutableSet.of(partition), "DataSourceMetadata is updated while reset"); - activelyReadingTaskGroups.remove(groupId); - partitionGroups.get(groupId).replaceAll((partitionId, sequence) -> getNotSetMarker()); - }); + resetMetadata.getSeekableStreamSequenceNumbers() + .getPartitionSequenceNumberMap() + .keySet() + .forEach(partition -> { + final int groupId = getTaskGroupIdForPartition(partition); + killTaskGroupForPartitions( + ImmutableSet.of(partition), + "DataSourceMetadata is updated while reset" + ); + activelyReadingTaskGroups.remove(groupId); + partitionGroups.get(groupId).replaceAll((partitionId, sequence) -> getNotSetMarker()); + }); } else { throw new ISE("Unable to reset metadata"); } @@ -1170,8 +1196,6 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) ); } } - - } private void killTask(final String id, String reasonFormat, Object... args) @@ -1313,6 +1337,7 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { + stateManager.storeThrowableEvent(e); log.warn(e, "Exception while stopping task"); } return false; @@ -1329,6 +1354,7 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { + stateManager.storeThrowableEvent(e); log.warn(e, "Exception while stopping task"); } return false; @@ -1366,6 +1392,7 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) return true; } catch (Throwable t) { + stateManager.storeThrowableEvent(t); log.error(t, "Something bad while discovering task [%s]", taskId); return null; } @@ -1404,6 +1431,7 @@ private void verifyAndMergeCheckpoints(final Collection taskGroupsToV Futures.allAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { + stateManager.storeThrowableEvent(e); throw new RuntimeException(e); } } @@ -1423,10 +1451,11 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) final List taskIds = new ArrayList<>(); for (String taskId : taskGroup.taskIds()) { - final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync( - taskId, - true - ); + final ListenableFuture>> checkpointsFuture = + taskClient.getCheckpointsAsync( + taskId, + true + ); futures.add(checkpointsFuture); taskIds.add(taskId); } @@ -1443,6 +1472,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) futures.get(i).get(); } catch (Exception e) { + stateManager.storeThrowableEvent(e); log.error(e, "Problem while getting checkpoints for task [%s], killing the task", taskId); killTask(taskId, "Exception[%s] while getting checkpoints", e.getClass()); taskGroup.tasks.remove(taskId); @@ -1455,6 +1485,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) } } catch (Exception e) { + stateManager.storeThrowableEvent(e); throw new RuntimeException(e); } @@ -1705,6 +1736,7 @@ private void updatePartitionDataFromStream() } } catch (Exception e) { + stateManager.storeThrowableEvent(e); log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getStream()); log.debug(e, "full stack trace"); return; @@ -1960,6 +1992,7 @@ public Map apply(List makeSequenceNumber(SequenceOff return makeSequenceNumber(seq, false); } - // exposed for testing for visibility into initialization state @VisibleForTesting public boolean isStarted() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index 29547c335a66..6f753ca18788 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -28,6 +28,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Queue; @JsonInclude(JsonInclude.Include.NON_NULL) public abstract class SeekableStreamSupervisorReportPayload @@ -44,6 +45,8 @@ public abstract class SeekableStreamSupervisorReportPayload recentErrors; public SeekableStreamSupervisorReportPayload( String dataSource, @@ -55,7 +58,9 @@ public SeekableStreamSupervisorReportPayload( @Nullable Map minimumLag, @Nullable Long aggregateLag, @Nullable DateTime offsetsLastUpdated, - boolean suspended + boolean suspended, + SeekableStreamSupervisorStateManager.State state, + Queue recentErrors ) { this.dataSource = dataSource; @@ -70,6 +75,8 @@ public SeekableStreamSupervisorReportPayload( this.aggregateLag = aggregateLag; this.offsetsLastUpdated = offsetsLastUpdated; this.suspended = suspended; + this.state = state; + this.recentErrors = recentErrors; } public void addTask(TaskReportData data) @@ -154,4 +161,16 @@ public DateTime getOffsetsLastUpdated() { return offsetsLastUpdated; } + + @JsonProperty + public SeekableStreamSupervisorStateManager.State getState() + { + return state; + } + + @JsonProperty + public Queue getRecentErrors() + { + return recentErrors; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index 2eb885865e9e..2f760940db94 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; @@ -57,6 +58,7 @@ public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec protected final ServiceEmitter emitter; protected final DruidMonitorSchedulerConfig monitorSchedulerConfig; private final boolean suspended; + protected final SeekableStreamSupervisorConfig supervisorConfig; @JsonCreator public SeekableStreamSupervisorSpec( @@ -72,7 +74,8 @@ public SeekableStreamSupervisorSpec( @JacksonInject @Json ObjectMapper mapper, @JacksonInject ServiceEmitter emitter, @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, - @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, + @JacksonInject SeekableStreamSupervisorConfig supervisorConfig ) { this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); @@ -89,6 +92,7 @@ public SeekableStreamSupervisorSpec( this.monitorSchedulerConfig = monitorSchedulerConfig; this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.suspended = suspended != null ? suspended : false; + this.supervisorConfig = supervisorConfig; } @JsonProperty diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java new file mode 100644 index 000000000000..ad7019375310 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java @@ -0,0 +1,335 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.seekablestream.supervisor; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; +import org.apache.druid.indexing.seekablestream.exceptions.NonTransientStreamException; +import org.apache.druid.indexing.seekablestream.exceptions.PossiblyTransientStreamException; +import org.apache.druid.indexing.seekablestream.exceptions.TransientStreamException; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.utils.CircularBuffer; +import org.codehaus.plexus.util.ExceptionUtils; +import org.joda.time.DateTime; + +import java.util.HashSet; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; + +public class SeekableStreamSupervisorStateManager +{ + public enum State + { + // Error states are ordered from high to low priority + UNHEALTHY_SUPERVISOR(1), + UNHEALTHY_TASKS(2), + UNABLE_TO_CONNECT_TO_STREAM(3), + LOST_CONTACT_WITH_STREAM(4), + // Non-error states are equal priority + WAITING_TO_RUN(5), + CONNECTING_TO_STREAM(5), + DISCOVERING_INITIAL_TASKS(5), + CREATING_TASKS(5), + RUNNING(5), + SUSPENDED(5), + SHUTTING_DOWN(5); + + // Lower priority number means higher priority and vice versa + private final int priority; + + State(int priority) + { + this.priority = priority; + } + + public boolean isHealthy() + { + Set unhealthyStates = ImmutableSet.of( + UNHEALTHY_SUPERVISOR, + UNHEALTHY_TASKS, + UNABLE_TO_CONNECT_TO_STREAM, + LOST_CONTACT_WITH_STREAM + ); + return !unhealthyStates.contains(this); + } + } + + private State supervisorState; + // Remove all throwableEvents that aren't in this set at the end of each run (transient) + private final int healthinessThreshold; + private final int unhealthinessThreshold; + private final int healthinessTaskThreshold; + private final int unhealthinessTaskThreshold; + + private boolean atLeastOneSuccessfulRun = false; + private boolean currentRunSuccessful = true; + private final CircularBuffer completedTaskHistory; + private final CircularBuffer stateHistory; // From previous runs + private final ExceptionEventStore eventStore; + + public SeekableStreamSupervisorStateManager( + State initialState, + SeekableStreamSupervisorConfig supervisorConfig + ) + { + this.supervisorState = initialState; + this.healthinessThreshold = supervisorConfig.getHealthinessThreshold(); + this.unhealthinessThreshold = supervisorConfig.getUnhealthinessThreshold(); + this.healthinessTaskThreshold = supervisorConfig.getTaskHealthinessThreshold(); + this.unhealthinessTaskThreshold = supervisorConfig.getTaskUnhealthinessThreshold(); + Preconditions.checkArgument( + supervisorConfig.getMaxStoredExceptionEvents() >= + Math.max(healthinessThreshold, unhealthinessThreshold), + "numExceptionEventsToStore must be greater than or equal to both " + + "healthinessThreshold and unhealthinessThreshold" + ); + this.eventStore = new ExceptionEventStore( + supervisorConfig.getMaxStoredExceptionEvents(), + supervisorConfig.isStoringStackTraces() + ); + this.completedTaskHistory = new CircularBuffer<>(Math.max(healthinessTaskThreshold, unhealthinessTaskThreshold)); + this.stateHistory = new CircularBuffer<>(Math.max(healthinessThreshold, unhealthinessThreshold)); + } + + public Optional setStateIfNoSuccessfulRunYet(State newState) + { + if (!atLeastOneSuccessfulRun) { + return Optional.of(setState(newState)); + } + return Optional.absent(); + } + + public State setState(State newState) + { + if (newState.equals(State.SUSPENDED)) { + atLeastOneSuccessfulRun = false; // We want the startup states again after being suspended + } + this.supervisorState = newState; + return newState; + } + + public void storeThrowableEvent(Throwable t) + { + if (t instanceof PossiblyTransientStreamException && atLeastOneSuccessfulRun) { + t = new TransientStreamException(t); + } + + eventStore.storeThrowable(t); + currentRunSuccessful = false; + } + + public void storeCompletedTaskState(TaskState state) + { + completedTaskHistory.add(state); + } + + public void markRunFinishedAndEvaluateHealth() + { + if (currentRunSuccessful) { + atLeastOneSuccessfulRun = true; + } + + State currentRunState = State.RUNNING; + + for (Map.Entry> events : eventStore.getNonTransientRecentEvents().entrySet()) { + if (events.getValue().size() >= unhealthinessThreshold) { + if (events.getKey().equals(NonTransientStreamException.class)) { + currentRunState = getHigherPriorityState(currentRunState, State.UNABLE_TO_CONNECT_TO_STREAM); + } else if (events.getKey().equals(TransientStreamException.class) || + events.getKey().equals(PossiblyTransientStreamException.class)) { + currentRunState = getHigherPriorityState(currentRunState, State.LOST_CONTACT_WITH_STREAM); + } else { + currentRunState = getHigherPriorityState(currentRunState, State.UNHEALTHY_SUPERVISOR); + } + } + } + + // Evaluate task health + if (supervisorState == State.UNHEALTHY_TASKS) { + boolean tasksHealthy = completedTaskHistory.size() >= healthinessTaskThreshold; + for (int i = 0; i < Math.min(healthinessTaskThreshold, completedTaskHistory.size()); i++) { + if (completedTaskHistory.getLatest(i) != TaskState.SUCCESS) { + tasksHealthy = false; + } + } + if (tasksHealthy && currentRunState == State.UNHEALTHY_TASKS) { + currentRunState = State.RUNNING; + } else if (!tasksHealthy) { + currentRunState = State.UNHEALTHY_TASKS; + } + } else { + boolean tasksUnhealthy = completedTaskHistory.size() >= unhealthinessTaskThreshold; + for (int i = 0; i < Math.min(unhealthinessTaskThreshold, completedTaskHistory.size()); i++) { + // Last unhealthinessTaskThreshold tasks must be unhealthy for state to change to + // UNHEALTHY_TASKS + if (completedTaskHistory.getLatest(i) != TaskState.FAILED) { + tasksUnhealthy = false; + } + } + + if (tasksUnhealthy) { + currentRunState = getHigherPriorityState(currentRunState, State.UNHEALTHY_TASKS); + } + } + + stateHistory.add(currentRunState); + + if (currentRunState.isHealthy() && supervisorState == State.UNHEALTHY_SUPERVISOR) { + currentRunState = State.UNHEALTHY_SUPERVISOR; + boolean supervisorHealthy = stateHistory.size() >= healthinessThreshold; + for (int i = 0; i < Math.min(healthinessThreshold, stateHistory.size()); i++) { + if (!stateHistory.getLatest(i).isHealthy()) { + supervisorHealthy = false; + } + } + if (supervisorHealthy) { + currentRunState = State.RUNNING; + } + } + + setState(currentRunState); + + // Reset manager state for next run + currentRunSuccessful = true; + eventStore.resetErrorsEncounteredOnRun(); + } + + public Queue getExceptionEvents() + { + return eventStore.getRecentEvents(); + } + + public State getSupervisorState() + { + return supervisorState; + } + + private State getHigherPriorityState(State s1, State s2) + { + return s1.priority < s2.priority ? s1 : s2; + } + + @JsonPropertyOrder({"timestamp", "exceptionClass", "message", "stackTrace"}) + public class ExceptionEvent + { + private Class clazz; + // Contains full stackTrace if storingStackTraces is true + private String errorMessage; + private DateTime timestamp; + + public ExceptionEvent( + Throwable t, + boolean storingStackTraces + ) + { + this.clazz = t.getClass(); + this.errorMessage = storingStackTraces ? ExceptionUtils.getStackTrace(t) : t.getMessage(); + this.timestamp = DateTimes.nowUtc(); + } + + @JsonProperty + public Class getExceptionClass() + { + return clazz; + } + + @JsonProperty + public String getErrorMessage() + { + return errorMessage; + } + + @JsonProperty + public DateTime getTimestamp() + { + return timestamp; + } + } + + private class ExceptionEventStore + { + private final Queue recentEventsQueue; + private final ConcurrentHashMap> recentEventsMap; + private final int numEventsToStore; + private final boolean storeStackTraces; + private final Set errorsEncounteredOnRun; + + public ExceptionEventStore(int numEventsToStore, boolean storeStackTraces) + { + this.recentEventsQueue = new ConcurrentLinkedQueue<>(); + this.recentEventsMap = new ConcurrentHashMap<>(numEventsToStore); + this.numEventsToStore = numEventsToStore; + this.storeStackTraces = storeStackTraces; + this.errorsEncounteredOnRun = new HashSet<>(); + } + + public void storeThrowable(Throwable t) + { + Queue exceptionEventsForClassT = recentEventsMap.getOrDefault( + t.getClass(), + new ConcurrentLinkedQueue<>() + ); + + ExceptionEvent eventToAdd = new ExceptionEvent(t, storeStackTraces); + + recentEventsQueue.add(eventToAdd); + + if (recentEventsQueue.size() > numEventsToStore) { + ExceptionEvent removedEvent = recentEventsQueue.poll(); + recentEventsMap.get(removedEvent.getExceptionClass()).poll(); + } + exceptionEventsForClassT.add(eventToAdd); + errorsEncounteredOnRun.add(t.getClass()); + recentEventsMap.put(t.getClass(), exceptionEventsForClassT); + } + + public void resetErrorsEncounteredOnRun() + { + errorsEncounteredOnRun.clear(); + } + + public Queue getRecentEvents() + { + return recentEventsQueue; + } + + public ConcurrentHashMap> getNonTransientRecentEvents() + { + ConcurrentHashMap> nonTransientRecentEventsMap = + new ConcurrentHashMap<>(recentEventsMap); + + for (Class throwableClass : recentEventsMap.keySet()) { + if (!errorsEncounteredOnRun.contains(throwableClass)) { + nonTransientRecentEventsMap.remove(throwableClass); + } + } + + return nonTransientRecentEventsMap; + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java new file mode 100644 index 000000000000..bb88d8cd162b --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.seekablestream.supervisor; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; +import org.apache.druid.indexing.seekablestream.exceptions.NonTransientStreamException; +import org.apache.druid.indexing.seekablestream.exceptions.PossiblyTransientStreamException; +import org.apache.druid.indexing.seekablestream.exceptions.TransientStreamException; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; +import java.util.Queue; + +public class SeekableStreamSupervisorStateManagerTest +{ + private SeekableStreamSupervisorStateManager stateManager; + private SeekableStreamSupervisorConfig config; + + @Before + public void setupTest() + { + config = new SeekableStreamSupervisorConfig(); + config.setMaxStoredExceptionEvents(10); + stateManager = new SeekableStreamSupervisorStateManager( + SeekableStreamSupervisorStateManager.State.WAITING_TO_RUN, + config + ); + } + + @Test + public void testHappyPath() + { + stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM, + stateManager.getSupervisorState() + ); + stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS, + stateManager.getSupervisorState() + ); + stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.CREATING_TASKS, stateManager.getSupervisorState()); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); + stateManager.markRunFinishedAndEvaluateHealth(); + + stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); + stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); + stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); + stateManager.markRunFinishedAndEvaluateHealth(); + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); + } + + @Test + public void testTransientStreamFailure() + { + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.markRunFinishedAndEvaluateHealth(); // clean run without errors + for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); + stateManager.storeThrowableEvent(new PossiblyTransientStreamException(new Exception("DOH!"))); + stateManager.markRunFinishedAndEvaluateHealth(); + } + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.LOST_CONTACT_WITH_STREAM, + stateManager.getSupervisorState() + ); + } + + @Test + public void testNonTransientStreamFailure() + { + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); + stateManager.storeThrowableEvent(new NonTransientStreamException(new Exception("DOH!"))); + stateManager.markRunFinishedAndEvaluateHealth(); + } + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.UNABLE_TO_CONNECT_TO_STREAM, + stateManager.getSupervisorState() + ); + } + + @Test + public void testPossiblyTransientStreamFailure() + { + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); + stateManager.storeThrowableEvent(new PossiblyTransientStreamException(new Exception("DOH!"))); + stateManager.markRunFinishedAndEvaluateHealth(); + } + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.LOST_CONTACT_WITH_STREAM, + stateManager.getSupervisorState() + ); + } + + @Test + public void testNonTransientUnhealthiness() + { + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { + Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); + stateManager.storeThrowableEvent(new NullPointerException("oof")); + stateManager.markRunFinishedAndEvaluateHealth(); + } + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.UNHEALTHY_SUPERVISOR, + stateManager.getSupervisorState() + ); + } + + @Test + public void testTransientUnhealthiness() + { + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + for (int i = 0; i < config.getUnhealthinessThreshold() - 1; i++) { + stateManager.storeThrowableEvent(new NullPointerException("oof")); + stateManager.markRunFinishedAndEvaluateHealth(); + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.RUNNING, + stateManager.getSupervisorState() + ); + } + stateManager.markRunFinishedAndEvaluateHealth(); // clean run + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.RUNNING, + stateManager.getSupervisorState() + ); + } + + @Test + public void testNonTransientTaskUnhealthiness() + { + for (int i = 0; i < config.getTaskUnhealthinessThreshold(); i++) { + Assert.assertNotEquals( + SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS, + stateManager.getSupervisorState() + ); + stateManager.storeCompletedTaskState(TaskState.FAILED); + stateManager.markRunFinishedAndEvaluateHealth(); + } + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS, + stateManager.getSupervisorState() + ); + } + + @Test + public void testTransientTaskUnhealthiness() + { + // Only half are failing + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + for (int i = 0; i < config.getTaskUnhealthinessThreshold() + 3; i++) { + Assert.assertNotEquals( + stateManager.getSupervisorState(), + SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS + ); + stateManager.storeCompletedTaskState(TaskState.FAILED); + stateManager.storeCompletedTaskState(TaskState.SUCCESS); + stateManager.markRunFinishedAndEvaluateHealth(); + } + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.RUNNING, + stateManager.getSupervisorState() + ); + } + + @Test + public void testSupervisorRecoveryWithHealthinessThreshold() + { + // Put into an unhealthy state + for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { + Assert.assertNotEquals( + SeekableStreamSupervisorStateManager.State.UNHEALTHY_SUPERVISOR, + stateManager.getSupervisorState() + ); + stateManager.storeThrowableEvent(new Exception("Except the inevitable")); + stateManager.markRunFinishedAndEvaluateHealth(); + } + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.UNHEALTHY_SUPERVISOR, + stateManager.getSupervisorState() + ); + // Recover after config.healthinessThreshold successful task completions + for (int i = 0; i < config.getHealthinessThreshold(); i++) { + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.UNHEALTHY_SUPERVISOR, + stateManager.getSupervisorState() + ); + stateManager.markRunFinishedAndEvaluateHealth(); + } + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.RUNNING, + stateManager.getSupervisorState() + ); + } + + @Test + public void testTaskRecoveryWithHealthinessThreshold() + { + // Put into an unhealthy state + for (int i = 0; i < config.getTaskUnhealthinessThreshold(); i++) { + Assert.assertNotEquals( + SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS, + stateManager.getSupervisorState() + ); + stateManager.storeCompletedTaskState(TaskState.FAILED); + stateManager.markRunFinishedAndEvaluateHealth(); + } + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS, + stateManager.getSupervisorState() + ); + // Recover after config.healthinessThreshold successful task completions + for (int i = 0; i < config.getTaskHealthinessThreshold(); i++) { + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS, + stateManager.getSupervisorState() + ); + stateManager.storeCompletedTaskState(TaskState.SUCCESS); + stateManager.markRunFinishedAndEvaluateHealth(); + } + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.RUNNING, + stateManager.getSupervisorState() + ); + } + + @Test + public void testTwoUnhealthyStates() + { + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + for (int i = 0; i < Math.max( + config.getTaskUnhealthinessThreshold(), + config.getUnhealthinessThreshold() + ); i++) { + stateManager.storeThrowableEvent(new NullPointerException("somebody goofed")); + stateManager.storeCompletedTaskState(TaskState.FAILED); + stateManager.markRunFinishedAndEvaluateHealth(); + } + // UNHEALTHY_SUPERVISOR should take priority over UNHEALTHY_TASKS + Assert.assertEquals( + SeekableStreamSupervisorStateManager.State.UNHEALTHY_SUPERVISOR, + stateManager.getSupervisorState() + ); + } + + @Test + public void testGetThrowableEvents() + { + List exceptions = ImmutableList.of( + new PossiblyTransientStreamException(new Exception("oof")), + new NullPointerException("oof"), + new TransientStreamException(new Exception("oof")), + new NonTransientStreamException(new Exception("oof")) + ); + for (Exception exception : exceptions) { + stateManager.storeThrowableEvent(exception); + stateManager.markRunFinishedAndEvaluateHealth(); + } + stateManager.markRunFinishedAndEvaluateHealth(); + stateManager.storeThrowableEvent(new PossiblyTransientStreamException(new Exception("oof"))); + Queue events = stateManager.getExceptionEvents(); + + Assert.assertNotNull(events.peek().getErrorMessage()); + Assert.assertEquals(PossiblyTransientStreamException.class, events.poll().getExceptionClass()); + Assert.assertNotNull(events.peek().getErrorMessage()); + Assert.assertEquals(NullPointerException.class, events.poll().getExceptionClass()); + Assert.assertNotNull(events.peek().getErrorMessage()); + Assert.assertEquals(TransientStreamException.class, events.poll().getExceptionClass()); + Assert.assertNotNull(events.peek().getErrorMessage()); + Assert.assertEquals(NonTransientStreamException.class, events.poll().getExceptionClass()); + Assert.assertNotNull(events.peek().getErrorMessage()); + Assert.assertEquals(TransientStreamException.class, events.poll().getExceptionClass()); + } +} diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 4a1c22a92a61..644ea3e6205d 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -50,6 +50,7 @@ import org.apache.druid.guice.ListProvider; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.PolyBind; +import org.apache.druid.guice.SeekableStreamSupervisorModule; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; @@ -331,7 +332,8 @@ private void configureOverlordHelpers(Binder binder) } }, new IndexingServiceFirehoseModule(), - new IndexingServiceTaskLogsModule() + new IndexingServiceTaskLogsModule(), + new SeekableStreamSupervisorModule() ); } From 0f0e950d2b5a18f6961fc3cd6a6c3d24b54464fe Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 8 Apr 2019 15:57:24 -0700 Subject: [PATCH 02/22] Fixed nits in docs --- docs/content/development/extensions-core/kafka-ingestion.md | 6 +++--- .../development/extensions-core/kinesis-ingestion.md | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index d49b2a28397b..3c0cb337258f 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -222,8 +222,8 @@ follows: |-----|-----------|--------| |UNHEALTHY_SUPERVISOR|The supervisor has encountered non-transient errors on the past `druid.supervisor.stream.unhealthinessThreshold` iterations|1| |UNHEALTHY_TASKS|The last `druid.supervisor.stream.taskUnhealthinessThreshold` tasks have all failed|2| -|UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kinesis and has not successfully connected in the past|3| -|LOST_CONTACT_WITH_STREAM|The supervisor is encountering transient connectivity issues with Kinesis but has successfully connected in the past|4| +|UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kafka and has not successfully connected in the past|3| +|LOST_CONTACT_WITH_STREAM|The supervisor is encountering transient connectivity issues with Kafka but has successfully connected in the past|4| |WAITING_TO_RUN (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream.|5| |CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data|5| |DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks|5| @@ -378,4 +378,4 @@ Hadoop (see [here](https://github.com/apache/incubator-druid/pull/5102)). |druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state|An integer in [3,2147483647]|3| |druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state|An integer in [3,2147483647]|3| |druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|true/false|false| -|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|An integer in [`max(healthinessThreshold, unhealthinessThreshold)`, 2147483647]|`max(healthinessThreshold, unhealthinessThreshold)`| \ No newline at end of file +|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|An integer in [`max(healthinessThreshold, unhealthinessThreshold)`, 2147483647]|`max(healthinessThreshold, unhealthinessThreshold)`| diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md index a3579ae3d929..825420485f37 100644 --- a/docs/content/development/extensions-core/kinesis-ingestion.md +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -428,4 +428,4 @@ Then when submitting a supervisor-spec, set `deaggregate` to true. |druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state|An integer in [3,2147483647]|3| |druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state|An integer in [3,2147483647]|3| |druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|true/false|false| -|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|An integer in [`max(healthinessThreshold, unhealthinessThreshold)`, 2147483647]|`max(healthinessThreshold, unhealthinessThreshold)`| \ No newline at end of file +|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|An integer in [`max(healthinessThreshold, unhealthinessThreshold)`, 2147483647]|`max(healthinessThreshold, unhealthinessThreshold)`| From 6f44ff7b825c3c32c2c21f4ae5b972766ec8aec0 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 9 Apr 2019 10:54:56 -0700 Subject: [PATCH 03/22] Made inner class static and updated spec test with jackson inject --- .../indexing/kafka/supervisor/KafkaSupervisorSpecTest.java | 2 ++ .../supervisor/SeekableStreamSupervisorStateManager.java | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java index b4c6dfd29972..ac8fb2057a2b 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java @@ -28,6 +28,7 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.math.expr.ExprMacroTable; @@ -56,6 +57,7 @@ public KafkaSupervisorSpecTest() .addValue(ServiceEmitter.class, new NoopServiceEmitter()) .addValue(DruidMonitorSchedulerConfig.class, null) .addValue(RowIngestionMetersFactory.class, null) + .addValue(SeekableStreamSupervisorConfig.class, null) .addValue(ExprMacroTable.class.getName(), LookupEnabledTestExprMacroTable.INSTANCE) ); mapper.registerModules((Iterable) new KafkaIndexTaskModule().getJacksonModules()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java index ad7019375310..4966551faee8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java @@ -235,7 +235,7 @@ private State getHigherPriorityState(State s1, State s2) } @JsonPropertyOrder({"timestamp", "exceptionClass", "message", "stackTrace"}) - public class ExceptionEvent + public static class ExceptionEvent { private Class clazz; // Contains full stackTrace if storingStackTraces is true @@ -271,7 +271,7 @@ public DateTime getTimestamp() } } - private class ExceptionEventStore + private static class ExceptionEventStore { private final Queue recentEventsQueue; private final ConcurrentHashMap> recentEventsMap; From f8d35329b53cc235c5b0b01c454d8dd65a42cd34 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 15 Apr 2019 18:33:18 -0700 Subject: [PATCH 04/22] Review changes --- docs/content/development/extensions-core/kafka-ingestion.md | 4 ++-- .../supervisor/SeekableStreamSupervisorStateManager.java | 5 ++--- .../realtime/appenderator/BaseAppenderatorDriver.java | 2 +- .../realtime/appenderator/StreamAppenderatorDriver.java | 2 +- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 7c7a699c70ca..30813f97491e 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -223,7 +223,7 @@ follows: |UNHEALTHY_SUPERVISOR|The supervisor has encountered non-transient errors on the past `druid.supervisor.stream.unhealthinessThreshold` iterations|1| |UNHEALTHY_TASKS|The last `druid.supervisor.stream.taskUnhealthinessThreshold` tasks have all failed|2| |UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kafka and has not successfully connected in the past|3| -|LOST_CONTACT_WITH_STREAM|The supervisor is encountering transient connectivity issues with Kafka but has successfully connected in the past|4| +|LOST_CONTACT_WITH_STREAM|The supervisor is encountering connectivity issues with Kafka but has successfully connected in the past|4| |WAITING_TO_RUN (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream.|5| |CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data|5| |DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks|5| @@ -235,7 +235,7 @@ follows: Notes about states: - Since it's possible that 2+ states can apply to a supervisor at the same time, each state is given a priority. The -active state with the highest priority will be returned in the status report. +active state with the highest priority (i.e. lowest priority number) will be returned in the status report. - States marked with "first iteration only" only occur on the supervisor's first iteration at startup or after suspension. ### Getting Supervisor Ingestion Stats Report diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java index 4966551faee8..20329a329583 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java @@ -80,7 +80,6 @@ public boolean isHealthy() } private State supervisorState; - // Remove all throwableEvents that aren't in this set at the end of each run (transient) private final int healthinessThreshold; private final int unhealthinessThreshold; private final int healthinessTaskThreshold; @@ -177,9 +176,9 @@ public void markRunFinishedAndEvaluateHealth() tasksHealthy = false; } } - if (tasksHealthy && currentRunState == State.UNHEALTHY_TASKS) { + if (tasksHealthy) { currentRunState = State.RUNNING; - } else if (!tasksHealthy) { + } else { currentRunState = State.UNHEALTHY_TASKS; } } else { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java index f8b12ad52ce3..ab9eb52cf130 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java @@ -79,7 +79,7 @@ public abstract class BaseAppenderatorDriver implements Closeable { /** - * Segments allocated for an intervval. + * Segments allocated for an interval. * There should be at most a single active (appending) segment at any time. */ static class SegmentsOfInterval diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index 2599387dd158..c31ed775223e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -167,7 +167,7 @@ public AppenderatorDriverAddResult add( * * @return {@link AppenderatorDriverAddResult} * - * @throws IOException if there is an I/O error while allocating or writing to a segment + * @throws IOException if there is an I/O error while allocating or writing to a segmentq */ public AppenderatorDriverAddResult add( final InputRow row, From 7ad9858e3af1a6b4a55e4d6204ba099ff9056778 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 16 Apr 2019 18:13:06 -0700 Subject: [PATCH 05/22] Remove redundant config param in supervisor --- .../druid/utils/CircularBufferTest.java | 62 +++++++++++++++---- .../kafka/supervisor/KafkaSupervisor.java | 7 +-- .../kafka/supervisor/KafkaSupervisorSpec.java | 3 +- .../kafka/supervisor/KafkaSupervisorTest.java | 11 ++-- .../kinesis/supervisor/KinesisSupervisor.java | 7 +-- .../supervisor/KinesisSupervisorSpec.java | 3 +- .../supervisor/KinesisSupervisorTest.java | 13 ++-- .../supervisor/SeekableStreamSupervisor.java | 7 +-- .../SeekableStreamSupervisorSpec.java | 5 ++ .../ChangeRequestHistoryTest.java | 53 ---------------- 10 files changed, 79 insertions(+), 92 deletions(-) diff --git a/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java b/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java index 5d9d77534e07..6ca299b6938a 100644 --- a/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java +++ b/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java @@ -25,19 +25,14 @@ public class CircularBufferTest { - private static CircularBuffer buff = new CircularBuffer(4); - - @BeforeClass - public static void setup() + @Test + public void testCircularBufferGetLatest() { + CircularBuffer buff = new CircularBuffer(4); + for (int i = 1; i <= 9; i++) { buff.add(i); // buffer should contain [9, 6, 7, 8] } - } - - @Test - public void testCircularBufferGetLatest() - { for (int i = 0; i < 4; i++) { Assert.assertEquals((Integer) (9 - i), buff.getLatest(i)); } @@ -46,8 +41,53 @@ public void testCircularBufferGetLatest() @Test public void testCircularBufferGet() { - for (int i = 0; i < 4; i++) { - Assert.assertEquals((Integer) (i + 6), buff.get(i)); + CircularBuffer circularBuffer = new CircularBuffer<>( + 3); + + circularBuffer.add(1); + Assert.assertEquals(1, circularBuffer.size()); + Assert.assertEquals(1, (int) circularBuffer.get(0)); + + circularBuffer.add(2); + Assert.assertEquals(2, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i + 1, (int) circularBuffer.get(i)); + } + + circularBuffer.add(3); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i + 1, (int) circularBuffer.get(i)); + } + + circularBuffer.add(4); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i + 2, (int) circularBuffer.get(i)); + } + + circularBuffer.add(5); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i + 3, (int) circularBuffer.get(i)); + } + + circularBuffer.add(6); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i + 4, (int) circularBuffer.get(i)); + } + + circularBuffer.add(7); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i + 5, (int) circularBuffer.get(i)); + } + + circularBuffer.add(8); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i + 6, (int) circularBuffer.get(i)); } } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 56db89db6921..33af1c87539a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -43,7 +43,6 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -106,8 +105,7 @@ public KafkaSupervisor( final KafkaIndexTaskClientFactory taskClientFactory, final ObjectMapper mapper, final KafkaSupervisorSpec spec, - final RowIngestionMetersFactory rowIngestionMetersFactory, - final SeekableStreamSupervisorConfig supervisorConfig + final RowIngestionMetersFactory rowIngestionMetersFactory ) { super( @@ -119,8 +117,7 @@ public KafkaSupervisor( mapper, spec, rowIngestionMetersFactory, - false, - supervisorConfig + false ); this.spec = spec; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index d7cb41326525..cee365c309d8 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -112,8 +112,7 @@ public Supervisor createSupervisor() (KafkaIndexTaskClientFactory) indexTaskClientFactory, mapper, this, - rowIngestionMetersFactory, - supervisorConfig + rowIngestionMetersFactory ); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index abad4b419a7f..8c4cb55129b5 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -157,6 +157,7 @@ public class KafkaSupervisorTest extends EasyMockSupport private String topic; private RowIngestionMetersFactory rowIngestionMetersFactory; private ExceptionCapturingServiceEmitter serviceEmitter; + private SeekableStreamSupervisorConfig supervisorConfig; private static String getTopic() { @@ -239,6 +240,7 @@ public void setupTest() rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); serviceEmitter = new ExceptionCapturingServiceEmitter(); EmittingLogger.registerEmitter(serviceEmitter); + supervisorConfig = new SeekableStreamSupervisorConfig(); } @After @@ -3304,7 +3306,8 @@ public KafkaIndexTaskClient build( objectMapper, new NoopServiceEmitter(), new DruidMonitorSchedulerConfig(), - rowIngestionMetersFactory + rowIngestionMetersFactory, + supervisorConfig ), rowIngestionMetersFactory, isTaskCurrentReturn @@ -3387,7 +3390,8 @@ public KafkaIndexTaskClient build( objectMapper, new NoopServiceEmitter(), new DruidMonitorSchedulerConfig(), - rowIngestionMetersFactory + rowIngestionMetersFactory, + supervisorConfig ), rowIngestionMetersFactory ); @@ -3558,8 +3562,7 @@ public TestableKafkaSupervisor( taskClientFactory, mapper, spec, - rowIngestionMetersFactory, - new SeekableStreamSupervisorConfig() + rowIngestionMetersFactory ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index af232a71e237..d8b7aee1f719 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -45,7 +45,6 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -93,8 +92,7 @@ public KinesisSupervisor( final ObjectMapper mapper, final KinesisSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, - final AWSCredentialsConfig awsCredentialsConfig, - final SeekableStreamSupervisorConfig supervisorConfig + final AWSCredentialsConfig awsCredentialsConfig ) { super( @@ -106,8 +104,7 @@ public KinesisSupervisor( mapper, spec, rowIngestionMetersFactory, - true, - supervisorConfig + true ); this.spec = spec; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index fa4e862aa91d..e93f79d7c4e9 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -126,8 +126,7 @@ public Supervisor createSupervisor() mapper, this, rowIngestionMetersFactory, - awsCredentialsConfig, - supervisorConfig + awsCredentialsConfig ); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 4aa181af46c8..335a648e938b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -142,6 +142,7 @@ public class KinesisSupervisorTest extends EasyMockSupport private TaskQueue taskQueue; private RowIngestionMetersFactory rowIngestionMetersFactory; private ExceptionCapturingServiceEmitter serviceEmitter; + private SeekableStreamSupervisorConfig supervisorConfig; public KinesisSupervisorTest() { @@ -199,6 +200,7 @@ public void setupTest() rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); serviceEmitter = new ExceptionCapturingServiceEmitter(); EmittingLogger.registerEmitter(serviceEmitter); + supervisorConfig = new SeekableStreamSupervisorConfig(); } @After @@ -3880,7 +3882,8 @@ public KinesisIndexTaskClient build( new NoopServiceEmitter(), new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, - null + null, + supervisorConfig ), rowIngestionMetersFactory ); @@ -3963,7 +3966,8 @@ public KinesisIndexTaskClient build( new NoopServiceEmitter(), new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, - null + null, + supervisorConfig ), rowIngestionMetersFactory, isTaskCurrentReturn @@ -4049,7 +4053,7 @@ public KinesisIndexTaskClient build( new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, null, - new SeekableStreamSupervisorConfig() + supervisorConfig ), rowIngestionMetersFactory, null @@ -4230,8 +4234,7 @@ public TestableKinesisSupervisor( mapper, spec, rowIngestionMetersFactory, - null, - new SeekableStreamSupervisorConfig() + null ); this.spec = spec; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 98aaf11d58c7..2f890e937829 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -62,7 +62,6 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -519,8 +518,7 @@ public SeekableStreamSupervisor( final ObjectMapper mapper, final SeekableStreamSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, - final boolean useExclusiveStartingSequence, - final SeekableStreamSupervisorConfig streamSupervisorConfig + final boolean useExclusiveStartingSequence ) { this.taskStorage = taskStorage; @@ -530,7 +528,6 @@ public SeekableStreamSupervisor( this.spec = spec; this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.useExclusiveStartingSequence = useExclusiveStartingSequence; - this.dataSource = spec.getDataSchema().getDataSource(); this.ioConfig = spec.getIoConfig(); this.tuningConfig = spec.getTuningConfig(); @@ -541,7 +538,7 @@ public SeekableStreamSupervisor( this.reportingExec = Execs.scheduledSingleThreaded(supervisorId + "-Reporting-%d"); this.stateManager = new SeekableStreamSupervisorStateManager( SeekableStreamSupervisorStateManager.State.WAITING_TO_RUN, - streamSupervisorConfig + spec.getSupervisorConfig() ); int workerThreads = (this.tuningConfig.getWorkerThreads() != null diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index 2f760940db94..bc655074e526 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -157,6 +157,11 @@ public SeekableStreamSupervisorSpec createRunningSpec() return toggleSuspend(false); } + public SeekableStreamSupervisorConfig getSupervisorConfig() + { + return supervisorConfig; + } + @Override @JsonProperty("suspended") public boolean isSuspended() diff --git a/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java b/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java index d513aebb92d3..d5e0395dbc4d 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java @@ -170,57 +170,4 @@ public void testNonImmediateFuture() throws Exception Assert.assertEquals(1, snapshot.getCounter().getCounter()); Assert.assertEquals(1, snapshot.getRequests().size()); } - - @Test - public void testCircularBuffer() - { - CircularBuffer circularBuffer = new CircularBuffer<>( - 3); - - circularBuffer.add(1); - Assert.assertEquals(1, circularBuffer.size()); - Assert.assertEquals(1, (int) circularBuffer.get(0)); - - circularBuffer.add(2); - Assert.assertEquals(2, circularBuffer.size()); - for (int i = 0; i < circularBuffer.size(); i++) { - Assert.assertEquals(i + 1, (int) circularBuffer.get(i)); - } - - circularBuffer.add(3); - Assert.assertEquals(3, circularBuffer.size()); - for (int i = 0; i < circularBuffer.size(); i++) { - Assert.assertEquals(i + 1, (int) circularBuffer.get(i)); - } - - circularBuffer.add(4); - Assert.assertEquals(3, circularBuffer.size()); - for (int i = 0; i < circularBuffer.size(); i++) { - Assert.assertEquals(i + 2, (int) circularBuffer.get(i)); - } - - circularBuffer.add(5); - Assert.assertEquals(3, circularBuffer.size()); - for (int i = 0; i < circularBuffer.size(); i++) { - Assert.assertEquals(i + 3, (int) circularBuffer.get(i)); - } - - circularBuffer.add(6); - Assert.assertEquals(3, circularBuffer.size()); - for (int i = 0; i < circularBuffer.size(); i++) { - Assert.assertEquals(i + 4, (int) circularBuffer.get(i)); - } - - circularBuffer.add(7); - Assert.assertEquals(3, circularBuffer.size()); - for (int i = 0; i < circularBuffer.size(); i++) { - Assert.assertEquals(i + 5, (int) circularBuffer.get(i)); - } - - circularBuffer.add(8); - Assert.assertEquals(3, circularBuffer.size()); - for (int i = 0; i < circularBuffer.size(); i++) { - Assert.assertEquals(i + 6, (int) circularBuffer.get(i)); - } - } } From 8632a23eb7bdf13ee7ce2f6e7362f7919b53a97e Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 17 Apr 2019 13:21:57 -0700 Subject: [PATCH 06/22] Style --- .../test/java/org/apache/druid/utils/CircularBufferTest.java | 1 - .../druid/server/coordination/ChangeRequestHistoryTest.java | 5 +++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java b/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java index 6ca299b6938a..5e611708aa49 100644 --- a/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java +++ b/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java @@ -20,7 +20,6 @@ package org.apache.druid.utils; import org.junit.Assert; -import org.junit.BeforeClass; import org.junit.Test; public class CircularBufferTest diff --git a/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java b/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java index d5e0395dbc4d..82a67b038ed6 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ChangeRequestHistoryTest.java @@ -22,7 +22,6 @@ import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.utils.CircularBuffer; import org.junit.Assert; import org.junit.Test; @@ -31,6 +30,7 @@ import java.util.concurrent.atomic.AtomicBoolean; /** + * */ public class ChangeRequestHistoryTest { @@ -43,7 +43,8 @@ public void testSimple() throws Exception history.addChangeRequest(new SegmentChangeRequestNoop()); Assert.assertEquals(1, history.getLastCounter().getCounter()); - ChangeRequestsSnapshot snapshot = history.getRequestsSince(ChangeRequestHistory.Counter.ZERO).get(); + ChangeRequestsSnapshot snapshot = history.getRequestsSince(ChangeRequestHistory.Counter.ZERO) + .get(); Assert.assertEquals(1, snapshot.getRequests().size()); Assert.assertEquals(1, snapshot.getCounter().getCounter()); From 9e5a20d86e7e7000101debdb1b2c3584caac266f Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 19 Apr 2019 15:51:53 -0700 Subject: [PATCH 07/22] Applied some of Jon's recommendations --- .../supervisor/SeekableStreamSupervisor.java | 10 ++--- .../SeekableStreamSupervisorStateManager.java | 38 ++++++++++++------- ...kableStreamSupervisorStateManagerTest.java | 30 +++++++-------- 3 files changed, 45 insertions(+), 33 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 2f890e937829..75a61c736878 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1034,9 +1034,9 @@ public void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { possiblyRegisterListener(); - stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); updatePartitionDataFromStream(); - stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); discoverTasks(); updateTaskStatus(); checkTaskDuration(); @@ -1046,13 +1046,13 @@ public void runInternal() // if suspended, ensure tasks have been requested to gracefully stop if (!spec.isSuspended()) { log.info("[%s] supervisor is running.", dataSource); - stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); createNewTasks(); - stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); } else { log.info("[%s] supervisor is suspended.", dataSource); gracefulShutdownInternal(); - stateManager.setState(SeekableStreamSupervisorStateManager.State.SUSPENDED); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.SUSPENDED); } if (log.isDebugEnabled()) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java index 20329a329583..56f70947965d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java @@ -67,6 +67,19 @@ public enum State this.priority = priority; } + // We only want to set these if the supervisor hasn't had a successful iteration yet + public boolean isOnlySetWhenNoSuccessfulRunYet() + { + Set firstRunStates = ImmutableSet.of( + WAITING_TO_RUN, + CONNECTING_TO_STREAM, + DISCOVERING_INITIAL_TASKS, + CREATING_TASKS, + RUNNING + ); + return firstRunStates.contains(this); + } + public boolean isHealthy() { Set unhealthyStates = ImmutableSet.of( @@ -115,21 +128,20 @@ public SeekableStreamSupervisorStateManager( this.stateHistory = new CircularBuffer<>(Math.max(healthinessThreshold, unhealthinessThreshold)); } - public Optional setStateIfNoSuccessfulRunYet(State newState) - { - if (!atLeastOneSuccessfulRun) { - return Optional.of(setState(newState)); - } - return Optional.absent(); - } - - public State setState(State newState) + public Optional setStateAndCheckIfFirstRun(State newState) { - if (newState.equals(State.SUSPENDED)) { + if (newState.isOnlySetWhenNoSuccessfulRunYet()) { + if (!atLeastOneSuccessfulRun) { + supervisorState = newState; + return Optional.of(newState); + } else { + return Optional.absent(); + } + } else if (newState.equals(State.SUSPENDED)) { atLeastOneSuccessfulRun = false; // We want the startup states again after being suspended } this.supervisorState = newState; - return newState; + return Optional.of(newState); } public void storeThrowableEvent(Throwable t) @@ -181,7 +193,7 @@ public void markRunFinishedAndEvaluateHealth() } else { currentRunState = State.UNHEALTHY_TASKS; } - } else { + } else if (supervisorState != State.UNHEALTHY_SUPERVISOR) { boolean tasksUnhealthy = completedTaskHistory.size() >= unhealthinessTaskThreshold; for (int i = 0; i < Math.min(unhealthinessTaskThreshold, completedTaskHistory.size()); i++) { // Last unhealthinessTaskThreshold tasks must be unhealthy for state to change to @@ -211,7 +223,7 @@ public void markRunFinishedAndEvaluateHealth() } } - setState(currentRunState); + this.supervisorState = currentRunState; // Reset manager state for next run currentRunSuccessful = true; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java index bb88d8cd162b..08ab0eb9743f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java @@ -51,29 +51,29 @@ public void setupTest() @Test public void testHappyPath() { - stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); Assert.assertEquals( SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM, stateManager.getSupervisorState() ); - stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); Assert.assertEquals( SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS, stateManager.getSupervisorState() ); - stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.CREATING_TASKS, stateManager.getSupervisorState()); - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.markRunFinishedAndEvaluateHealth(); - stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.setStateIfNoSuccessfulRunYet(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.markRunFinishedAndEvaluateHealth(); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); @@ -82,7 +82,7 @@ public void testHappyPath() @Test public void testTransientStreamFailure() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); stateManager.markRunFinishedAndEvaluateHealth(); // clean run without errors for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); @@ -98,7 +98,7 @@ public void testTransientStreamFailure() @Test public void testNonTransientStreamFailure() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.storeThrowableEvent(new NonTransientStreamException(new Exception("DOH!"))); @@ -113,7 +113,7 @@ public void testNonTransientStreamFailure() @Test public void testPossiblyTransientStreamFailure() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.storeThrowableEvent(new PossiblyTransientStreamException(new Exception("DOH!"))); @@ -128,7 +128,7 @@ public void testPossiblyTransientStreamFailure() @Test public void testNonTransientUnhealthiness() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.storeThrowableEvent(new NullPointerException("oof")); @@ -143,7 +143,7 @@ public void testNonTransientUnhealthiness() @Test public void testTransientUnhealthiness() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold() - 1; i++) { stateManager.storeThrowableEvent(new NullPointerException("oof")); stateManager.markRunFinishedAndEvaluateHealth(); @@ -180,7 +180,7 @@ public void testNonTransientTaskUnhealthiness() public void testTransientTaskUnhealthiness() { // Only half are failing - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getTaskUnhealthinessThreshold() + 3; i++) { Assert.assertNotEquals( stateManager.getSupervisorState(), @@ -260,7 +260,7 @@ public void testTaskRecoveryWithHealthinessThreshold() @Test public void testTwoUnhealthyStates() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < Math.max( config.getTaskUnhealthinessThreshold(), config.getUnhealthinessThreshold() From 46e200c1d00e1bd276702bb15b7f80b7c5b118b3 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 23 Apr 2019 12:11:45 -0700 Subject: [PATCH 08/22] Add transience field --- .../supervisor/SeekableStreamSupervisor.java | 10 +- .../SeekableStreamSupervisorStateManager.java | 82 ++++++++++++----- ...kableStreamSupervisorStateManagerTest.java | 91 +++++++++++++++---- 3 files changed, 141 insertions(+), 42 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 75a61c736878..04f82d425646 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1034,9 +1034,9 @@ public void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { possiblyRegisterListener(); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + stateManager.setState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); updatePartitionDataFromStream(); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + stateManager.setState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); discoverTasks(); updateTaskStatus(); checkTaskDuration(); @@ -1046,13 +1046,13 @@ public void runInternal() // if suspended, ensure tasks have been requested to gracefully stop if (!spec.isSuspended()) { log.info("[%s] supervisor is running.", dataSource); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + stateManager.setState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); createNewTasks(); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); } else { log.info("[%s] supervisor is suspended.", dataSource); gracefulShutdownInternal(); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.SUSPENDED); + stateManager.setState(SeekableStreamSupervisorStateManager.State.SUSPENDED); } if (log.isDebugEnabled()) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java index 56f70947965d..2a309f33c018 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java @@ -92,6 +92,14 @@ public boolean isHealthy() } } + public enum StreamErrorTransience + { + TRANSIENT, + POSSIBLY_TRANSIENT, + NON_TRANSIENT, + NON_STREAM_ERROR + } + private State supervisorState; private final int healthinessThreshold; private final int unhealthinessThreshold; @@ -101,7 +109,7 @@ public boolean isHealthy() private boolean atLeastOneSuccessfulRun = false; private boolean currentRunSuccessful = true; private final CircularBuffer completedTaskHistory; - private final CircularBuffer stateHistory; // From previous runs + private final CircularBuffer supervisorStateHistory; // From previous runs private final ExceptionEventStore eventStore; public SeekableStreamSupervisorStateManager( @@ -125,10 +133,15 @@ public SeekableStreamSupervisorStateManager( supervisorConfig.isStoringStackTraces() ); this.completedTaskHistory = new CircularBuffer<>(Math.max(healthinessTaskThreshold, unhealthinessTaskThreshold)); - this.stateHistory = new CircularBuffer<>(Math.max(healthinessThreshold, unhealthinessThreshold)); + this.supervisorStateHistory = new CircularBuffer<>(Math.max(healthinessThreshold, unhealthinessThreshold)); } - public Optional setStateAndCheckIfFirstRun(State newState) + /** + * Certain supervisor states can only be set if the supervisor hasn't had a successful iteration yet. This function + * checks if there's been at least one successful iteration if needed and sets supervisor state to an appropriate + * new state. + */ + public Optional setState(State newState) { if (newState.isOnlySetWhenNoSuccessfulRunYet()) { if (!atLeastOneSuccessfulRun) { @@ -169,11 +182,14 @@ public void markRunFinishedAndEvaluateHealth() for (Map.Entry> events : eventStore.getNonTransientRecentEvents().entrySet()) { if (events.getValue().size() >= unhealthinessThreshold) { - if (events.getKey().equals(NonTransientStreamException.class)) { - currentRunState = getHigherPriorityState(currentRunState, State.UNABLE_TO_CONNECT_TO_STREAM); - } else if (events.getKey().equals(TransientStreamException.class) || - events.getKey().equals(PossiblyTransientStreamException.class)) { - currentRunState = getHigherPriorityState(currentRunState, State.LOST_CONTACT_WITH_STREAM); + if (events.getKey().equals(NonTransientStreamException.class) || + events.getKey().equals(TransientStreamException.class) || + events.getKey().equals(PossiblyTransientStreamException.class)) { + if (atLeastOneSuccessfulRun) { + currentRunState = getHigherPriorityState(currentRunState, State.LOST_CONTACT_WITH_STREAM); + } else { + currentRunState = getHigherPriorityState(currentRunState, State.UNABLE_TO_CONNECT_TO_STREAM); + } } else { currentRunState = getHigherPriorityState(currentRunState, State.UNHEALTHY_SUPERVISOR); } @@ -208,13 +224,13 @@ public void markRunFinishedAndEvaluateHealth() } } - stateHistory.add(currentRunState); + supervisorStateHistory.add(currentRunState); if (currentRunState.isHealthy() && supervisorState == State.UNHEALTHY_SUPERVISOR) { currentRunState = State.UNHEALTHY_SUPERVISOR; - boolean supervisorHealthy = stateHistory.size() >= healthinessThreshold; - for (int i = 0; i < Math.min(healthinessThreshold, stateHistory.size()); i++) { - if (!stateHistory.getLatest(i).isHealthy()) { + boolean supervisorHealthy = supervisorStateHistory.size() >= healthinessThreshold; + for (int i = 0; i < Math.min(healthinessThreshold, supervisorStateHistory.size()); i++) { + if (!supervisorStateHistory.getLatest(i).isHealthy()) { supervisorHealthy = false; } } @@ -245,41 +261,54 @@ private State getHigherPriorityState(State s1, State s2) return s1.priority < s2.priority ? s1 : s2; } - @JsonPropertyOrder({"timestamp", "exceptionClass", "message", "stackTrace"}) + @JsonPropertyOrder({"timestamp", "exceptionClass", "streamErrorTransience", "message"}) public static class ExceptionEvent { - private Class clazz; + private Class exceptionClass; // Contains full stackTrace if storingStackTraces is true private String errorMessage; private DateTime timestamp; + private StreamErrorTransience streamErrorTransience; + + public ExceptionEvent() + { + } public ExceptionEvent( Throwable t, - boolean storingStackTraces + boolean storingStackTraces, + StreamErrorTransience streamErrorTransience ) { - this.clazz = t.getClass(); + this.exceptionClass = t.getClass(); this.errorMessage = storingStackTraces ? ExceptionUtils.getStackTrace(t) : t.getMessage(); this.timestamp = DateTimes.nowUtc(); + this.streamErrorTransience = streamErrorTransience; } - @JsonProperty + @JsonProperty("exceptionClass") public Class getExceptionClass() { - return clazz; + return exceptionClass; } - @JsonProperty + @JsonProperty("message") public String getErrorMessage() { return errorMessage; } - @JsonProperty + @JsonProperty("timestamp") public DateTime getTimestamp() { return timestamp; } + + @JsonProperty("errorTransience") + public StreamErrorTransience getStreamErrorTransience() + { + return streamErrorTransience; + } } private static class ExceptionEventStore @@ -306,7 +335,18 @@ public void storeThrowable(Throwable t) new ConcurrentLinkedQueue<>() ); - ExceptionEvent eventToAdd = new ExceptionEvent(t, storeStackTraces); + StreamErrorTransience transience; + if (t instanceof PossiblyTransientStreamException) { + transience = StreamErrorTransience.POSSIBLY_TRANSIENT; + } else if (t instanceof TransientStreamException) { + transience = StreamErrorTransience.TRANSIENT; + } else if (t instanceof NonTransientStreamException) { + transience = StreamErrorTransience.NON_TRANSIENT; + } else { + transience = StreamErrorTransience.NON_STREAM_ERROR; + } + + ExceptionEvent eventToAdd = new ExceptionEvent(t, storeStackTraces, transience); recentEventsQueue.add(eventToAdd); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java index 08ab0eb9743f..184854bdc22c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java @@ -19,16 +19,19 @@ package org.apache.druid.indexing.seekablestream.supervisor; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.exceptions.NonTransientStreamException; import org.apache.druid.indexing.seekablestream.exceptions.PossiblyTransientStreamException; import org.apache.druid.indexing.seekablestream.exceptions.TransientStreamException; +import org.apache.druid.jackson.DefaultObjectMapper; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.io.IOException; import java.util.List; import java.util.Queue; @@ -36,6 +39,7 @@ public class SeekableStreamSupervisorStateManagerTest { private SeekableStreamSupervisorStateManager stateManager; private SeekableStreamSupervisorConfig config; + private ObjectMapper defaultMapper; @Before public void setupTest() @@ -46,34 +50,35 @@ public void setupTest() SeekableStreamSupervisorStateManager.State.WAITING_TO_RUN, config ); + defaultMapper = new DefaultObjectMapper(); } @Test public void testHappyPath() { - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + stateManager.setState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); Assert.assertEquals( SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM, stateManager.getSupervisorState() ); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + stateManager.setState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); Assert.assertEquals( SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS, stateManager.getSupervisorState() ); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + stateManager.setState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.CREATING_TASKS, stateManager.getSupervisorState()); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.markRunFinishedAndEvaluateHealth(); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + stateManager.setState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + stateManager.setState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + stateManager.setState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.markRunFinishedAndEvaluateHealth(); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); @@ -82,7 +87,7 @@ public void testHappyPath() @Test public void testTransientStreamFailure() { - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); stateManager.markRunFinishedAndEvaluateHealth(); // clean run without errors for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); @@ -93,12 +98,18 @@ public void testTransientStreamFailure() SeekableStreamSupervisorStateManager.State.LOST_CONTACT_WITH_STREAM, stateManager.getSupervisorState() ); + Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); + for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { + SeekableStreamSupervisorStateManager.ExceptionEvent event = stateManager.getExceptionEvents().poll(); + Assert.assertEquals(SeekableStreamSupervisorStateManager.StreamErrorTransience.TRANSIENT, event.getStreamErrorTransience()); + Assert.assertEquals(TransientStreamException.class, event.getExceptionClass()); + } } @Test public void testNonTransientStreamFailure() { - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.storeThrowableEvent(new NonTransientStreamException(new Exception("DOH!"))); @@ -108,27 +119,39 @@ public void testNonTransientStreamFailure() SeekableStreamSupervisorStateManager.State.UNABLE_TO_CONNECT_TO_STREAM, stateManager.getSupervisorState() ); + Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); + for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { + SeekableStreamSupervisorStateManager.ExceptionEvent event = stateManager.getExceptionEvents().poll(); + Assert.assertEquals(SeekableStreamSupervisorStateManager.StreamErrorTransience.NON_TRANSIENT, event.getStreamErrorTransience()); + Assert.assertEquals(NonTransientStreamException.class, event.getExceptionClass()); + } } @Test public void testPossiblyTransientStreamFailure() { - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.storeThrowableEvent(new PossiblyTransientStreamException(new Exception("DOH!"))); stateManager.markRunFinishedAndEvaluateHealth(); } Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.LOST_CONTACT_WITH_STREAM, + SeekableStreamSupervisorStateManager.State.UNABLE_TO_CONNECT_TO_STREAM, stateManager.getSupervisorState() ); + Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); + for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { + SeekableStreamSupervisorStateManager.ExceptionEvent event = stateManager.getExceptionEvents().poll(); + Assert.assertEquals(SeekableStreamSupervisorStateManager.StreamErrorTransience.POSSIBLY_TRANSIENT, event.getStreamErrorTransience()); + Assert.assertEquals(PossiblyTransientStreamException.class, event.getExceptionClass()); + } } @Test public void testNonTransientUnhealthiness() { - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.storeThrowableEvent(new NullPointerException("oof")); @@ -138,12 +161,18 @@ public void testNonTransientUnhealthiness() SeekableStreamSupervisorStateManager.State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState() ); + Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); + for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { + SeekableStreamSupervisorStateManager.ExceptionEvent event = stateManager.getExceptionEvents().poll(); + Assert.assertEquals(SeekableStreamSupervisorStateManager.StreamErrorTransience.NON_STREAM_ERROR, event.getStreamErrorTransience()); + Assert.assertEquals(NullPointerException.class, event.getExceptionClass()); + } } @Test public void testTransientUnhealthiness() { - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold() - 1; i++) { stateManager.storeThrowableEvent(new NullPointerException("oof")); stateManager.markRunFinishedAndEvaluateHealth(); @@ -157,6 +186,7 @@ public void testTransientUnhealthiness() SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState() ); + Assert.assertEquals(config.getUnhealthinessThreshold() - 1, stateManager.getExceptionEvents().size()); } @Test @@ -174,13 +204,14 @@ public void testNonTransientTaskUnhealthiness() SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS, stateManager.getSupervisorState() ); + Assert.assertEquals(0, stateManager.getExceptionEvents().size()); } @Test public void testTransientTaskUnhealthiness() { // Only half are failing - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getTaskUnhealthinessThreshold() + 3; i++) { Assert.assertNotEquals( stateManager.getSupervisorState(), @@ -194,6 +225,7 @@ public void testTransientTaskUnhealthiness() SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState() ); + Assert.assertEquals(0, stateManager.getExceptionEvents().size()); } @Test @@ -224,6 +256,15 @@ public void testSupervisorRecoveryWithHealthinessThreshold() SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState() ); + Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); + for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { + SeekableStreamSupervisorStateManager.ExceptionEvent event = stateManager.getExceptionEvents().poll(); + Assert.assertEquals( + SeekableStreamSupervisorStateManager.StreamErrorTransience.NON_STREAM_ERROR, + event.getStreamErrorTransience() + ); + Assert.assertEquals(Exception.class, event.getExceptionClass()); + } } @Test @@ -260,7 +301,7 @@ public void testTaskRecoveryWithHealthinessThreshold() @Test public void testTwoUnhealthyStates() { - stateManager.setStateAndCheckIfFirstRun(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < Math.max( config.getTaskUnhealthinessThreshold(), config.getUnhealthinessThreshold() @@ -304,4 +345,22 @@ public void testGetThrowableEvents() Assert.assertNotNull(events.peek().getErrorMessage()); Assert.assertEquals(TransientStreamException.class, events.poll().getExceptionClass()); } + + @Test + public void testExceptionEventSerDe() throws IOException + { + SeekableStreamSupervisorStateManager.ExceptionEvent event = + new SeekableStreamSupervisorStateManager.ExceptionEvent( + new NullPointerException("msg"), + true, + SeekableStreamSupervisorStateManager.StreamErrorTransience.TRANSIENT + ); + String serialized = defaultMapper.writeValueAsString(event); + SeekableStreamSupervisorStateManager.ExceptionEvent deserialized = + defaultMapper.readValue(serialized, SeekableStreamSupervisorStateManager.ExceptionEvent.class); + Assert.assertEquals(event.getErrorMessage(), deserialized.getErrorMessage()); + Assert.assertEquals(event.getExceptionClass(), deserialized.getExceptionClass()); + Assert.assertEquals(event.getTimestamp(), deserialized.getTimestamp()); + Assert.assertEquals(event.getStreamErrorTransience(), deserialized.getStreamErrorTransience()); + } } From 68dfcdebfe2a8e48e6b732ca9759a27c18b7a489 Mon Sep 17 00:00:00 2001 From: dclim Date: Thu, 2 May 2019 17:17:48 -0600 Subject: [PATCH 09/22] write test --- .../src/test/java/org/apache/druid/utils/CircularBufferTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java b/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java index 5e611708aa49..9a2fbed77868 100644 --- a/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java +++ b/core/src/test/java/org/apache/druid/utils/CircularBufferTest.java @@ -22,6 +22,7 @@ import org.junit.Assert; import org.junit.Test; + public class CircularBufferTest { @Test From d98623213d0c57c6626888549002e048d7975908 Mon Sep 17 00:00:00 2001 From: dclim Date: Fri, 3 May 2019 00:55:12 -0600 Subject: [PATCH 10/22] implement code review changes except for reconsidering logic of markRunFinishedAndEvaluateHealth() --- .../indexing/kafka/KafkaRecordSupplier.java | 96 ++++++----- .../KafkaSupervisorReportPayload.java | 5 +- .../kinesis/KinesisRecordSupplier.java | 81 +++++---- .../KinesisSupervisorReportPayload.java | 5 +- .../SeekableStreamSupervisorConfig.java | 8 +- .../supervisor/SeekableStreamSupervisor.java | 10 +- ...SeekableStreamSupervisorReportPayload.java | 7 +- .../SeekableStreamSupervisorStateManager.java | 156 +++++++----------- ...kableStreamSupervisorStateManagerTest.java | 132 ++++++++------- .../StreamAppenderatorDriver.java | 2 +- 10 files changed, 243 insertions(+), 259 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index b7522f6061a3..640251d4a544 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import org.apache.curator.shaded.com.google.common.base.Throwables; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; @@ -48,6 +49,7 @@ import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.concurrent.Callable; import java.util.stream.Collectors; public class KafkaRecordSupplier implements RecordSupplier @@ -70,44 +72,46 @@ public KafkaRecordSupplier( @Override public void assign(Set> streamPartitions) { - consumer.assign(streamPartitions - .stream() - .map(x -> new TopicPartition(x.getStream(), x.getPartitionId())) - .collect(Collectors.toSet())); + wrapExceptions(() -> consumer.assign(streamPartitions + .stream() + .map(x -> new TopicPartition(x.getStream(), x.getPartitionId())) + .collect(Collectors.toSet()))); } @Override public void seek(StreamPartition partition, Long sequenceNumber) { - consumer.seek(new TopicPartition(partition.getStream(), partition.getPartitionId()), sequenceNumber); + wrapExceptions(() -> consumer.seek( + new TopicPartition(partition.getStream(), partition.getPartitionId()), + sequenceNumber + )); } @Override public void seekToEarliest(Set> partitions) { - consumer.seekToBeginning(partitions - .stream() - .map(e -> new TopicPartition(e.getStream(), e.getPartitionId())) - .collect(Collectors.toList())); + wrapExceptions(() -> consumer.seekToBeginning(partitions + .stream() + .map(e -> new TopicPartition(e.getStream(), e.getPartitionId())) + .collect(Collectors.toList()))); } @Override public void seekToLatest(Set> partitions) { - consumer.seekToEnd(partitions - .stream() - .map(e -> new TopicPartition(e.getStream(), e.getPartitionId())) - .collect(Collectors.toList())); + wrapExceptions(() -> consumer.seekToEnd(partitions + .stream() + .map(e -> new TopicPartition(e.getStream(), e.getPartitionId())) + .collect(Collectors.toList()))); } @Override public Set> getAssignment() { - Set topicPartitions = consumer.assignment(); - return topicPartitions - .stream() - .map(e -> new StreamPartition<>(e.topic(), e.partition())) - .collect(Collectors.toSet()); + return wrapExceptions(() -> consumer.assignment() + .stream() + .map(e -> new StreamPartition<>(e.topic(), e.partition())) + .collect(Collectors.toSet())); } @Nonnull @@ -149,39 +153,22 @@ public Long getEarliestSequenceNumber(StreamPartition partition) @Override public Long getPosition(StreamPartition partition) { - try { - return consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); - } - catch (TimeoutException e) { - throw new PossiblyTransientStreamException(e); - } - catch (WakeupException | InterruptException e) { - throw new TransientStreamException(e); - } - catch (KafkaException e) { - throw new NonTransientStreamException(e); - } + return wrapExceptions(() -> consumer.position(new TopicPartition( + partition.getStream(), + partition.getPartitionId() + ))); } @Override public Set getPartitionIds(String stream) { - try { + return wrapExceptions(() -> { List partitions = consumer.partitionsFor(stream); if (partitions == null) { throw new ISE("Topic [%s] is not found in KafkaConsumer's list of topics", stream); } return partitions.stream().map(PartitionInfo::partition).collect(Collectors.toSet()); - } - catch (TimeoutException e) { - throw new PossiblyTransientStreamException(e); - } - catch (WakeupException | InterruptException e) { - throw new TransientStreamException(e); - } - catch (KafkaException e) { - throw new NonTransientStreamException(e); - } + }); } @Override @@ -234,4 +221,31 @@ private KafkaConsumer getKafkaConsumer() } } + private static T wrapExceptions(Callable callable) + { + try { + return callable.call(); + } + catch (TimeoutException e) { + throw new PossiblyTransientStreamException(e); + } + catch (WakeupException | InterruptException e) { + throw new TransientStreamException(e); + } + catch (KafkaException e) { + throw new NonTransientStreamException(e); + } + catch (Exception e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } + } + + private static void wrapExceptions(Runnable runnable) + { + wrapExceptions(() -> { + runnable.run(); + return null; + }); + } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index c3d6e4807e8e..d3c33fc6ba12 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -24,8 +24,8 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; +import java.util.List; import java.util.Map; -import java.util.Queue; public class KafkaSupervisorReportPayload extends SeekableStreamSupervisorReportPayload { @@ -41,7 +41,7 @@ public KafkaSupervisorReportPayload( @Nullable DateTime offsetsLastUpdated, boolean suspended, SeekableStreamSupervisorStateManager.State state, - Queue recentErrors + List recentErrors ) { super( @@ -76,6 +76,7 @@ public String toString() (getAggregateLag() != null ? ", aggregateLag=" + getAggregateLag() : "") + (getOffsetsLastUpdated() != null ? ", sequenceLastUpdated=" + getOffsetsLastUpdated() : "") + ", suspended=" + getSuspended() + + ", recentErrors=" + getRecentErrors() + '}'; } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index f6d85f54bb99..d072c3c38c3c 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -44,6 +44,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Queues; +import org.apache.curator.shaded.com.google.common.base.Throwables; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.aws.AWSCredentialsUtils; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; @@ -73,6 +74,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executors; @@ -585,23 +587,11 @@ public String getPosition(StreamPartition partition) @Override public Set getPartitionIds(String stream) { - try { - checkIfClosed(); - return kinesis.describeStream(stream) - .getStreamDescription() - .getShards() - .stream() - .map(Shard::getShardId).collect(Collectors.toSet()); - } - catch (LimitExceededException | ProvisionedThroughputExceededException | ResourceInUseException e) { - throw new TransientStreamException(e); - } - catch (ResourceNotFoundException e) { - throw new PossiblyTransientStreamException(e); - } - catch (AmazonKinesisException e) { - throw new NonTransientStreamException(e); - } + return wrapExceptions(() -> kinesis.describeStream(stream) + .getStreamDescription() + .getShards() + .stream() + .map(Shard::getShardId).collect(Collectors.toSet())); } @Override @@ -641,12 +631,12 @@ private void seekInternal(StreamPartition partition, String sequenceNumb sequenceNumber != null ? sequenceNumber : iteratorEnum.toString() ); - resource.shardIterator = kinesis.getShardIterator( + resource.shardIterator = wrapExceptions(() -> kinesis.getShardIterator( partition.getStream(), partition.getPartitionId(), iteratorEnum.toString(), sequenceNumber - ).getShardIterator(); + ).getShardIterator()); checkPartitionsStarted = true; } @@ -672,10 +662,10 @@ private void filterBufferAndResetFetchRunnable(Set> part // filter records in buffer and only retain ones whose partition was not seeked BlockingQueue> newQ = new LinkedBlockingQueue<>(recordBufferSize); - records - .stream() - .filter(x -> !partitions.contains(x.getStreamPartition())) - .forEachOrdered(newQ::offer); + + records.stream() + .filter(x -> !partitions.contains(x.getStreamPartition())) + .forEachOrdered(newQ::offer); records = newQ; @@ -687,26 +677,11 @@ private void filterBufferAndResetFetchRunnable(Set> part @Nullable private String getSequenceNumberInternal(StreamPartition partition, ShardIteratorType iteratorEnum) { - - String shardIterator; - try { - shardIterator = kinesis.getShardIterator( - partition.getStream(), - partition.getPartitionId(), - iteratorEnum.toString() - ).getShardIterator(); - } - catch (LimitExceededException | ProvisionedThroughputExceededException | ResourceInUseException e) { - throw new TransientStreamException(e); - } - catch (ResourceNotFoundException e) { - throw new PossiblyTransientStreamException(e); - } - catch (AmazonKinesisException e) { - throw new NonTransientStreamException(e); - } - - return getSequenceNumberInternal(partition, shardIterator); + return wrapExceptions(() -> getSequenceNumberInternal( + partition, + kinesis.getShardIterator(partition.getStream(), partition.getPartitionId(), iteratorEnum.toString()) + .getShardIterator() + )); } @Nullable @@ -797,6 +772,26 @@ private static byte[] toByteArray(final ByteBuffer buffer) } } + private static T wrapExceptions(Callable callable) + { + try { + return callable.call(); + } + catch (LimitExceededException | ProvisionedThroughputExceededException | ResourceInUseException e) { + throw new TransientStreamException(e); + } + catch (ResourceNotFoundException e) { + throw new PossiblyTransientStreamException(e); + } + catch (AmazonKinesisException e) { + throw new NonTransientStreamException(e); + } + catch (Exception e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } + } + @VisibleForTesting public int bufferSize() { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java index 09ace4698092..c8ca0e9fb713 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java @@ -23,7 +23,7 @@ import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import java.util.Collections; -import java.util.Queue; +import java.util.List; public class KinesisSupervisorReportPayload extends SeekableStreamSupervisorReportPayload { @@ -35,7 +35,7 @@ public KinesisSupervisorReportPayload( Long durationSeconds, boolean suspended, SeekableStreamSupervisorStateManager.State state, - Queue recentErrors + List recentErrors ) { super( @@ -66,6 +66,7 @@ public String toString() ", active=" + getActiveTasks() + ", publishing=" + getPublishingTasks() + ", suspended=" + getSuspended() + + ", recentErrors=" + getRecentErrors() + '}'; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorConfig.java index f5041d1ecafc..67207d24f5a4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorConfig.java @@ -22,8 +22,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; -import javax.validation.constraints.Min; - public class SeekableStreamSupervisorConfig { @JsonProperty @@ -31,22 +29,18 @@ public class SeekableStreamSupervisorConfig // The number of runs failed before the supervisor flips from a RUNNING to an UNHEALTHY state @JsonProperty - @Min(3) private int unhealthinessThreshold = 3; - // The number of successful before the supervisor flips from an UNHEALTHY to a RUNNING state + // The number of successful runs before the supervisor flips from an UNHEALTHY to a RUNNING state @JsonProperty - @Min(3) private int healthinessThreshold = 3; // The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state @JsonProperty - @Min(3) private int taskUnhealthinessThreshold = 3; // The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state @JsonProperty - @Min(3) private int taskHealthinessThreshold = 3; // The maximum number of exception events that can be returned through the supervisor status endpoint diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 04f82d425646..c3425d30f9d6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1034,9 +1034,9 @@ public void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { possiblyRegisterListener(); - stateManager.setState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); updatePartitionDataFromStream(); - stateManager.setState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); discoverTasks(); updateTaskStatus(); checkTaskDuration(); @@ -1046,13 +1046,13 @@ public void runInternal() // if suspended, ensure tasks have been requested to gracefully stop if (!spec.isSuspended()) { log.info("[%s] supervisor is running.", dataSource); - stateManager.setState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); createNewTasks(); - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); } else { log.info("[%s] supervisor is suspended.", dataSource); gracefulShutdownInternal(); - stateManager.setState(SeekableStreamSupervisorStateManager.State.SUSPENDED); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.SUSPENDED); } if (log.isDebugEnabled()) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index 6f753ca18788..f6c949389f63 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -28,7 +28,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.Queue; @JsonInclude(JsonInclude.Include.NON_NULL) public abstract class SeekableStreamSupervisorReportPayload @@ -46,7 +45,7 @@ public abstract class SeekableStreamSupervisorReportPayload recentErrors; + private final List recentErrors; public SeekableStreamSupervisorReportPayload( String dataSource, @@ -60,7 +59,7 @@ public SeekableStreamSupervisorReportPayload( @Nullable DateTime offsetsLastUpdated, boolean suspended, SeekableStreamSupervisorStateManager.State state, - Queue recentErrors + List recentErrors ) { this.dataSource = dataSource; @@ -169,7 +168,7 @@ public SeekableStreamSupervisorStateManager.State getState() } @JsonProperty - public Queue getRecentErrors() + public List getRecentErrors() { return recentErrors; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java index 2a309f33c018..63c2ccccede0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java @@ -21,9 +21,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyOrder; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableSet; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.exceptions.NonTransientStreamException; @@ -34,61 +32,56 @@ import org.codehaus.plexus.util.ExceptionUtils; import org.joda.time.DateTime; +import java.util.ArrayList; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Queue; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.stream.Collectors; public class SeekableStreamSupervisorStateManager { public enum State { - // Error states are ordered from high to low priority - UNHEALTHY_SUPERVISOR(1), - UNHEALTHY_TASKS(2), - UNABLE_TO_CONNECT_TO_STREAM(3), - LOST_CONTACT_WITH_STREAM(4), - // Non-error states are equal priority - WAITING_TO_RUN(5), - CONNECTING_TO_STREAM(5), - DISCOVERING_INITIAL_TASKS(5), - CREATING_TASKS(5), - RUNNING(5), - SUSPENDED(5), - SHUTTING_DOWN(5); + // Error states - ordered from high to low priority + UNHEALTHY_SUPERVISOR(1, false, false), + UNHEALTHY_TASKS(2, false, false), + UNABLE_TO_CONNECT_TO_STREAM(3, false, false), + LOST_CONTACT_WITH_STREAM(4, false, false), + + // Non-error states - equal priority + WAITING_TO_RUN(5, true, true), + CONNECTING_TO_STREAM(5, true, true), + DISCOVERING_INITIAL_TASKS(5, true, true), + CREATING_TASKS(5, true, true), + RUNNING(5, true, false), + SUSPENDED(5, true, false), + SHUTTING_DOWN(5, true, false); // Lower priority number means higher priority and vice versa private final int priority; + private final boolean healthy; + private final boolean firstRunOnly; - State(int priority) + State(int priority, boolean healthy, boolean firstRunOnly) { this.priority = priority; + this.healthy = healthy; + this.firstRunOnly = firstRunOnly; } - // We only want to set these if the supervisor hasn't had a successful iteration yet - public boolean isOnlySetWhenNoSuccessfulRunYet() + // We only want to set these only if the supervisor hasn't had a successful iteration yet + public boolean isFirstRunOnly() { - Set firstRunStates = ImmutableSet.of( - WAITING_TO_RUN, - CONNECTING_TO_STREAM, - DISCOVERING_INITIAL_TASKS, - CREATING_TASKS, - RUNNING - ); - return firstRunStates.contains(this); + return firstRunOnly; } public boolean isHealthy() { - Set unhealthyStates = ImmutableSet.of( - UNHEALTHY_SUPERVISOR, - UNHEALTHY_TASKS, - UNABLE_TO_CONNECT_TO_STREAM, - LOST_CONTACT_WITH_STREAM - ); - return !unhealthyStates.contains(this); + return healthy; } } @@ -122,12 +115,12 @@ public SeekableStreamSupervisorStateManager( this.unhealthinessThreshold = supervisorConfig.getUnhealthinessThreshold(); this.healthinessTaskThreshold = supervisorConfig.getTaskHealthinessThreshold(); this.unhealthinessTaskThreshold = supervisorConfig.getTaskUnhealthinessThreshold(); - Preconditions.checkArgument( - supervisorConfig.getMaxStoredExceptionEvents() >= - Math.max(healthinessThreshold, unhealthinessThreshold), - "numExceptionEventsToStore must be greater than or equal to both " - + "healthinessThreshold and unhealthinessThreshold" - ); + + Preconditions.checkArgument(supervisorConfig.getMaxStoredExceptionEvents() >= Math.max( + healthinessThreshold, + unhealthinessThreshold + ), "maxStoredExceptionEvents must be >= to max(healthinessThreshold, unhealthinessThreshold)"); + this.eventStore = new ExceptionEventStore( supervisorConfig.getMaxStoredExceptionEvents(), supervisorConfig.isStoringStackTraces() @@ -137,33 +130,27 @@ public SeekableStreamSupervisorStateManager( } /** - * Certain supervisor states can only be set if the supervisor hasn't had a successful iteration yet. This function - * checks if there's been at least one successful iteration if needed and sets supervisor state to an appropriate + * Certain supervisor states are only valid if the supervisor hasn't had a successful iteration yet. This function + * checks if there's been at least one successful iteration, and if applicable sets supervisor state to an appropriate * new state. */ - public Optional setState(State newState) + public void maybeSetState(State newState) { - if (newState.isOnlySetWhenNoSuccessfulRunYet()) { - if (!atLeastOneSuccessfulRun) { - supervisorState = newState; - return Optional.of(newState); - } else { - return Optional.absent(); - } - } else if (newState.equals(State.SUSPENDED)) { + if (!newState.isFirstRunOnly() || !atLeastOneSuccessfulRun) { + supervisorState = newState; + } + + if (State.SUSPENDED.equals(newState)) { atLeastOneSuccessfulRun = false; // We want the startup states again after being suspended } - this.supervisorState = newState; - return Optional.of(newState); } public void storeThrowableEvent(Throwable t) { - if (t instanceof PossiblyTransientStreamException && atLeastOneSuccessfulRun) { - t = new TransientStreamException(t); - } + eventStore.storeThrowable(t instanceof PossiblyTransientStreamException && atLeastOneSuccessfulRun + ? new TransientStreamException(t.getCause()) + : t); - eventStore.storeThrowable(t); currentRunSuccessful = false; } @@ -174,22 +161,19 @@ public void storeCompletedTaskState(TaskState state) public void markRunFinishedAndEvaluateHealth() { - if (currentRunSuccessful) { - atLeastOneSuccessfulRun = true; - } + atLeastOneSuccessfulRun |= currentRunSuccessful; State currentRunState = State.RUNNING; - for (Map.Entry> events : eventStore.getNonTransientRecentEvents().entrySet()) { + for (Map.Entry> events : eventStore.getRecentEventsMatchingExceptionsThrownOnCurrentRun().entrySet()) { if (events.getValue().size() >= unhealthinessThreshold) { if (events.getKey().equals(NonTransientStreamException.class) || events.getKey().equals(TransientStreamException.class) || events.getKey().equals(PossiblyTransientStreamException.class)) { - if (atLeastOneSuccessfulRun) { - currentRunState = getHigherPriorityState(currentRunState, State.LOST_CONTACT_WITH_STREAM); - } else { - currentRunState = getHigherPriorityState(currentRunState, State.UNABLE_TO_CONNECT_TO_STREAM); - } + + currentRunState = atLeastOneSuccessfulRun + ? getHigherPriorityState(currentRunState, State.LOST_CONTACT_WITH_STREAM) + : getHigherPriorityState(currentRunState, State.UNABLE_TO_CONNECT_TO_STREAM); } else { currentRunState = getHigherPriorityState(currentRunState, State.UNHEALTHY_SUPERVISOR); } @@ -227,15 +211,13 @@ public void markRunFinishedAndEvaluateHealth() supervisorStateHistory.add(currentRunState); if (currentRunState.isHealthy() && supervisorState == State.UNHEALTHY_SUPERVISOR) { - currentRunState = State.UNHEALTHY_SUPERVISOR; boolean supervisorHealthy = supervisorStateHistory.size() >= healthinessThreshold; for (int i = 0; i < Math.min(healthinessThreshold, supervisorStateHistory.size()); i++) { - if (!supervisorStateHistory.getLatest(i).isHealthy()) { - supervisorHealthy = false; - } + supervisorHealthy &= supervisorStateHistory.getLatest(i).isHealthy(); } - if (supervisorHealthy) { - currentRunState = State.RUNNING; + + if (!supervisorHealthy) { + currentRunState = State.UNHEALTHY_SUPERVISOR; } } @@ -246,7 +228,7 @@ public void markRunFinishedAndEvaluateHealth() eventStore.resetErrorsEncounteredOnRun(); } - public Queue getExceptionEvents() + public List getExceptionEvents() { return eventStore.getRecentEvents(); } @@ -270,10 +252,6 @@ public static class ExceptionEvent private DateTime timestamp; private StreamErrorTransience streamErrorTransience; - public ExceptionEvent() - { - } - public ExceptionEvent( Throwable t, boolean storingStackTraces, @@ -319,7 +297,7 @@ private static class ExceptionEventStore private final boolean storeStackTraces; private final Set errorsEncounteredOnRun; - public ExceptionEventStore(int numEventsToStore, boolean storeStackTraces) + private ExceptionEventStore(int numEventsToStore, boolean storeStackTraces) { this.recentEventsQueue = new ConcurrentLinkedQueue<>(); this.recentEventsMap = new ConcurrentHashMap<>(numEventsToStore); @@ -328,7 +306,7 @@ public ExceptionEventStore(int numEventsToStore, boolean storeStackTraces) this.errorsEncounteredOnRun = new HashSet<>(); } - public void storeThrowable(Throwable t) + private void storeThrowable(Throwable t) { Queue exceptionEventsForClassT = recentEventsMap.getOrDefault( t.getClass(), @@ -359,28 +337,22 @@ public void storeThrowable(Throwable t) recentEventsMap.put(t.getClass(), exceptionEventsForClassT); } - public void resetErrorsEncounteredOnRun() + private void resetErrorsEncounteredOnRun() { errorsEncounteredOnRun.clear(); } - public Queue getRecentEvents() + private List getRecentEvents() { - return recentEventsQueue; + return new ArrayList<>(recentEventsQueue); } - public ConcurrentHashMap> getNonTransientRecentEvents() + private Map> getRecentEventsMatchingExceptionsThrownOnCurrentRun() { - ConcurrentHashMap> nonTransientRecentEventsMap = - new ConcurrentHashMap<>(recentEventsMap); - - for (Class throwableClass : recentEventsMap.keySet()) { - if (!errorsEncounteredOnRun.contains(throwableClass)) { - nonTransientRecentEventsMap.remove(throwableClass); - } - } - - return nonTransientRecentEventsMap; + return recentEventsMap.entrySet() + .stream() + .filter(x -> errorsEncounteredOnRun.contains(x.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java index 184854bdc22c..aa329d6c836e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.seekablestream.supervisor; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import org.apache.druid.indexer.TaskState; @@ -26,14 +27,17 @@ import org.apache.druid.indexing.seekablestream.exceptions.NonTransientStreamException; import org.apache.druid.indexing.seekablestream.exceptions.PossiblyTransientStreamException; import org.apache.druid.indexing.seekablestream.exceptions.TransientStreamException; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.StreamErrorTransience; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Pair; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.io.IOException; +import java.util.Iterator; import java.util.List; -import java.util.Queue; +import java.util.Map; public class SeekableStreamSupervisorStateManagerTest { @@ -56,29 +60,29 @@ public void setupTest() @Test public void testHappyPath() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); Assert.assertEquals( SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM, stateManager.getSupervisorState() ); - stateManager.setState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); Assert.assertEquals( SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS, stateManager.getSupervisorState() ); - stateManager.setState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.CREATING_TASKS, stateManager.getSupervisorState()); - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.markRunFinishedAndEvaluateHealth(); - stateManager.setState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.setState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.setState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.markRunFinishedAndEvaluateHealth(); Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); @@ -87,7 +91,7 @@ public void testHappyPath() @Test public void testTransientStreamFailure() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); stateManager.markRunFinishedAndEvaluateHealth(); // clean run without errors for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); @@ -99,17 +103,17 @@ public void testTransientStreamFailure() stateManager.getSupervisorState() ); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); - for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - SeekableStreamSupervisorStateManager.ExceptionEvent event = stateManager.getExceptionEvents().poll(); - Assert.assertEquals(SeekableStreamSupervisorStateManager.StreamErrorTransience.TRANSIENT, event.getStreamErrorTransience()); - Assert.assertEquals(TransientStreamException.class, event.getExceptionClass()); - } + + stateManager.getExceptionEvents().forEach(x -> { + Assert.assertEquals(StreamErrorTransience.TRANSIENT, x.getStreamErrorTransience()); + Assert.assertEquals(TransientStreamException.class, x.getExceptionClass()); + }); } @Test public void testNonTransientStreamFailure() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.storeThrowableEvent(new NonTransientStreamException(new Exception("DOH!"))); @@ -120,17 +124,17 @@ public void testNonTransientStreamFailure() stateManager.getSupervisorState() ); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); - for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - SeekableStreamSupervisorStateManager.ExceptionEvent event = stateManager.getExceptionEvents().poll(); - Assert.assertEquals(SeekableStreamSupervisorStateManager.StreamErrorTransience.NON_TRANSIENT, event.getStreamErrorTransience()); - Assert.assertEquals(NonTransientStreamException.class, event.getExceptionClass()); - } + + stateManager.getExceptionEvents().forEach(x -> { + Assert.assertEquals(StreamErrorTransience.NON_TRANSIENT, x.getStreamErrorTransience()); + Assert.assertEquals(NonTransientStreamException.class, x.getExceptionClass()); + }); } @Test public void testPossiblyTransientStreamFailure() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.storeThrowableEvent(new PossiblyTransientStreamException(new Exception("DOH!"))); @@ -141,17 +145,17 @@ public void testPossiblyTransientStreamFailure() stateManager.getSupervisorState() ); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); - for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - SeekableStreamSupervisorStateManager.ExceptionEvent event = stateManager.getExceptionEvents().poll(); - Assert.assertEquals(SeekableStreamSupervisorStateManager.StreamErrorTransience.POSSIBLY_TRANSIENT, event.getStreamErrorTransience()); - Assert.assertEquals(PossiblyTransientStreamException.class, event.getExceptionClass()); - } + + stateManager.getExceptionEvents().forEach(x -> { + Assert.assertEquals(StreamErrorTransience.POSSIBLY_TRANSIENT, x.getStreamErrorTransience()); + Assert.assertEquals(PossiblyTransientStreamException.class, x.getExceptionClass()); + }); } @Test public void testNonTransientUnhealthiness() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); stateManager.storeThrowableEvent(new NullPointerException("oof")); @@ -162,17 +166,17 @@ public void testNonTransientUnhealthiness() stateManager.getSupervisorState() ); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); - for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - SeekableStreamSupervisorStateManager.ExceptionEvent event = stateManager.getExceptionEvents().poll(); - Assert.assertEquals(SeekableStreamSupervisorStateManager.StreamErrorTransience.NON_STREAM_ERROR, event.getStreamErrorTransience()); - Assert.assertEquals(NullPointerException.class, event.getExceptionClass()); - } + + stateManager.getExceptionEvents().forEach(x -> { + Assert.assertEquals(StreamErrorTransience.NON_STREAM_ERROR, x.getStreamErrorTransience()); + Assert.assertEquals(NullPointerException.class, x.getExceptionClass()); + }); } @Test public void testTransientUnhealthiness() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getUnhealthinessThreshold() - 1; i++) { stateManager.storeThrowableEvent(new NullPointerException("oof")); stateManager.markRunFinishedAndEvaluateHealth(); @@ -211,7 +215,7 @@ public void testNonTransientTaskUnhealthiness() public void testTransientTaskUnhealthiness() { // Only half are failing - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < config.getTaskUnhealthinessThreshold() + 3; i++) { Assert.assertNotEquals( stateManager.getSupervisorState(), @@ -257,14 +261,11 @@ public void testSupervisorRecoveryWithHealthinessThreshold() stateManager.getSupervisorState() ); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); - for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - SeekableStreamSupervisorStateManager.ExceptionEvent event = stateManager.getExceptionEvents().poll(); - Assert.assertEquals( - SeekableStreamSupervisorStateManager.StreamErrorTransience.NON_STREAM_ERROR, - event.getStreamErrorTransience() - ); - Assert.assertEquals(Exception.class, event.getExceptionClass()); - } + + stateManager.getExceptionEvents().forEach(x -> { + Assert.assertEquals(StreamErrorTransience.NON_STREAM_ERROR, x.getStreamErrorTransience()); + Assert.assertEquals(Exception.class, x.getExceptionClass()); + }); } @Test @@ -301,7 +302,7 @@ public void testTaskRecoveryWithHealthinessThreshold() @Test public void testTwoUnhealthyStates() { - stateManager.setState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); for (int i = 0; i < Math.max( config.getTaskUnhealthinessThreshold(), config.getUnhealthinessThreshold() @@ -332,18 +333,24 @@ public void testGetThrowableEvents() } stateManager.markRunFinishedAndEvaluateHealth(); stateManager.storeThrowableEvent(new PossiblyTransientStreamException(new Exception("oof"))); - Queue events = stateManager.getExceptionEvents(); - Assert.assertNotNull(events.peek().getErrorMessage()); - Assert.assertEquals(PossiblyTransientStreamException.class, events.poll().getExceptionClass()); - Assert.assertNotNull(events.peek().getErrorMessage()); - Assert.assertEquals(NullPointerException.class, events.poll().getExceptionClass()); - Assert.assertNotNull(events.peek().getErrorMessage()); - Assert.assertEquals(TransientStreamException.class, events.poll().getExceptionClass()); - Assert.assertNotNull(events.peek().getErrorMessage()); - Assert.assertEquals(NonTransientStreamException.class, events.poll().getExceptionClass()); - Assert.assertNotNull(events.peek().getErrorMessage()); - Assert.assertEquals(TransientStreamException.class, events.poll().getExceptionClass()); + List> expected = ImmutableList.of( + Pair.of(PossiblyTransientStreamException.class, StreamErrorTransience.POSSIBLY_TRANSIENT), + Pair.of(NullPointerException.class, StreamErrorTransience.NON_STREAM_ERROR), + Pair.of(TransientStreamException.class, StreamErrorTransience.TRANSIENT), + Pair.of(NonTransientStreamException.class, StreamErrorTransience.NON_TRANSIENT), + Pair.of(TransientStreamException.class, StreamErrorTransience.TRANSIENT) + ); + + Iterator it = stateManager.getExceptionEvents().iterator(); + expected.forEach(x -> { + SeekableStreamSupervisorStateManager.ExceptionEvent event = it.next(); + Assert.assertNotNull(event.getErrorMessage()); + Assert.assertEquals(x.lhs, event.getExceptionClass()); + Assert.assertEquals(x.rhs, event.getStreamErrorTransience()); + }); + + Assert.assertFalse(it.hasNext()); } @Test @@ -353,14 +360,15 @@ public void testExceptionEventSerDe() throws IOException new SeekableStreamSupervisorStateManager.ExceptionEvent( new NullPointerException("msg"), true, - SeekableStreamSupervisorStateManager.StreamErrorTransience.TRANSIENT + StreamErrorTransience.TRANSIENT ); + String serialized = defaultMapper.writeValueAsString(event); - SeekableStreamSupervisorStateManager.ExceptionEvent deserialized = - defaultMapper.readValue(serialized, SeekableStreamSupervisorStateManager.ExceptionEvent.class); - Assert.assertEquals(event.getErrorMessage(), deserialized.getErrorMessage()); - Assert.assertEquals(event.getExceptionClass(), deserialized.getExceptionClass()); - Assert.assertEquals(event.getTimestamp(), deserialized.getTimestamp()); - Assert.assertEquals(event.getStreamErrorTransience(), deserialized.getStreamErrorTransience()); + + Map deserialized = defaultMapper.readValue(serialized, new TypeReference>(){}); + Assert.assertNotNull(deserialized.get("timestamp")); + Assert.assertEquals("java.lang.NullPointerException", deserialized.get("exceptionClass")); + Assert.assertEquals("TRANSIENT", deserialized.get("errorTransience")); + Assert.assertNotNull(deserialized.get("message")); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index c31ed775223e..2599387dd158 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -167,7 +167,7 @@ public AppenderatorDriverAddResult add( * * @return {@link AppenderatorDriverAddResult} * - * @throws IOException if there is an I/O error while allocating or writing to a segmentq + * @throws IOException if there is an I/O error while allocating or writing to a segment */ public AppenderatorDriverAddResult add( final InputRow row, From 128edad16d6642e19a4df975b0311c72ab46af83 Mon Sep 17 00:00:00 2001 From: dclim Date: Fri, 3 May 2019 23:41:34 -0600 Subject: [PATCH 11/22] remove transience reporting and fix SeekableStreamSupervisorStateManager impl --- .../extensions-core/kafka-ingestion.md | 50 ++- .../extensions-core/kinesis-ingestion.md | 50 ++- .../indexing/kafka/KafkaRecordSupplier.java | 21 +- .../kafka/supervisor/KafkaSupervisor.java | 1 + .../KafkaSupervisorReportPayload.java | 4 + .../kafka/supervisor/KafkaSupervisorSpec.java | 2 +- .../supervisor/KafkaSupervisorSpecTest.java | 2 +- .../kafka/supervisor/KafkaSupervisorTest.java | 2 +- .../kinesis/KinesisRecordSupplier.java | 20 +- .../kinesis/supervisor/KinesisSupervisor.java | 1 + .../KinesisSupervisorReportPayload.java | 4 + .../supervisor/KinesisSupervisorSpec.java | 2 +- .../supervisor/KinesisSupervisorTest.java | 2 +- .../guice/SeekableStreamSupervisorModule.java | 2 +- .../StreamException.java} | 11 +- .../NonTransientStreamException.java | 33 -- .../PossiblyTransientStreamException.java | 34 -- .../supervisor/SeekableStreamSupervisor.java | 34 +- .../SeekableStreamSupervisorConfig.java | 4 +- ...SeekableStreamSupervisorReportPayload.java | 9 + .../SeekableStreamSupervisorSpec.java | 1 - .../SeekableStreamSupervisorStateManager.java | 364 +++++++----------- ...kableStreamSupervisorStateManagerTest.java | 316 ++++++--------- 23 files changed, 359 insertions(+), 610 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/{exceptions/TransientStreamException.java => common/StreamException.java} (71%) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/NonTransientStreamException.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/PossiblyTransientStreamException.java rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/{ => supervisor}/SeekableStreamSupervisorConfig.java (95%) diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 617c8466545b..18929d92ecad 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -218,25 +218,21 @@ The status report also contains the supervisor's state and a list of recently th controlled using the `druid.supervisor.stream.maxStoredExceptionEvents` config parameter). The list of states is as follows: -|State|Description|Priority| -|-----|-----------|--------| -|UNHEALTHY_SUPERVISOR|The supervisor has encountered non-transient errors on the past `druid.supervisor.stream.unhealthinessThreshold` iterations|1| -|UNHEALTHY_TASKS|The last `druid.supervisor.stream.taskUnhealthinessThreshold` tasks have all failed|2| -|UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kafka and has not successfully connected in the past|3| -|LOST_CONTACT_WITH_STREAM|The supervisor is encountering connectivity issues with Kafka but has successfully connected in the past|4| -|WAITING_TO_RUN (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream.|5| -|CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data|5| -|DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks|5| -|CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state|5| -|RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse|5| -|SUSPENDED|The supervisor has been suspended|5| -|SHUTTING_DOWN|Shutdown has been called but the supervisor hasn’t fully shutdown yet|5| - -Notes about states: - -- Since it's possible that 2+ states can apply to a supervisor at the same time, each state is given a priority. The -active state with the highest priority (i.e. lowest priority number) will be returned in the status report. -- States marked with "first iteration only" only occur on the supervisor's first iteration at startup or after suspension. +|State|Description| +|-----|-----------| +|UNHEALTHY_SUPERVISOR|The supervisor has encountered errors on the past `druid.supervisor.stream.unhealthinessThreshold` iterations| +|UNHEALTHY_TASKS|The last `druid.supervisor.stream.taskUnhealthinessThreshold` tasks have all failed| +|UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kafka and has not successfully connected in the past| +|LOST_CONTACT_WITH_STREAM|The supervisor is encountering connectivity issues with Kafka but has successfully connected in the past| +|WAITING_TO_RUN (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream| +|CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data| +|DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks| +|CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state| +|RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse| +|SUSPENDED|The supervisor has been suspended| +|SHUTTING_DOWN|Shutdown has been called but the supervisor hasn’t fully shutdown yet| + +States marked with "first iteration only" only occur on the supervisor's first iteration at startup or after suspension. ### Getting Supervisor Ingestion Stats Report @@ -371,11 +367,11 @@ Hadoop (see [here](https://github.com/apache/incubator-druid/pull/5102)). ## Configuration Properties -|property|description|values|default| -|--------|-----------|------|-------| -|druid.supervisor.stream.healthinessThreshold|The number of successful iterations before the supervisor flips from an UNHEALTHY to a RUNNING state|An integer in [3,2147483647]|3| -|druid.supervisor.stream.unhealthinessThreshold|The number of iterations failed before the supervisor flips from a RUNNING to an UNHEALTHY state|An integer in [3,2147483647]|3| -|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state|An integer in [3,2147483647]|3| -|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state|An integer in [3,2147483647]|3| -|druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|true/false|false| -|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|An integer in [`max(healthinessThreshold, unhealthinessThreshold)`, 2147483647]|`max(healthinessThreshold, unhealthinessThreshold)`| +|Property|Description|Default| +|--------|-----------|-------| +|druid.supervisor.stream.healthinessThreshold|The number of successful iterations before the supervisor flips from an UNHEALTHY to a RUNNING state|3| +|druid.supervisor.stream.unhealthinessThreshold|The number of iterations failed before the supervisor flips from a RUNNING to an UNHEALTHY state|3| +|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state|3| +|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state|3| +|druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|false| +|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|`max(healthinessThreshold, unhealthinessThreshold)`| diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md index 9bdf5068c797..98c463544ee7 100644 --- a/docs/content/development/extensions-core/kinesis-ingestion.md +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -232,25 +232,21 @@ The status report also contains the supervisor's state and a list of recently th controlled using the `druid.supervisor.stream.maxStoredExceptionEvents` config parameter). The list of states is as follows: -|State|Description|Priority| -|-----|-----------|--------| -|UNHEALTHY_SUPERVISOR|The supervisor has encountered non-transient errors on the past `druid.supervisor.stream.unhealthinessThreshold` iterations|1| -|UNHEALTHY_TASKS|The last `druid.supervisor.stream.taskUnhealthinessThreshold` tasks have all failed|2| -|UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kinesis and has not successfully connected in the past|3| -|LOST_CONTACT_WITH_STREAM|The supervisor is encountering transient connectivity issues with Kinesis but has successfully connected in the past|4| -|WAITING_TO_RUN (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream.|5| -|CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data|5| -|DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks|5| -|CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state|5| -|RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse|5| -|SUSPENDED|The supervisor has been suspended|5| -|SHUTTING_DOWN|Shutdown has been called but the supervisor hasn’t fully shutdown yet|5| - -Notes about states: - -- Since it's possible that 2+ states can apply to a supervisor at the same time, each state is given a priority. The -active state with the highest priority will be returned in the status report. -- States marked with "first iteration only" only occur on the supervisor's first iteration at startup or after suspension. +|State|Description| +|-----|-----------| +|UNHEALTHY_SUPERVISOR|The supervisor has encountered errors on the past `druid.supervisor.stream.unhealthinessThreshold` iterations| +|UNHEALTHY_TASKS|The last `druid.supervisor.stream.taskUnhealthinessThreshold` tasks have all failed| +|UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kinesis and has not successfully connected in the past| +|LOST_CONTACT_WITH_STREAM|The supervisor is encountering connectivity issues with Kinesis but has successfully connected in the past| +|WAITING_TO_RUN (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream| +|CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data| +|DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks| +|CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state| +|RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse| +|SUSPENDED|The supervisor has been suspended| +|SHUTTING_DOWN|Shutdown has been called but the supervisor hasn’t fully shutdown yet| + +States marked with "first iteration only" only occur on the supervisor's first iteration at startup or after suspension. ### Updating Existing Supervisors @@ -421,11 +417,11 @@ Then when submitting a supervisor-spec, set `deaggregate` to true. ## Configuration Properties -|property|description|values|default| -|--------|-----------|------|-------| -|druid.supervisor.stream.healthinessThreshold|The number of successful iterations before the supervisor flips from an UNHEALTHY to a RUNNING state|An integer in [3,2147483647]|3| -|druid.supervisor.stream.unhealthinessThreshold|The number of iterations failed before the supervisor flips from a RUNNING to an UNHEALTHY state|An integer in [3,2147483647]|3| -|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state|An integer in [3,2147483647]|3| -|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state|An integer in [3,2147483647]|3| -|druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|true/false|false| -|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|An integer in [`max(healthinessThreshold, unhealthinessThreshold)`, 2147483647]|`max(healthinessThreshold, unhealthinessThreshold)`| +|Property|Description|Default| +|--------|-----------|-------| +|druid.supervisor.stream.healthinessThreshold|The number of successful iterations before the supervisor flips from an UNHEALTHY to a RUNNING state|3| +|druid.supervisor.stream.unhealthinessThreshold|The number of iterations failed before the supervisor flips from a RUNNING to an UNHEALTHY state|3| +|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state|3| +|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state|3| +|druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|false| +|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|`max(healthinessThreshold, unhealthinessThreshold)`| diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 640251d4a544..e40f77a11523 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -21,24 +21,17 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import org.apache.curator.shaded.com.google.common.base.Throwables; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamException; import org.apache.druid.indexing.seekablestream.common.StreamPartition; -import org.apache.druid.indexing.seekablestream.exceptions.NonTransientStreamException; -import org.apache.druid.indexing.seekablestream.exceptions.PossiblyTransientStreamException; -import org.apache.druid.indexing.seekablestream.exceptions.TransientStreamException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.metadata.PasswordProvider; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.InterruptException; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import javax.annotation.Nonnull; @@ -226,18 +219,8 @@ private static T wrapExceptions(Callable callable) try { return callable.call(); } - catch (TimeoutException e) { - throw new PossiblyTransientStreamException(e); - } - catch (WakeupException | InterruptException e) { - throw new TransientStreamException(e); - } - catch (KafkaException e) { - throw new NonTransientStreamException(e); - } catch (Exception e) { - Throwables.throwIfUnchecked(e); - throw new RuntimeException(e); + throw new StreamException(e); } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 33af1c87539a..18148cac3a5d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -193,6 +193,7 @@ protected SeekableStreamSupervisorReportPayload createReportPaylo includeOffsets ? sequenceLastUpdated : null, spec.isSuspended(), stateManager.getSupervisorState(), + stateManager.getStateHistory(), stateManager.getExceptionEvents() ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index d3c33fc6ba12..699cf28edc56 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -41,6 +41,7 @@ public KafkaSupervisorReportPayload( @Nullable DateTime offsetsLastUpdated, boolean suspended, SeekableStreamSupervisorStateManager.State state, + List stateHistory, List recentErrors ) { @@ -56,6 +57,7 @@ public KafkaSupervisorReportPayload( offsetsLastUpdated, suspended, state, + stateHistory, recentErrors ); } @@ -76,6 +78,8 @@ public String toString() (getAggregateLag() != null ? ", aggregateLag=" + getAggregateLag() : "") + (getOffsetsLastUpdated() != null ? ", sequenceLastUpdated=" + getOffsetsLastUpdated() : "") + ", suspended=" + getSuspended() + + ", state=" + getState() + + ", stateHistory=" + getStateHistory() + ", recentErrors=" + getRecentErrors() + '}'; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index cee365c309d8..cde8f0b86877 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -30,7 +30,7 @@ import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java index ac8fb2057a2b..729d8a51fd57 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java @@ -28,7 +28,7 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.math.expr.ExprMacroTable; diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 8c4cb55129b5..16c22cb19e66 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -65,7 +65,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index d072c3c38c3c..a55a57282d3e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -26,15 +26,12 @@ import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; -import com.amazonaws.services.kinesis.model.AmazonKinesisException; import com.amazonaws.services.kinesis.model.ExpiredIteratorException; import com.amazonaws.services.kinesis.model.GetRecordsRequest; import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.amazonaws.services.kinesis.model.InvalidArgumentException; -import com.amazonaws.services.kinesis.model.LimitExceededException; import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; import com.amazonaws.services.kinesis.model.Record; -import com.amazonaws.services.kinesis.model.ResourceInUseException; import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.model.ShardIteratorType; @@ -44,15 +41,12 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Queues; -import org.apache.curator.shaded.com.google.common.base.Throwables; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.aws.AWSCredentialsUtils; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamException; import org.apache.druid.indexing.seekablestream.common.StreamPartition; -import org.apache.druid.indexing.seekablestream.exceptions.NonTransientStreamException; -import org.apache.druid.indexing.seekablestream.exceptions.PossiblyTransientStreamException; -import org.apache.druid.indexing.seekablestream.exceptions.TransientStreamException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; @@ -777,18 +771,8 @@ private static T wrapExceptions(Callable callable) try { return callable.call(); } - catch (LimitExceededException | ProvisionedThroughputExceededException | ResourceInUseException e) { - throw new TransientStreamException(e); - } - catch (ResourceNotFoundException e) { - throw new PossiblyTransientStreamException(e); - } - catch (AmazonKinesisException e) { - throw new NonTransientStreamException(e); - } catch (Exception e) { - Throwables.throwIfUnchecked(e); - throw new RuntimeException(e); + throw new StreamException(e); } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index d8b7aee1f719..96ac5901d283 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -249,6 +249,7 @@ protected SeekableStreamSupervisorReportPayload createReportPayl ioConfig.getTaskDuration().getMillis() / 1000, spec.isSuspended(), stateManager.getSupervisorState(), + stateManager.getStateHistory(), stateManager.getExceptionEvents() ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java index c8ca0e9fb713..f48898aa02d7 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java @@ -35,6 +35,7 @@ public KinesisSupervisorReportPayload( Long durationSeconds, boolean suspended, SeekableStreamSupervisorStateManager.State state, + List stateHistory, List recentErrors ) { @@ -50,6 +51,7 @@ public KinesisSupervisorReportPayload( null, suspended, state, + stateHistory, recentErrors ); } @@ -66,6 +68,8 @@ public String toString() ", active=" + getActiveTasks() + ", publishing=" + getPublishingTasks() + ", suspended=" + getSuspended() + + ", state=" + getState() + + ", stateHistory=" + getStateHistory() + ", recentErrors=" + getRecentErrors() + '}'; } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index e93f79d7c4e9..03c70450dcb2 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -32,7 +32,7 @@ import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 335a648e938b..c53b70bf8047 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -60,9 +60,9 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java index bb56610f73f3..6feea536842b 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java @@ -22,7 +22,7 @@ import com.google.inject.Binder; import com.google.inject.Module; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; public class SeekableStreamSupervisorModule implements Module { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/TransientStreamException.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamException.java similarity index 71% rename from indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/TransientStreamException.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamException.java index c0514cf3631a..4d2d7cede31d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/TransientStreamException.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamException.java @@ -17,16 +17,11 @@ * under the License. */ -package org.apache.druid.indexing.seekablestream.exceptions; +package org.apache.druid.indexing.seekablestream.common; -/** - * An exception wrapper for stream (Kafka/Kinesis) exceptions that will resolve over time. An example - * of such is some sort of rate limit exceeded exception. - */ - -public class TransientStreamException extends RuntimeException +public class StreamException extends RuntimeException { - public TransientStreamException(Throwable t) + public StreamException(Throwable t) { super(t); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/NonTransientStreamException.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/NonTransientStreamException.java deleted file mode 100644 index 546bbed3775c..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/NonTransientStreamException.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.seekablestream.exceptions; - -/** - * An exception wrapper for stream (Kafka/Kinesis) exceptions that are not expected to resolve over time. An example - * of such is some sort of auth exception. - */ - -public class NonTransientStreamException extends RuntimeException -{ - public NonTransientStreamException(Throwable t) - { - super(t); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/PossiblyTransientStreamException.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/PossiblyTransientStreamException.java deleted file mode 100644 index d0b0627e19c1..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/exceptions/PossiblyTransientStreamException.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.seekablestream.exceptions; - -/** - * An exception wrapper for stream (Kafka/Kinesis) exceptions that might resolve over time. An example - * of such is some sort of connectivity exception that could signify that the resource doesn't exist - * or that there's some temporary network issue. - */ - -public class PossiblyTransientStreamException extends RuntimeException -{ - public PossiblyTransientStreamException(Throwable t) - { - super(t); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index c3425d30f9d6..316e8fc7253a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -288,7 +288,12 @@ public void handle() throws ExecutionException, InterruptedException, TimeoutExc } lastRunTime = nowTime; - runInternal(); + try { + runInternal(); + } + finally { + stateManager.markRunFinished(); + } } } @@ -538,6 +543,8 @@ public SeekableStreamSupervisor( this.reportingExec = Execs.scheduledSingleThreaded(supervisorId + "-Reporting-%d"); this.stateManager = new SeekableStreamSupervisorStateManager( SeekableStreamSupervisorStateManager.State.WAITING_TO_RUN, + spec.isSuspended() ? SeekableStreamSupervisorStateManager.State.SUSPENDED + : SeekableStreamSupervisorStateManager.State.RUNNING, spec.getSupervisorConfig() ); @@ -659,6 +666,7 @@ public void stop(boolean stopGracefully) Preconditions.checkState(lifecycleStarted, "lifecycle not started"); log.info("Beginning shutdown of [%s]", supervisorId); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.SHUTTING_DOWN); try { scheduledExec.shutdownNow(); // stop recurring executions @@ -886,14 +894,12 @@ private SupervisorReport> getStats() { @@ -901,13 +907,11 @@ public Map> getStats() return getCurrentTotalStats(); } catch (InterruptedException ie) { - stateManager.storeThrowableEvent(ie); Thread.currentThread().interrupt(); log.error(ie, "getStats() interrupted."); throw new RuntimeException(ie); } catch (ExecutionException | TimeoutException eete) { - stateManager.storeThrowableEvent(eete); throw new RuntimeException(eete); } } @@ -1030,29 +1034,34 @@ public void addTaskGroupToPendingCompletionTaskGroup( } @VisibleForTesting - public void runInternal() - throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + public void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { possiblyRegisterListener(); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); updatePartitionDataFromStream(); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); discoverTasks(); + updateTaskStatus(); + checkTaskDuration(); + checkPendingCompletionTasks(); + checkCurrentTaskState(); + // if supervisor is not suspended, ensure required tasks are running // if suspended, ensure tasks have been requested to gracefully stop if (!spec.isSuspended()) { log.info("[%s] supervisor is running.", dataSource); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); createNewTasks(); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); } else { log.info("[%s] supervisor is suspended.", dataSource); gracefulShutdownInternal(); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.SUSPENDED); } if (log.isDebugEnabled()) { @@ -1060,7 +1069,6 @@ public void runInternal() } else { log.info(generateReport(false).toString()); } - stateManager.markRunFinishedAndEvaluateHealth(); } private void possiblyRegisterListener() @@ -1448,7 +1456,6 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) // make sure the checkpoints are consistent with each other and with the metadata store verifyAndMergeCheckpoints(taskGroupsToVerify.values()); - } private void verifyAndMergeCheckpoints(final Collection taskGroupsToVerify) @@ -2009,6 +2016,7 @@ private ListenableFuture> checkpointTas // This will cause us to create a new set of tasks next cycle that will start from the sequences 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) + stateManager.storeCompletedTaskState(TaskState.SUCCESS); return Futures.transform( stopTasksInGroup(taskGroup, "task[%s] succeeded in the taskGroup", task.status.getId()), new Function>() @@ -2220,6 +2228,7 @@ private void checkPendingCompletionTasks() Preconditions.checkNotNull(taskData.status, "WTH? task[%s] has a null status", taskId); if (taskData.status.isFailure()) { + stateManager.storeCompletedTaskState(TaskState.FAILED); iTask.remove(); // remove failed task if (group.tasks.isEmpty()) { // if all tasks in the group have failed, just nuke all task groups with this partition set and restart @@ -2232,6 +2241,7 @@ private void checkPendingCompletionTasks() // If one of the pending completion tasks was successful, stop the rest of the tasks in the group as // we no longer need them to publish their segment. log.info("Task [%s] completed successfully, stopping tasks %s", taskId, group.taskIds()); + stateManager.storeCompletedTaskState(TaskState.SUCCESS); futures.add( stopTasksInGroup(group, "Task [%s] completed successfully, stopping tasks %s", taskId, group.taskIds()) ); @@ -2636,7 +2646,6 @@ public Runnable updateCurrentAndLatestOffsets() sequenceLastUpdated = DateTimes.nowUtc(); } catch (Exception e) { - stateManager.storeThrowableEvent(e); log.warn(e, "Exception while getting current/latest sequences"); } }; @@ -2675,7 +2684,6 @@ private void updateLatestOffsetsFromStream() throws InterruptedException partitionIds = recordSupplier.getPartitionIds(ioConfig.getStream()); } catch (Exception e) { - stateManager.storeThrowableEvent(e); log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getStream()); throw new RuntimeException(e); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorConfig.java similarity index 95% rename from indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorConfig.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorConfig.java index 67207d24f5a4..da6b54930445 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.seekablestream; +package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; @@ -78,7 +78,7 @@ public int getMaxStoredExceptionEvents() } @VisibleForTesting - public void setMaxStoredExceptionEvents(int maxStoredExceptionEvents) + void setMaxStoredExceptionEvents(int maxStoredExceptionEvents) { this.maxStoredExceptionEvents = maxStoredExceptionEvents; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index f6c949389f63..1de4e8114808 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -45,6 +45,7 @@ public abstract class SeekableStreamSupervisorReportPayload stateHistory; private final List recentErrors; public SeekableStreamSupervisorReportPayload( @@ -59,6 +60,7 @@ public SeekableStreamSupervisorReportPayload( @Nullable DateTime offsetsLastUpdated, boolean suspended, SeekableStreamSupervisorStateManager.State state, + List stateHistory, List recentErrors ) { @@ -75,6 +77,7 @@ public SeekableStreamSupervisorReportPayload( this.offsetsLastUpdated = offsetsLastUpdated; this.suspended = suspended; this.state = state; + this.stateHistory = stateHistory; this.recentErrors = recentErrors; } @@ -167,6 +170,12 @@ public SeekableStreamSupervisorStateManager.State getState() return state; } + @JsonProperty + public List getStateHistory() + { + return stateHistory; + } + @JsonProperty public List getRecentErrors() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index bc655074e526..afd8fb9b7457 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -33,7 +33,6 @@ import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java index 63c2ccccede0..326c19c288a6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java @@ -22,215 +22,174 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.google.common.base.Preconditions; +import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; -import org.apache.druid.indexing.seekablestream.exceptions.NonTransientStreamException; -import org.apache.druid.indexing.seekablestream.exceptions.PossiblyTransientStreamException; -import org.apache.druid.indexing.seekablestream.exceptions.TransientStreamException; +import org.apache.druid.indexing.seekablestream.common.StreamException; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.utils.CircularBuffer; -import org.codehaus.plexus.util.ExceptionUtils; import org.joda.time.DateTime; import java.util.ArrayList; -import java.util.HashSet; +import java.util.Deque; import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.stream.Collectors; +import java.util.concurrent.ConcurrentLinkedDeque; public class SeekableStreamSupervisorStateManager { public enum State { // Error states - ordered from high to low priority - UNHEALTHY_SUPERVISOR(1, false, false), - UNHEALTHY_TASKS(2, false, false), - UNABLE_TO_CONNECT_TO_STREAM(3, false, false), - LOST_CONTACT_WITH_STREAM(4, false, false), + UNHEALTHY_SUPERVISOR(false, false), + UNHEALTHY_TASKS(false, false), + UNABLE_TO_CONNECT_TO_STREAM(false, false), + LOST_CONTACT_WITH_STREAM(false, false), // Non-error states - equal priority - WAITING_TO_RUN(5, true, true), - CONNECTING_TO_STREAM(5, true, true), - DISCOVERING_INITIAL_TASKS(5, true, true), - CREATING_TASKS(5, true, true), - RUNNING(5, true, false), - SUSPENDED(5, true, false), - SHUTTING_DOWN(5, true, false); - - // Lower priority number means higher priority and vice versa - private final int priority; + WAITING_TO_RUN(true, true), + CONNECTING_TO_STREAM(true, true), + DISCOVERING_INITIAL_TASKS(true, true), + CREATING_TASKS(true, true), + RUNNING(true, false), + SUSPENDED(true, false), + SHUTTING_DOWN(true, false); + private final boolean healthy; private final boolean firstRunOnly; - State(int priority, boolean healthy, boolean firstRunOnly) + State(boolean healthy, boolean firstRunOnly) { - this.priority = priority; this.healthy = healthy; this.firstRunOnly = firstRunOnly; } - - // We only want to set these only if the supervisor hasn't had a successful iteration yet - public boolean isFirstRunOnly() - { - return firstRunOnly; - } - - public boolean isHealthy() - { - return healthy; - } } - public enum StreamErrorTransience - { - TRANSIENT, - POSSIBLY_TRANSIENT, - NON_TRANSIENT, - NON_STREAM_ERROR - } + private final SeekableStreamSupervisorConfig supervisorConfig; + private final State healthySteadyState; + + private final Deque recentEventsQueue; + private final CircularBuffer stateHistory; private State supervisorState; - private final int healthinessThreshold; - private final int unhealthinessThreshold; - private final int healthinessTaskThreshold; - private final int unhealthinessTaskThreshold; private boolean atLeastOneSuccessfulRun = false; private boolean currentRunSuccessful = true; - private final CircularBuffer completedTaskHistory; - private final CircularBuffer supervisorStateHistory; // From previous runs - private final ExceptionEventStore eventStore; + + // Used to determine if a low consecutiveSuccessfulRuns/consecutiveSuccessfulTasks means that the supervisor is + // recovering from an unhealthy state, or if the supervisor just started and hasn't run many times yet. + private boolean hasHitUnhealthinessThreshold = false; + private boolean hasHitTaskUnhealthinessThreshold = false; + + private int consecutiveFailedRuns = 0; + private int consecutiveSuccessfulRuns = 0; + private int consecutiveFailedTasks = 0; + private int consecutiveSuccessfulTasks = 0; public SeekableStreamSupervisorStateManager( State initialState, + State healthySteadyState, SeekableStreamSupervisorConfig supervisorConfig ) { - this.supervisorState = initialState; - this.healthinessThreshold = supervisorConfig.getHealthinessThreshold(); - this.unhealthinessThreshold = supervisorConfig.getUnhealthinessThreshold(); - this.healthinessTaskThreshold = supervisorConfig.getTaskHealthinessThreshold(); - this.unhealthinessTaskThreshold = supervisorConfig.getTaskUnhealthinessThreshold(); - Preconditions.checkArgument(supervisorConfig.getMaxStoredExceptionEvents() >= Math.max( - healthinessThreshold, - unhealthinessThreshold + supervisorConfig.getHealthinessThreshold(), + supervisorConfig.getUnhealthinessThreshold() ), "maxStoredExceptionEvents must be >= to max(healthinessThreshold, unhealthinessThreshold)"); - this.eventStore = new ExceptionEventStore( - supervisorConfig.getMaxStoredExceptionEvents(), - supervisorConfig.isStoringStackTraces() - ); - this.completedTaskHistory = new CircularBuffer<>(Math.max(healthinessTaskThreshold, unhealthinessTaskThreshold)); - this.supervisorStateHistory = new CircularBuffer<>(Math.max(healthinessThreshold, unhealthinessThreshold)); + this.supervisorState = initialState; + this.supervisorConfig = supervisorConfig; + this.healthySteadyState = healthySteadyState; + + this.recentEventsQueue = new ConcurrentLinkedDeque<>(); + this.stateHistory = new CircularBuffer<>(supervisorConfig.getMaxStoredExceptionEvents()); } /** - * Certain supervisor states are only valid if the supervisor hasn't had a successful iteration yet. This function - * checks if there's been at least one successful iteration, and if applicable sets supervisor state to an appropriate - * new state. + * Certain states are only valid if the supervisor hasn't had a successful iteration. This method checks if there's + * been at least one successful iteration, and if applicable sets supervisor state to an appropriate new state. */ - public void maybeSetState(State newState) + public void maybeSetState(State proposedState) { - if (!newState.isFirstRunOnly() || !atLeastOneSuccessfulRun) { - supervisorState = newState; + // if we're over our unhealthiness threshold, set the state to the appropriate unhealthy state + if (consecutiveFailedRuns >= supervisorConfig.getUnhealthinessThreshold()) { + hasHitUnhealthinessThreshold = true; + supervisorState = recentEventsQueue.getLast().isStreamException() + ? (atLeastOneSuccessfulRun ? State.LOST_CONTACT_WITH_STREAM : State.UNABLE_TO_CONNECT_TO_STREAM) + : State.UNHEALTHY_SUPERVISOR; + return; + } + + // if we're over our task unhealthiness threshold, set the state to UNHEALTHY_TASKS + if (consecutiveFailedTasks >= supervisorConfig.getTaskUnhealthinessThreshold()) { + hasHitTaskUnhealthinessThreshold = true; + supervisorState = State.UNHEALTHY_TASKS; + return; + } + + // if we're currently in an unhealthy state and are below our healthiness threshold for either runs and tasks, + // ignore the proposed state; the healthiness threshold only applies if we've had a failure in the past + if (!this.supervisorState.healthy + && ((hasHitUnhealthinessThreshold && consecutiveSuccessfulRuns < supervisorConfig.getHealthinessThreshold()) + || (hasHitTaskUnhealthinessThreshold + && consecutiveSuccessfulTasks < supervisorConfig.getTaskHealthinessThreshold()))) { + return; } - if (State.SUSPENDED.equals(newState)) { - atLeastOneSuccessfulRun = false; // We want the startup states again after being suspended + // if we're trying to switch to a healthy steady state (i.e. RUNNING or SUSPENDED) but haven't had a successful run + // yet, refuse to switch and prefer the more specific states used for first run (CONNECTING_TO_STREAM, + // DISCOVERING_INITIAL_TASKS, CREATING_TASKS, etc.) + if (healthySteadyState.equals(proposedState) && !atLeastOneSuccessfulRun) { + return; + } + + // accept the state if it is not a firstRunOnly state OR we are still on the first run + if (!proposedState.firstRunOnly || !atLeastOneSuccessfulRun) { + supervisorState = proposedState; } } public void storeThrowableEvent(Throwable t) { - eventStore.storeThrowable(t instanceof PossiblyTransientStreamException && atLeastOneSuccessfulRun - ? new TransientStreamException(t.getCause()) - : t); + recentEventsQueue.add(new ExceptionEvent(t, supervisorConfig.isStoringStackTraces())); + + if (recentEventsQueue.size() > supervisorConfig.getMaxStoredExceptionEvents()) { + recentEventsQueue.poll(); + } currentRunSuccessful = false; } public void storeCompletedTaskState(TaskState state) { - completedTaskHistory.add(state); + if (state.isSuccess()) { + consecutiveSuccessfulTasks++; + consecutiveFailedTasks = 0; + } else if (state.isFailure()) { + consecutiveFailedTasks++; + consecutiveSuccessfulTasks = 0; + } } - public void markRunFinishedAndEvaluateHealth() + public void markRunFinished() { atLeastOneSuccessfulRun |= currentRunSuccessful; - State currentRunState = State.RUNNING; - - for (Map.Entry> events : eventStore.getRecentEventsMatchingExceptionsThrownOnCurrentRun().entrySet()) { - if (events.getValue().size() >= unhealthinessThreshold) { - if (events.getKey().equals(NonTransientStreamException.class) || - events.getKey().equals(TransientStreamException.class) || - events.getKey().equals(PossiblyTransientStreamException.class)) { - - currentRunState = atLeastOneSuccessfulRun - ? getHigherPriorityState(currentRunState, State.LOST_CONTACT_WITH_STREAM) - : getHigherPriorityState(currentRunState, State.UNABLE_TO_CONNECT_TO_STREAM); - } else { - currentRunState = getHigherPriorityState(currentRunState, State.UNHEALTHY_SUPERVISOR); - } - } - } + consecutiveSuccessfulRuns = currentRunSuccessful ? consecutiveSuccessfulRuns + 1 : 0; + consecutiveFailedRuns = currentRunSuccessful ? 0 : consecutiveFailedRuns + 1; - // Evaluate task health - if (supervisorState == State.UNHEALTHY_TASKS) { - boolean tasksHealthy = completedTaskHistory.size() >= healthinessTaskThreshold; - for (int i = 0; i < Math.min(healthinessTaskThreshold, completedTaskHistory.size()); i++) { - if (completedTaskHistory.getLatest(i) != TaskState.SUCCESS) { - tasksHealthy = false; - } - } - if (tasksHealthy) { - currentRunState = State.RUNNING; - } else { - currentRunState = State.UNHEALTHY_TASKS; - } - } else if (supervisorState != State.UNHEALTHY_SUPERVISOR) { - boolean tasksUnhealthy = completedTaskHistory.size() >= unhealthinessTaskThreshold; - for (int i = 0; i < Math.min(unhealthinessTaskThreshold, completedTaskHistory.size()); i++) { - // Last unhealthinessTaskThreshold tasks must be unhealthy for state to change to - // UNHEALTHY_TASKS - if (completedTaskHistory.getLatest(i) != TaskState.FAILED) { - tasksUnhealthy = false; - } - } - - if (tasksUnhealthy) { - currentRunState = getHigherPriorityState(currentRunState, State.UNHEALTHY_TASKS); - } - } - - supervisorStateHistory.add(currentRunState); + // Try to set the state to RUNNING or SUSPENDED, depending on how the supervisor was configured. This will be + // rejected if we haven't had atLeastOneSuccessfulRun (in favor of the more specific states for the initial run) and + // will instead trigger setting the state to an unhealthy one if we are now over the error thresholds. + maybeSetState(healthySteadyState); - if (currentRunState.isHealthy() && supervisorState == State.UNHEALTHY_SUPERVISOR) { - boolean supervisorHealthy = supervisorStateHistory.size() >= healthinessThreshold; - for (int i = 0; i < Math.min(healthinessThreshold, supervisorStateHistory.size()); i++) { - supervisorHealthy &= supervisorStateHistory.getLatest(i).isHealthy(); - } - - if (!supervisorHealthy) { - currentRunState = State.UNHEALTHY_SUPERVISOR; - } - } + stateHistory.add(supervisorState); - this.supervisorState = currentRunState; - - // Reset manager state for next run + // reset for next run currentRunSuccessful = true; - eventStore.resetErrorsEncounteredOnRun(); } public List getExceptionEvents() { - return eventStore.getRecentEvents(); + return new ArrayList<>(recentEventsQueue); } public State getSupervisorState() @@ -238,121 +197,64 @@ public State getSupervisorState() return supervisorState; } - private State getHigherPriorityState(State s1, State s2) + public List getStateHistory() { - return s1.priority < s2.priority ? s1 : s2; + List retVal = new ArrayList<>(); + for (int i = 0; i < stateHistory.size(); i++) { + retVal.add(stateHistory.get(i)); + } + + return retVal; } - @JsonPropertyOrder({"timestamp", "exceptionClass", "streamErrorTransience", "message"}) + @JsonPropertyOrder({"timestamp", "exceptionClass", "streamException", "message"}) public static class ExceptionEvent { - private Class exceptionClass; - // Contains full stackTrace if storingStackTraces is true - private String errorMessage; - private DateTime timestamp; - private StreamErrorTransience streamErrorTransience; - - public ExceptionEvent( - Throwable t, - boolean storingStackTraces, - StreamErrorTransience streamErrorTransience - ) - { - this.exceptionClass = t.getClass(); - this.errorMessage = storingStackTraces ? ExceptionUtils.getStackTrace(t) : t.getMessage(); - this.timestamp = DateTimes.nowUtc(); - this.streamErrorTransience = streamErrorTransience; - } + private final DateTime timestamp; + private final String exceptionClass; + private final boolean streamException; + private final String message; // contains full stackTrace if storingStackTraces is true - @JsonProperty("exceptionClass") - public Class getExceptionClass() + public ExceptionEvent(Throwable t, boolean storingStackTraces) { - return exceptionClass; - } - - @JsonProperty("message") - public String getErrorMessage() - { - return errorMessage; + this.timestamp = DateTimes.nowUtc(); + this.exceptionClass = getMeaningfulExceptionClass(t); + this.streamException = ExceptionUtils.indexOfType(t, StreamException.class) != -1; + this.message = storingStackTraces ? ExceptionUtils.getStackTrace(t) : t.getMessage(); } - @JsonProperty("timestamp") + @JsonProperty public DateTime getTimestamp() { return timestamp; } - @JsonProperty("errorTransience") - public StreamErrorTransience getStreamErrorTransience() - { - return streamErrorTransience; - } - } - - private static class ExceptionEventStore - { - private final Queue recentEventsQueue; - private final ConcurrentHashMap> recentEventsMap; - private final int numEventsToStore; - private final boolean storeStackTraces; - private final Set errorsEncounteredOnRun; - - private ExceptionEventStore(int numEventsToStore, boolean storeStackTraces) - { - this.recentEventsQueue = new ConcurrentLinkedQueue<>(); - this.recentEventsMap = new ConcurrentHashMap<>(numEventsToStore); - this.numEventsToStore = numEventsToStore; - this.storeStackTraces = storeStackTraces; - this.errorsEncounteredOnRun = new HashSet<>(); - } - - private void storeThrowable(Throwable t) + @JsonProperty + public String getExceptionClass() { - Queue exceptionEventsForClassT = recentEventsMap.getOrDefault( - t.getClass(), - new ConcurrentLinkedQueue<>() - ); - - StreamErrorTransience transience; - if (t instanceof PossiblyTransientStreamException) { - transience = StreamErrorTransience.POSSIBLY_TRANSIENT; - } else if (t instanceof TransientStreamException) { - transience = StreamErrorTransience.TRANSIENT; - } else if (t instanceof NonTransientStreamException) { - transience = StreamErrorTransience.NON_TRANSIENT; - } else { - transience = StreamErrorTransience.NON_STREAM_ERROR; - } - - ExceptionEvent eventToAdd = new ExceptionEvent(t, storeStackTraces, transience); - - recentEventsQueue.add(eventToAdd); - - if (recentEventsQueue.size() > numEventsToStore) { - ExceptionEvent removedEvent = recentEventsQueue.poll(); - recentEventsMap.get(removedEvent.getExceptionClass()).poll(); - } - exceptionEventsForClassT.add(eventToAdd); - errorsEncounteredOnRun.add(t.getClass()); - recentEventsMap.put(t.getClass(), exceptionEventsForClassT); + return exceptionClass; } - private void resetErrorsEncounteredOnRun() + @JsonProperty + public boolean isStreamException() { - errorsEncounteredOnRun.clear(); + return streamException; } - private List getRecentEvents() + @JsonProperty + public String getMessage() { - return new ArrayList<>(recentEventsQueue); + return message; } - private Map> getRecentEventsMatchingExceptionsThrownOnCurrentRun() + private String getMeaningfulExceptionClass(Throwable t) { - return recentEventsMap.entrySet() - .stream() - .filter(x -> errorsEncounteredOnRun.contains(x.getKey())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + return ((List) ExceptionUtils.getThrowableList(t)) + .stream() + .map(x -> x.getClass().getName()) + .filter(x -> !RuntimeException.class.getName().equals(x)) + .findFirst() + .orElse(Exception.class.getName()); } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java index aa329d6c836e..2ee2fbd3089f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java @@ -23,11 +23,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexing.seekablestream.SeekableStreamSupervisorConfig; -import org.apache.druid.indexing.seekablestream.exceptions.NonTransientStreamException; -import org.apache.druid.indexing.seekablestream.exceptions.PossiblyTransientStreamException; -import org.apache.druid.indexing.seekablestream.exceptions.TransientStreamException; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.StreamErrorTransience; +import org.apache.druid.indexing.seekablestream.common.StreamException; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.State; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Pair; import org.junit.Assert; @@ -50,164 +47,128 @@ public void setupTest() { config = new SeekableStreamSupervisorConfig(); config.setMaxStoredExceptionEvents(10); - stateManager = new SeekableStreamSupervisorStateManager( - SeekableStreamSupervisorStateManager.State.WAITING_TO_RUN, - config - ); + stateManager = new SeekableStreamSupervisorStateManager(State.WAITING_TO_RUN, State.RUNNING, config); defaultMapper = new DefaultObjectMapper(); } @Test public void testHappyPath() { - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM, - stateManager.getSupervisorState() - ); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS, - stateManager.getSupervisorState() - ); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.CREATING_TASKS, stateManager.getSupervisorState()); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.markRunFinishedAndEvaluateHealth(); - - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.markRunFinishedAndEvaluateHealth(); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - } + Assert.assertEquals(State.WAITING_TO_RUN, stateManager.getSupervisorState()); - @Test - public void testTransientStreamFailure() - { - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); - stateManager.markRunFinishedAndEvaluateHealth(); // clean run without errors - for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.storeThrowableEvent(new PossiblyTransientStreamException(new Exception("DOH!"))); - stateManager.markRunFinishedAndEvaluateHealth(); - } - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.LOST_CONTACT_WITH_STREAM, - stateManager.getSupervisorState() - ); - Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); + stateManager.maybeSetState(State.CONNECTING_TO_STREAM); + Assert.assertEquals(State.CONNECTING_TO_STREAM, stateManager.getSupervisorState()); - stateManager.getExceptionEvents().forEach(x -> { - Assert.assertEquals(StreamErrorTransience.TRANSIENT, x.getStreamErrorTransience()); - Assert.assertEquals(TransientStreamException.class, x.getExceptionClass()); - }); + stateManager.maybeSetState(State.DISCOVERING_INITIAL_TASKS); + Assert.assertEquals(State.DISCOVERING_INITIAL_TASKS, stateManager.getSupervisorState()); + + stateManager.maybeSetState(State.CREATING_TASKS); + Assert.assertEquals(State.CREATING_TASKS, stateManager.getSupervisorState()); + + stateManager.markRunFinished(); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + + + stateManager.maybeSetState(State.WAITING_TO_RUN); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + + stateManager.maybeSetState(State.CONNECTING_TO_STREAM); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + + stateManager.maybeSetState(State.DISCOVERING_INITIAL_TASKS); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + + stateManager.maybeSetState(State.CREATING_TASKS); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + + stateManager.markRunFinished(); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); } @Test - public void testNonTransientStreamFailure() + public void testStreamFailureLostContact() { - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.markRunFinished(); // clean run without errors + + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.storeThrowableEvent(new NonTransientStreamException(new Exception("DOH!"))); - stateManager.markRunFinishedAndEvaluateHealth(); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + stateManager.storeThrowableEvent(new StreamException(new Exception("DOH!"))); + stateManager.markRunFinished(); } - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.UNABLE_TO_CONNECT_TO_STREAM, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, stateManager.getSupervisorState()); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); stateManager.getExceptionEvents().forEach(x -> { - Assert.assertEquals(StreamErrorTransience.NON_TRANSIENT, x.getStreamErrorTransience()); - Assert.assertEquals(NonTransientStreamException.class, x.getExceptionClass()); + Assert.assertTrue(x.isStreamException()); + Assert.assertEquals(StreamException.class.getName(), x.getExceptionClass()); }); } @Test - public void testPossiblyTransientStreamFailure() + public void testStreamFailureUnableToConnect() { - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(State.CONNECTING_TO_STREAM); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); - stateManager.storeThrowableEvent(new PossiblyTransientStreamException(new Exception("DOH!"))); - stateManager.markRunFinishedAndEvaluateHealth(); + Assert.assertEquals(State.CONNECTING_TO_STREAM, stateManager.getSupervisorState()); + stateManager.storeThrowableEvent(new StreamException(new Exception("DOH!"))); + stateManager.markRunFinished(); } - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.UNABLE_TO_CONNECT_TO_STREAM, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, stateManager.getSupervisorState()); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); stateManager.getExceptionEvents().forEach(x -> { - Assert.assertEquals(StreamErrorTransience.POSSIBLY_TRANSIENT, x.getStreamErrorTransience()); - Assert.assertEquals(PossiblyTransientStreamException.class, x.getExceptionClass()); + Assert.assertTrue(x.isStreamException()); + Assert.assertEquals(StreamException.class.getName(), x.getExceptionClass()); }); } @Test - public void testNonTransientUnhealthiness() + public void testNonStreamUnhealthiness() { - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.maybeSetState(State.DISCOVERING_INITIAL_TASKS); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(State.DISCOVERING_INITIAL_TASKS, stateManager.getSupervisorState()); stateManager.storeThrowableEvent(new NullPointerException("oof")); - stateManager.markRunFinishedAndEvaluateHealth(); + stateManager.markRunFinished(); } - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.UNHEALTHY_SUPERVISOR, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); stateManager.getExceptionEvents().forEach(x -> { - Assert.assertEquals(StreamErrorTransience.NON_STREAM_ERROR, x.getStreamErrorTransience()); - Assert.assertEquals(NullPointerException.class, x.getExceptionClass()); + Assert.assertFalse(x.isStreamException()); + Assert.assertEquals(NullPointerException.class.getName(), x.getExceptionClass()); }); } @Test public void testTransientUnhealthiness() { - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); - for (int i = 0; i < config.getUnhealthinessThreshold() - 1; i++) { - stateManager.storeThrowableEvent(new NullPointerException("oof")); - stateManager.markRunFinishedAndEvaluateHealth(); - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.RUNNING, - stateManager.getSupervisorState() - ); + stateManager.markRunFinished(); + for (int j = 1; j < 3; j++) { + for (int i = 0; i < config.getUnhealthinessThreshold() - 1; i++) { + stateManager.storeThrowableEvent(new NullPointerException("oof")); + stateManager.markRunFinished(); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + } + + stateManager.markRunFinished(); // clean run + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(j * (config.getUnhealthinessThreshold() - 1), stateManager.getExceptionEvents().size()); } - stateManager.markRunFinishedAndEvaluateHealth(); // clean run - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.RUNNING, - stateManager.getSupervisorState() - ); - Assert.assertEquals(config.getUnhealthinessThreshold() - 1, stateManager.getExceptionEvents().size()); } @Test public void testNonTransientTaskUnhealthiness() { + stateManager.markRunFinished(); for (int i = 0; i < config.getTaskUnhealthinessThreshold(); i++) { - Assert.assertNotEquals( - SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); stateManager.storeCompletedTaskState(TaskState.FAILED); - stateManager.markRunFinishedAndEvaluateHealth(); + stateManager.markRunFinished(); } - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.UNHEALTHY_TASKS, stateManager.getSupervisorState()); Assert.assertEquals(0, stateManager.getExceptionEvents().size()); } @@ -215,20 +176,14 @@ public void testNonTransientTaskUnhealthiness() public void testTransientTaskUnhealthiness() { // Only half are failing - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); + stateManager.markRunFinished(); for (int i = 0; i < config.getTaskUnhealthinessThreshold() + 3; i++) { - Assert.assertNotEquals( - stateManager.getSupervisorState(), - SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS - ); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); stateManager.storeCompletedTaskState(TaskState.FAILED); stateManager.storeCompletedTaskState(TaskState.SUCCESS); - stateManager.markRunFinishedAndEvaluateHealth(); + stateManager.markRunFinished(); } - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.RUNNING, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); Assert.assertEquals(0, stateManager.getExceptionEvents().size()); } @@ -237,138 +192,117 @@ public void testSupervisorRecoveryWithHealthinessThreshold() { // Put into an unhealthy state for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - Assert.assertNotEquals( - SeekableStreamSupervisorStateManager.State.UNHEALTHY_SUPERVISOR, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.WAITING_TO_RUN, stateManager.getSupervisorState()); stateManager.storeThrowableEvent(new Exception("Except the inevitable")); - stateManager.markRunFinishedAndEvaluateHealth(); + stateManager.markRunFinished(); } - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.UNHEALTHY_SUPERVISOR, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); + // Recover after config.healthinessThreshold successful task completions for (int i = 0; i < config.getHealthinessThreshold(); i++) { - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.UNHEALTHY_SUPERVISOR, - stateManager.getSupervisorState() - ); - stateManager.markRunFinishedAndEvaluateHealth(); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); + stateManager.markRunFinished(); } - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.RUNNING, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); stateManager.getExceptionEvents().forEach(x -> { - Assert.assertEquals(StreamErrorTransience.NON_STREAM_ERROR, x.getStreamErrorTransience()); - Assert.assertEquals(Exception.class, x.getExceptionClass()); + Assert.assertFalse(x.isStreamException()); + Assert.assertEquals(Exception.class.getName(), x.getExceptionClass()); }); } @Test public void testTaskRecoveryWithHealthinessThreshold() { + stateManager.markRunFinished(); + // Put into an unhealthy state for (int i = 0; i < config.getTaskUnhealthinessThreshold(); i++) { - Assert.assertNotEquals( - SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); stateManager.storeCompletedTaskState(TaskState.FAILED); - stateManager.markRunFinishedAndEvaluateHealth(); + stateManager.markRunFinished(); } - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.UNHEALTHY_TASKS, stateManager.getSupervisorState()); + // Recover after config.healthinessThreshold successful task completions for (int i = 0; i < config.getTaskHealthinessThreshold(); i++) { - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.UNHEALTHY_TASKS, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.UNHEALTHY_TASKS, stateManager.getSupervisorState()); stateManager.storeCompletedTaskState(TaskState.SUCCESS); - stateManager.markRunFinishedAndEvaluateHealth(); + stateManager.markRunFinished(); } - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.RUNNING, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); } @Test public void testTwoUnhealthyStates() { - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.RUNNING); - for (int i = 0; i < Math.max( - config.getTaskUnhealthinessThreshold(), - config.getUnhealthinessThreshold() - ); i++) { + stateManager.markRunFinished(); + + for (int i = 0; i < Math.max(config.getTaskUnhealthinessThreshold(), config.getUnhealthinessThreshold()); i++) { stateManager.storeThrowableEvent(new NullPointerException("somebody goofed")); stateManager.storeCompletedTaskState(TaskState.FAILED); - stateManager.markRunFinishedAndEvaluateHealth(); + stateManager.markRunFinished(); } // UNHEALTHY_SUPERVISOR should take priority over UNHEALTHY_TASKS - Assert.assertEquals( - SeekableStreamSupervisorStateManager.State.UNHEALTHY_SUPERVISOR, - stateManager.getSupervisorState() - ); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); } @Test public void testGetThrowableEvents() { List exceptions = ImmutableList.of( - new PossiblyTransientStreamException(new Exception("oof")), + new StreamException(new Exception("oof")), new NullPointerException("oof"), - new TransientStreamException(new Exception("oof")), - new NonTransientStreamException(new Exception("oof")) + new RuntimeException(new StreamException(new Exception("oof"))), + new RuntimeException(new IllegalArgumentException("oof")) ); for (Exception exception : exceptions) { stateManager.storeThrowableEvent(exception); - stateManager.markRunFinishedAndEvaluateHealth(); + stateManager.markRunFinished(); } - stateManager.markRunFinishedAndEvaluateHealth(); - stateManager.storeThrowableEvent(new PossiblyTransientStreamException(new Exception("oof"))); - - List> expected = ImmutableList.of( - Pair.of(PossiblyTransientStreamException.class, StreamErrorTransience.POSSIBLY_TRANSIENT), - Pair.of(NullPointerException.class, StreamErrorTransience.NON_STREAM_ERROR), - Pair.of(TransientStreamException.class, StreamErrorTransience.TRANSIENT), - Pair.of(NonTransientStreamException.class, StreamErrorTransience.NON_TRANSIENT), - Pair.of(TransientStreamException.class, StreamErrorTransience.TRANSIENT) + + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); + + Assert.assertEquals(ImmutableList.of( + State.WAITING_TO_RUN, + State.WAITING_TO_RUN, + State.UNABLE_TO_CONNECT_TO_STREAM, + State.UNHEALTHY_SUPERVISOR + ), stateManager.getStateHistory()); + + List> expected = ImmutableList.of( + Pair.of("org.apache.druid.indexing.seekablestream.common.StreamException", true), + Pair.of("java.lang.NullPointerException", false), + Pair.of("org.apache.druid.indexing.seekablestream.common.StreamException", true), + Pair.of("java.lang.IllegalArgumentException", false) ); Iterator it = stateManager.getExceptionEvents().iterator(); expected.forEach(x -> { SeekableStreamSupervisorStateManager.ExceptionEvent event = it.next(); - Assert.assertNotNull(event.getErrorMessage()); + Assert.assertNotNull(event.getMessage()); Assert.assertEquals(x.lhs, event.getExceptionClass()); - Assert.assertEquals(x.rhs, event.getStreamErrorTransience()); + Assert.assertEquals(x.rhs, event.isStreamException()); }); Assert.assertFalse(it.hasNext()); } @Test - public void testExceptionEventSerDe() throws IOException + public void testExceptionEventSerde() throws IOException { SeekableStreamSupervisorStateManager.ExceptionEvent event = - new SeekableStreamSupervisorStateManager.ExceptionEvent( - new NullPointerException("msg"), - true, - StreamErrorTransience.TRANSIENT - ); + new SeekableStreamSupervisorStateManager.ExceptionEvent(new NullPointerException("msg"), true); String serialized = defaultMapper.writeValueAsString(event); - Map deserialized = defaultMapper.readValue(serialized, new TypeReference>(){}); + Map deserialized = defaultMapper.readValue(serialized, new TypeReference>() + { + }); Assert.assertNotNull(deserialized.get("timestamp")); Assert.assertEquals("java.lang.NullPointerException", deserialized.get("exceptionClass")); - Assert.assertEquals("TRANSIENT", deserialized.get("errorTransience")); + Assert.assertFalse(Boolean.getBoolean(deserialized.get("streamException"))); Assert.assertNotNull(deserialized.get("message")); } } From f686d2aba74e12db45c95d9b179cc6b62bed5be2 Mon Sep 17 00:00:00 2001 From: dclim Date: Sat, 4 May 2019 01:43:27 -0600 Subject: [PATCH 12/22] move call to stateManager.markRunFinished() from RunNotice to runInternal() for tests --- .../supervisor/SeekableStreamSupervisor.java | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 316e8fc7253a..5e08339fa942 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -288,12 +288,7 @@ public void handle() throws ExecutionException, InterruptedException, TimeoutExc } lastRunTime = nowTime; - try { - runInternal(); - } - finally { - stateManager.markRunFinished(); - } + runInternal(); } } @@ -1036,32 +1031,37 @@ public void addTaskGroupToPendingCompletionTaskGroup( @VisibleForTesting public void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { - possiblyRegisterListener(); + try { + possiblyRegisterListener(); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); - updatePartitionDataFromStream(); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + updatePartitionDataFromStream(); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); - discoverTasks(); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + discoverTasks(); - updateTaskStatus(); + updateTaskStatus(); - checkTaskDuration(); + checkTaskDuration(); - checkPendingCompletionTasks(); + checkPendingCompletionTasks(); - checkCurrentTaskState(); + checkCurrentTaskState(); - // if supervisor is not suspended, ensure required tasks are running - // if suspended, ensure tasks have been requested to gracefully stop - if (!spec.isSuspended()) { - log.info("[%s] supervisor is running.", dataSource); + // if supervisor is not suspended, ensure required tasks are running + // if suspended, ensure tasks have been requested to gracefully stop + if (!spec.isSuspended()) { + log.info("[%s] supervisor is running.", dataSource); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); - createNewTasks(); - } else { - log.info("[%s] supervisor is suspended.", dataSource); - gracefulShutdownInternal(); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + createNewTasks(); + } else { + log.info("[%s] supervisor is suspended.", dataSource); + gracefulShutdownInternal(); + } + } + finally { + stateManager.markRunFinished(); } if (log.isDebugEnabled()) { From 5519d4c4cd6bac2fed66f2e60672241acf459d17 Mon Sep 17 00:00:00 2001 From: dclim Date: Mon, 6 May 2019 01:30:28 -0600 Subject: [PATCH 13/22] remove stateHistory because it wasn't adding much value, some fixes, and add more tests --- .../indexing/kafka/KafkaRecordSupplier.java | 8 +- .../kafka/supervisor/KafkaSupervisor.java | 1 - .../KafkaSupervisorReportPayload.java | 3 - .../kinesis/supervisor/KinesisSupervisor.java | 1 - .../KinesisSupervisorReportPayload.java | 3 - .../supervisor/SeekableStreamSupervisor.java | 33 +- ...SeekableStreamSupervisorReportPayload.java | 9 - .../SeekableStreamSupervisorStateManager.java | 22 +- ...kableStreamSupervisorStateManagerTest.java | 21 +- .../SeekableStreamSupervisorStateTest.java | 841 ++++++++++++++++++ 10 files changed, 881 insertions(+), 61 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index e40f77a11523..3329a286a7ec 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -156,11 +156,13 @@ public Long getPosition(StreamPartition partition) public Set getPartitionIds(String stream) { return wrapExceptions(() -> { - List partitions = consumer.partitionsFor(stream); - if (partitions == null) { + // use consumer.listTopics() instead of partitionsFor() to force a remote call so we can detect stream issues + Map> topics = consumer.listTopics(); + if (topics == null || topics.get(stream) == null) { throw new ISE("Topic [%s] is not found in KafkaConsumer's list of topics", stream); } - return partitions.stream().map(PartitionInfo::partition).collect(Collectors.toSet()); + + return topics.get(stream).stream().map(PartitionInfo::partition).collect(Collectors.toSet()); }); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 18148cac3a5d..33af1c87539a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -193,7 +193,6 @@ protected SeekableStreamSupervisorReportPayload createReportPaylo includeOffsets ? sequenceLastUpdated : null, spec.isSuspended(), stateManager.getSupervisorState(), - stateManager.getStateHistory(), stateManager.getExceptionEvents() ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index 699cf28edc56..49a2c9457083 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -41,7 +41,6 @@ public KafkaSupervisorReportPayload( @Nullable DateTime offsetsLastUpdated, boolean suspended, SeekableStreamSupervisorStateManager.State state, - List stateHistory, List recentErrors ) { @@ -57,7 +56,6 @@ public KafkaSupervisorReportPayload( offsetsLastUpdated, suspended, state, - stateHistory, recentErrors ); } @@ -79,7 +77,6 @@ public String toString() (getOffsetsLastUpdated() != null ? ", sequenceLastUpdated=" + getOffsetsLastUpdated() : "") + ", suspended=" + getSuspended() + ", state=" + getState() + - ", stateHistory=" + getStateHistory() + ", recentErrors=" + getRecentErrors() + '}'; } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 96ac5901d283..d8b7aee1f719 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -249,7 +249,6 @@ protected SeekableStreamSupervisorReportPayload createReportPayl ioConfig.getTaskDuration().getMillis() / 1000, spec.isSuspended(), stateManager.getSupervisorState(), - stateManager.getStateHistory(), stateManager.getExceptionEvents() ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java index f48898aa02d7..6a4a4236ba0a 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java @@ -35,7 +35,6 @@ public KinesisSupervisorReportPayload( Long durationSeconds, boolean suspended, SeekableStreamSupervisorStateManager.State state, - List stateHistory, List recentErrors ) { @@ -51,7 +50,6 @@ public KinesisSupervisorReportPayload( null, suspended, state, - stateHistory, recentErrors ); } @@ -69,7 +67,6 @@ public String toString() ", publishing=" + getPublishingTasks() + ", suspended=" + getSuspended() + ", state=" + getState() + - ", stateHistory=" + getStateHistory() + ", recentErrors=" + getRecentErrors() + '}'; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 5e08339fa942..44cd0d56d242 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1035,7 +1035,9 @@ public void runInternal() throws ExecutionException, InterruptedException, Timeo possiblyRegisterListener(); stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); - updatePartitionDataFromStream(); + if (!updatePartitionDataFromStream() && !stateManager.isAtLeastOneSuccessfulRun()) { + return; // if we can't connect to the stream and this is the first run, stop and wait to retry the connection + } stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); discoverTasks(); @@ -1059,16 +1061,20 @@ public void runInternal() throws ExecutionException, InterruptedException, Timeo log.info("[%s] supervisor is suspended.", dataSource); gracefulShutdownInternal(); } + + if (log.isDebugEnabled()) { + log.debug(generateReport(true).toString()); + } else { + log.info(generateReport(false).toString()); + } + } + catch (Exception e) { + stateManager.storeThrowableEvent(e); + log.warn(e, "Exception in supervisor run loop for dataSource [%s]", dataSource); } finally { stateManager.markRunFinished(); } - - if (log.isDebugEnabled()) { - log.debug(generateReport(true).toString()); - } else { - log.info(generateReport(false).toString()); - } } private void possiblyRegisterListener() @@ -1197,7 +1203,6 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata(dataSource, newMetadata); } catch (IOException e) { - stateManager.storeThrowableEvent(e); log.error("Resetting DataSourceMetadata failed [%s]", e.getMessage()); throw new RuntimeException(e); } @@ -1468,7 +1473,6 @@ private void verifyAndMergeCheckpoints(final Collection taskGroupsToV Futures.allAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { - stateManager.storeThrowableEvent(e); throw new RuntimeException(e); } } @@ -1522,7 +1526,6 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) } } catch (Exception e) { - stateManager.storeThrowableEvent(e); throw new RuntimeException(e); } @@ -1812,7 +1815,7 @@ protected String generateSequenceName( protected abstract String baseTaskName(); - private void updatePartitionDataFromStream() + private boolean updatePartitionDataFromStream() { Set partitionIds; try { @@ -1822,14 +1825,14 @@ private void updatePartitionDataFromStream() } catch (Exception e) { stateManager.storeThrowableEvent(e); - log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getStream()); + log.warn("Could not fetch partitions for topic/stream [%s]: %s", ioConfig.getStream(), e.getMessage()); log.debug(e, "full stack trace"); - return; + return false; } if (partitionIds == null || partitionIds.size() == 0) { log.warn("No partitions found for stream[%s]", ioConfig.getStream()); - return; + return false; } log.debug("Found [%d] partitions for stream [%s]", partitionIds.size(), ioConfig.getStream()); @@ -1869,6 +1872,8 @@ private void updatePartitionDataFromStream() ); } } + + return true; } private void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index 1de4e8114808..f6c949389f63 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -45,7 +45,6 @@ public abstract class SeekableStreamSupervisorReportPayload stateHistory; private final List recentErrors; public SeekableStreamSupervisorReportPayload( @@ -60,7 +59,6 @@ public SeekableStreamSupervisorReportPayload( @Nullable DateTime offsetsLastUpdated, boolean suspended, SeekableStreamSupervisorStateManager.State state, - List stateHistory, List recentErrors ) { @@ -77,7 +75,6 @@ public SeekableStreamSupervisorReportPayload( this.offsetsLastUpdated = offsetsLastUpdated; this.suspended = suspended; this.state = state; - this.stateHistory = stateHistory; this.recentErrors = recentErrors; } @@ -170,12 +167,6 @@ public SeekableStreamSupervisorStateManager.State getState() return state; } - @JsonProperty - public List getStateHistory() - { - return stateHistory; - } - @JsonProperty public List getRecentErrors() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java index 326c19c288a6..c2d014976c76 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java @@ -22,11 +22,11 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.seekablestream.common.StreamException; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.utils.CircularBuffer; import org.joda.time.DateTime; import java.util.ArrayList; @@ -67,7 +67,6 @@ public enum State private final State healthySteadyState; private final Deque recentEventsQueue; - private final CircularBuffer stateHistory; private State supervisorState; @@ -100,7 +99,6 @@ public SeekableStreamSupervisorStateManager( this.healthySteadyState = healthySteadyState; this.recentEventsQueue = new ConcurrentLinkedDeque<>(); - this.stateHistory = new CircularBuffer<>(supervisorConfig.getMaxStoredExceptionEvents()); } /** @@ -181,8 +179,6 @@ public void markRunFinished() // will instead trigger setting the state to an unhealthy one if we are now over the error thresholds. maybeSetState(healthySteadyState); - stateHistory.add(supervisorState); - // reset for next run currentRunSuccessful = true; } @@ -197,19 +193,19 @@ public State getSupervisorState() return supervisorState; } - public List getStateHistory() + public boolean isAtLeastOneSuccessfulRun() { - List retVal = new ArrayList<>(); - for (int i = 0; i < stateHistory.size(); i++) { - retVal.add(stateHistory.get(i)); - } - - return retVal; + return atLeastOneSuccessfulRun; } @JsonPropertyOrder({"timestamp", "exceptionClass", "streamException", "message"}) public static class ExceptionEvent { + private static final List SKIPPED_EXCEPTION_CLASSES = ImmutableList.of( + RuntimeException.class.getName(), + StreamException.class.getName() + ); + private final DateTime timestamp; private final String exceptionClass; private final boolean streamException; @@ -252,7 +248,7 @@ private String getMeaningfulExceptionClass(Throwable t) return ((List) ExceptionUtils.getThrowableList(t)) .stream() .map(x -> x.getClass().getName()) - .filter(x -> !RuntimeException.class.getName().equals(x)) + .filter(x -> !SKIPPED_EXCEPTION_CLASSES.contains(x)) .findFirst() .orElse(Exception.class.getName()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java index 2ee2fbd3089f..732e774c3910 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java @@ -94,7 +94,7 @@ public void testStreamFailureLostContact() for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); - stateManager.storeThrowableEvent(new StreamException(new Exception("DOH!"))); + stateManager.storeThrowableEvent(new StreamException(new IllegalStateException("DOH!"))); stateManager.markRunFinished(); } Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, stateManager.getSupervisorState()); @@ -102,7 +102,7 @@ public void testStreamFailureLostContact() stateManager.getExceptionEvents().forEach(x -> { Assert.assertTrue(x.isStreamException()); - Assert.assertEquals(StreamException.class.getName(), x.getExceptionClass()); + Assert.assertEquals(IllegalStateException.class.getName(), x.getExceptionClass()); }); } @@ -112,7 +112,7 @@ public void testStreamFailureUnableToConnect() stateManager.maybeSetState(State.CONNECTING_TO_STREAM); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(State.CONNECTING_TO_STREAM, stateManager.getSupervisorState()); - stateManager.storeThrowableEvent(new StreamException(new Exception("DOH!"))); + stateManager.storeThrowableEvent(new StreamException(new IllegalStateException("DOH!"))); stateManager.markRunFinished(); } Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, stateManager.getSupervisorState()); @@ -120,7 +120,7 @@ public void testStreamFailureUnableToConnect() stateManager.getExceptionEvents().forEach(x -> { Assert.assertTrue(x.isStreamException()); - Assert.assertEquals(StreamException.class.getName(), x.getExceptionClass()); + Assert.assertEquals(IllegalStateException.class.getName(), x.getExceptionClass()); }); } @@ -252,7 +252,7 @@ public void testTwoUnhealthyStates() public void testGetThrowableEvents() { List exceptions = ImmutableList.of( - new StreamException(new Exception("oof")), + new StreamException(new UnsupportedOperationException("oof")), new NullPointerException("oof"), new RuntimeException(new StreamException(new Exception("oof"))), new RuntimeException(new IllegalArgumentException("oof")) @@ -264,17 +264,10 @@ public void testGetThrowableEvents() Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); - Assert.assertEquals(ImmutableList.of( - State.WAITING_TO_RUN, - State.WAITING_TO_RUN, - State.UNABLE_TO_CONNECT_TO_STREAM, - State.UNHEALTHY_SUPERVISOR - ), stateManager.getStateHistory()); - List> expected = ImmutableList.of( - Pair.of("org.apache.druid.indexing.seekablestream.common.StreamException", true), + Pair.of("java.lang.UnsupportedOperationException", true), Pair.of("java.lang.NullPointerException", false), - Pair.of("org.apache.druid.indexing.seekablestream.common.StreamException", true), + Pair.of("java.lang.Exception", true), Pair.of("java.lang.IllegalArgumentException", false) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java new file mode 100644 index 000000000000..3a45e23211ce --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -0,0 +1,841 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.seekablestream.supervisor; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskQueue; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamException; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.State; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; +import org.apache.druid.server.security.AuthorizerMapper; +import org.easymock.EasyMockSupport; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.File; +import java.math.BigInteger; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; + +import static org.easymock.EasyMock.anyInt; +import static org.easymock.EasyMock.anyLong; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.anyString; +import static org.easymock.EasyMock.expect; + +public class SeekableStreamSupervisorStateTest extends EasyMockSupport +{ + private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + private static final String DATASOURCE = "testDS"; + private static final String STREAM = "stream"; + private static final String SHARD_ID = "0"; + private static final StreamPartition shard0Partition = StreamPartition.of(STREAM, SHARD_ID); + private static final String EXCEPTION_MSG = "I had an exception"; + + private TaskStorage taskStorage; + private TaskMaster taskMaster; + private TaskRunner taskRunner; + private TaskQueue taskQueue; + private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + private SeekableStreamIndexTaskClientFactory taskClientFactory; + private SeekableStreamSupervisorSpec spec; + private SeekableStreamIndexTaskClient indexTaskClient; + private RecordSupplier recordSupplier; + + private RowIngestionMetersFactory rowIngestionMetersFactory; + private SeekableStreamSupervisorConfig supervisorConfig; + + @Before + public void setupTest() + { + taskStorage = createMock(TaskStorage.class); + taskMaster = createMock(TaskMaster.class); + taskRunner = createMock(TaskRunner.class); + taskQueue = createMock(TaskQueue.class); + indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class); + taskClientFactory = createMock(SeekableStreamIndexTaskClientFactory.class); + spec = createMock(SeekableStreamSupervisorSpec.class); + indexTaskClient = createMock(SeekableStreamIndexTaskClient.class); + recordSupplier = (RecordSupplier) createMock(RecordSupplier.class); + + rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); + + supervisorConfig = new SeekableStreamSupervisorConfig(); + + expect(spec.getSupervisorConfig()).andReturn(supervisorConfig).anyTimes(); + + expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); + expect(spec.getIoConfig()).andReturn(getIOConfig()).anyTimes(); + expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); + + expect(taskClientFactory.build(anyObject(), anyString(), anyInt(), anyObject(), anyLong())).andReturn( + indexTaskClient).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null).anyTimes(); + expect(recordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard0Partition)).anyTimes(); + expect(recordSupplier.getLatestSequenceNumber(anyObject())).andReturn("10").anyTimes(); + } + + @Test + public void testRunning() throws Exception + { + expect(spec.isSuspended()).andReturn(false).anyTimes(); + expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskQueue.add(anyObject())).andReturn(true).anyTimes(); + + replayAll(); + + SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + + supervisor.start(); + + Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + verifyAll(); + } + + @Test + public void testConnectingToStreamFail() throws Exception + { + expect(spec.isSuspended()).andReturn(false).anyTimes(); + expect(recordSupplier.getPartitionIds(STREAM)).andThrow(new StreamException(new IllegalStateException(EXCEPTION_MSG))) + .anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskQueue.add(anyObject())).andReturn(true).anyTimes(); + + replayAll(); + + SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + + supervisor.start(); + + Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + + Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); + List exceptionEvents = supervisor.stateManager.getExceptionEvents(); + Assert.assertEquals(1, exceptionEvents.size()); + Assert.assertTrue(exceptionEvents.get(0).isStreamException()); + Assert.assertEquals(IllegalStateException.class.getName(), exceptionEvents.get(0).getExceptionClass()); + Assert.assertEquals( + String.format("%s: %s", IllegalStateException.class.getName(), EXCEPTION_MSG), + exceptionEvents.get(0).getMessage() + ); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + + Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(2, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + + Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + verifyAll(); + } + + @Test + public void testConnectingToStreamFailRecoveryFailRecovery() throws Exception + { + expect(spec.isSuspended()).andReturn(false).anyTimes(); + expect(recordSupplier.getPartitionIds(STREAM)).andThrow(new StreamException(new IllegalStateException())).times(3); + expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).times(3); + expect(recordSupplier.getPartitionIds(STREAM)).andThrow(new StreamException(new IllegalStateException())).times(3); + expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).times(3); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskQueue.add(anyObject())).andReturn(true).anyTimes(); + + replayAll(); + + SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + + supervisor.start(); + + Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); + supervisor.runInternal(); + Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); + supervisor.runInternal(); + Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); + supervisor.runInternal(); + Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + supervisor.runInternal(); + Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + supervisor.runInternal(); + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + supervisor.runInternal(); + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + supervisor.runInternal(); + Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, supervisor.stateManager.getSupervisorState()); + supervisor.runInternal(); + Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, supervisor.stateManager.getSupervisorState()); + supervisor.runInternal(); + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + verifyAll(); + } + + @Test + public void testDiscoveringInitialTasksFailRecoveryFail() throws Exception + { + expect(spec.isSuspended()).andReturn(false).anyTimes(); + expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); + expect(taskStorage.getActiveTasks()).andThrow(new IllegalStateException(EXCEPTION_MSG)).times(3); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).times(3); + expect(taskStorage.getActiveTasks()).andThrow(new IllegalStateException(EXCEPTION_MSG)).times(3); + expect(taskQueue.add(anyObject())).andReturn(true).anyTimes(); + + replayAll(); + + SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + supervisor.start(); + + supervisor.runInternal(); + Assert.assertEquals(State.DISCOVERING_INITIAL_TASKS, supervisor.stateManager.getSupervisorState()); + List exceptionEvents = supervisor.stateManager.getExceptionEvents(); + Assert.assertEquals(1, exceptionEvents.size()); + Assert.assertFalse(exceptionEvents.get(0).isStreamException()); + Assert.assertEquals(IllegalStateException.class.getName(), exceptionEvents.get(0).getExceptionClass()); + Assert.assertEquals(EXCEPTION_MSG, exceptionEvents.get(0).getMessage()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.DISCOVERING_INITIAL_TASKS, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(2, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + verifyAll(); + } + + @Test + public void testCreatingTasksFailRecoveryFail() throws Exception + { + expect(spec.isSuspended()).andReturn(false).anyTimes(); + expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskQueue.add(anyObject())).andThrow(new IllegalStateException(EXCEPTION_MSG)).times(3); + expect(taskQueue.add(anyObject())).andReturn(true).times(3); + expect(taskQueue.add(anyObject())).andThrow(new IllegalStateException(EXCEPTION_MSG)).times(3); + + replayAll(); + + SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + supervisor.start(); + + supervisor.runInternal(); + Assert.assertEquals(State.CREATING_TASKS, supervisor.stateManager.getSupervisorState()); + List exceptionEvents = supervisor.stateManager.getExceptionEvents(); + Assert.assertEquals(1, exceptionEvents.size()); + Assert.assertFalse(exceptionEvents.get(0).isStreamException()); + Assert.assertEquals(IllegalStateException.class.getName(), exceptionEvents.get(0).getExceptionClass()); + Assert.assertEquals(EXCEPTION_MSG, exceptionEvents.get(0).getMessage()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.CREATING_TASKS, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(2, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + verifyAll(); + } + + @Test + public void testSuspended() throws Exception + { + expect(spec.isSuspended()).andReturn(true).anyTimes(); + expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskQueue.add(anyObject())).andReturn(true).anyTimes(); + + replayAll(); + + SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + + supervisor.start(); + + Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + + Assert.assertEquals(State.SUSPENDED, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + + Assert.assertEquals(State.SUSPENDED, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + verifyAll(); + } + + @Test + public void testShuttingDown() throws Exception + { + expect(spec.isSuspended()).andReturn(false).anyTimes(); + expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskQueue.add(anyObject())).andReturn(true).anyTimes(); + + taskRunner.unregisterListener("testSupervisorId"); + indexTaskClient.close(); + + replayAll(); + + SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + + supervisor.start(); + + Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); + Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.runInternal(); + + Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); + Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); + + supervisor.stop(false); + + Assert.assertEquals(State.SHUTTING_DOWN, supervisor.stateManager.getSupervisorState()); + + verifyAll(); + } + + private static DataSchema getDataSchema() + { + List dimensions = new ArrayList<>(); + dimensions.add(StringDimensionSchema.create("dim1")); + dimensions.add(StringDimensionSchema.create("dim2")); + + return new DataSchema( + DATASOURCE, + objectMapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + dimensions, + null, + null + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ), + StandardCharsets.UTF_8.name() + ), + Map.class + ), + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ), + null, + objectMapper + ); + } + + private static SeekableStreamSupervisorIOConfig getIOConfig() + { + return new SeekableStreamSupervisorIOConfig( + "stream", + 1, + 1, + new Period("PT1H"), + new Period("P1D"), + new Period("PT30S"), + false, + new Period("PT30M"), + null, + null + ) + { + }; + } + + private static SeekableStreamSupervisorTuningConfig getTuningConfig() + { + return new SeekableStreamSupervisorTuningConfig() + { + @Override + public Integer getWorkerThreads() + { + return 1; + } + + @Override + public Integer getChatThreads() + { + return 1; + } + + @Override + public Long getChatRetries() + { + return 1L; + } + + @Override + public Duration getHttpTimeout() + { + return new Period("PT1M").toStandardDuration(); + } + + @Override + public Duration getShutdownTimeout() + { + return new Period("PT1S").toStandardDuration(); + } + + @Override + public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig() + { + return new SeekableStreamIndexTaskTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + { + @Override + public SeekableStreamIndexTaskTuningConfig withBasePersistDirectory(File dir) + { + return null; + } + + @Override + public String toString() + { + return null; + } + }; + } + }; + } + + private class TestSeekableStreamIndexTask extends SeekableStreamIndexTask + { + public TestSeekableStreamIndexTask( + String id, + @Nullable TaskResource taskResource, + DataSchema dataSchema, + SeekableStreamIndexTaskTuningConfig tuningConfig, + SeekableStreamIndexTaskIOConfig ioConfig, + @Nullable Map context, + @Nullable ChatHandlerProvider chatHandlerProvider, + AuthorizerMapper authorizerMapper, + RowIngestionMetersFactory rowIngestionMetersFactory, + @Nullable String groupId + ) + { + super( + id, + taskResource, + dataSchema, + tuningConfig, + ioConfig, + context, + chatHandlerProvider, + authorizerMapper, + rowIngestionMetersFactory, + groupId + ); + } + + @Override + protected SeekableStreamIndexTaskRunner createTaskRunner() + { + return null; + } + + @Override + protected RecordSupplier newTaskRecordSupplier() + { + return recordSupplier; + } + + @Override + public String getType() + { + return "test"; + } + } + + private class TestSeekableStreamSupervisor extends SeekableStreamSupervisor + { + private TestSeekableStreamSupervisor() + { + super( + "testSupervisorId", + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + spec, + rowIngestionMetersFactory, + false + ); + } + + @Override + protected String baseTaskName() + { + return "test"; + } + + @Override + protected void updateLatestSequenceFromStream( + RecordSupplier recordSupplier, Set> streamPartitions + ) + { + // do nothing + } + + @Override + protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( + int groupId, + Map startPartitions, + Map endPartitions, + String baseSequenceName, + DateTime minimumMessageTime, + DateTime maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions, + SeekableStreamSupervisorIOConfig ioConfig + ) + { + return new SeekableStreamIndexTaskIOConfig( + groupId, + baseSequenceName, + new SeekableStreamStartSequenceNumbers<>(STREAM, startPartitions, exclusiveStartSequenceNumberPartitions), + new SeekableStreamEndSequenceNumbers<>(STREAM, endPartitions), + true, + minimumMessageTime, + maximumMessageTime + ) + { + }; + } + + @Override + protected List> createIndexTasks( + int replicas, + String baseSequenceName, + ObjectMapper sortingMapper, + TreeMap> sequenceOffsets, + SeekableStreamIndexTaskIOConfig taskIoConfig, + SeekableStreamIndexTaskTuningConfig taskTuningConfig, + RowIngestionMetersFactory rowIngestionMetersFactory + ) + { + return ImmutableList.of(new TestSeekableStreamIndexTask( + "id", + null, + getDataSchema(), + taskTuningConfig, + taskIoConfig, + null, + null, + null, + rowIngestionMetersFactory, + null + )); + } + + @Override + protected int getTaskGroupIdForPartition(String partition) + { + return 0; + } + + @Override + protected boolean checkSourceMetadataMatch(DataSourceMetadata metadata) + { + return true; + } + + @Override + protected boolean doesTaskTypeMatchSupervisor(Task task) + { + return true; + } + + @Override + protected SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( + String stream, + Map map + ) + { + return null; + } + + @Override + protected OrderedSequenceNumber makeSequenceNumber(String seq, boolean isExclusive) + { + return new OrderedSequenceNumber(seq, isExclusive) + { + @Override + public int compareTo(OrderedSequenceNumber o) + { + return new BigInteger(this.get()).compareTo(new BigInteger(o.get())); + } + }; + } + + @Override + protected void scheduleReporting(ScheduledExecutorService reportingExec) + { + // do nothing + } + + @Override + protected Map getLagPerPartition(Map currentOffsets) + { + return null; + } + + @Override + protected RecordSupplier setupRecordSupplier() + { + return recordSupplier; + } + + @Override + protected SeekableStreamSupervisorReportPayload createReportPayload( + int numPartitions, + boolean includeOffsets + ) + { + return new SeekableStreamSupervisorReportPayload( + DATASOURCE, + STREAM, + 1, + 1, + 1L, + null, + null, + null, + null, + false, + null, + null + ) + { + }; + } + + @Override + protected String getNotSetMarker() + { + return "NOT_SET"; + } + + @Override + protected String getEndOfPartitionMarker() + { + return "EOF"; + } + + @Override + protected boolean isEndOfShard(String seqNum) + { + return false; + } + + @Override + protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() + { + return false; + } + } +} From aab44c11650d2a8a03d7034c09ae143382441bba Mon Sep 17 00:00:00 2001 From: dclim Date: Mon, 6 May 2019 12:31:50 -0600 Subject: [PATCH 14/22] fix tests --- .../kafka/supervisor/KafkaSupervisorTest.java | 26 ++++++++--- .../supervisor/KinesisSupervisorTest.java | 45 ++++++++++--------- .../supervisor/SeekableStreamSupervisor.java | 19 ++++---- .../SeekableStreamSupervisorStateTest.java | 3 +- 4 files changed, 59 insertions(+), 34 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 16c22cb19e66..e93556a9b9ef 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -146,7 +146,7 @@ public class KafkaSupervisorTest extends EasyMockSupport private final int numThreads; - private KafkaSupervisor supervisor; + private TestableKafkaSupervisor supervisor; private KafkaSupervisorTuningConfig tuningConfig; private TaskStorage taskStorage; private TaskMaster taskMaster; @@ -620,7 +620,7 @@ public void testDatasourceMetadata() throws Exception ); } - @Test(expected = ISE.class) + @Test public void testBadMetadataOffsets() throws Exception { supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); @@ -641,6 +641,11 @@ public void testBadMetadataOffsets() throws Exception supervisor.start(); supervisor.runInternal(); + + Assert.assertEquals( + "org.apache.druid.java.util.common.ISE", + supervisor.getStateManager().getExceptionEvents().get(0).getExceptionClass() + ); } @Test @@ -2117,6 +2122,8 @@ public void testNoDataIngestionTasks() throws Exception { final DateTime startTime = DateTimes.nowUtc(); supervisor = getTestableSupervisor(2, 1, true, "PT1S", null, null); + supervisor.getStateManager().markRunFinished(); + //not adding any events Task id1 = createKafkaIndexTask( "id1", @@ -2213,6 +2220,8 @@ public void testCheckpointForInactiveTaskGroup() throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException { supervisor = getTestableSupervisor(2, 1, true, "PT1S", null, null); + supervisor.getStateManager().markRunFinished(); + //not adding any events final Task id1 = createKafkaIndexTask( "id1", @@ -2430,6 +2439,8 @@ public void testCheckpointWithNullTaskGroupId() throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException { supervisor = getTestableSupervisor(1, 3, true, "PT1S", null, null); + supervisor.getStateManager().markRunFinished(); + //not adding any events final Task id1 = createKafkaIndexTask( "id1", @@ -3136,7 +3147,7 @@ private void addSomeEvents(int numEventsPerPartition) throws Exception } } - private KafkaSupervisor getTestableSupervisor( + private TestableKafkaSupervisor getTestableSupervisor( int replicas, int taskCount, boolean useEarliestOffset, @@ -3157,7 +3168,7 @@ private KafkaSupervisor getTestableSupervisor( ); } - private KafkaSupervisor getTestableSupervisor( + private TestableKafkaSupervisor getTestableSupervisor( int replicas, int taskCount, boolean useEarliestOffset, @@ -3236,7 +3247,7 @@ public KafkaIndexTaskClient build( /** * Use when you want to mock the return value of SeekableStreamSupervisor#isTaskCurrent() */ - private KafkaSupervisor getTestableSupervisorCustomIsTaskCurrent( + private TestableKafkaSupervisor getTestableSupervisorCustomIsTaskCurrent( int replicas, int taskCount, boolean useEarliestOffset, @@ -3578,6 +3589,11 @@ protected String generateSequenceName( final int groupId = getTaskGroupIdForPartition(startPartitions.keySet().iterator().next()); return StringUtils.format("sequenceName-%d", groupId); } + + private SeekableStreamSupervisorStateManager getStateManager() + { + return stateManager; + } } private static class TestableKafkaSupervisorWithCustomIsTaskCurrent extends TestableKafkaSupervisor diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index c53b70bf8047..c914902c8b91 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.kinesis.supervisor; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; @@ -104,9 +103,7 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; -import java.util.concurrent.TimeoutException; import static org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import static org.easymock.EasyMock.anyObject; @@ -132,7 +129,7 @@ public class KinesisSupervisorTest extends EasyMockSupport private KinesisRecordSupplier supervisorRecordSupplier; private final int numThreads; - private KinesisSupervisor supervisor; + private TestableKinesisSupervisor supervisor; private KinesisSupervisorTuningConfig tuningConfig; private TaskStorage taskStorage; private TaskMaster taskMaster; @@ -561,7 +558,7 @@ public void testDatasourceMetadata() throws Exception ); } - @Test(expected = ISE.class) + @Test public void testBadMetadataOffsets() throws Exception { supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); @@ -594,6 +591,11 @@ public void testBadMetadataOffsets() throws Exception supervisor.start(); supervisor.runInternal(); + + Assert.assertEquals( + "org.apache.druid.java.util.common.ISE", + supervisor.getStateManager().getExceptionEvents().get(0).getExceptionClass() + ); } @Test @@ -2257,7 +2259,7 @@ public void testStopGracefully() throws Exception } @Test - public void testResetNoTasks() throws Exception + public void testResetNoTasks() { expect(supervisorRecordSupplier.getPartitionIds(anyObject())).andReturn(Collections.emptySet()).anyTimes(); @@ -2362,7 +2364,7 @@ public void testResetDataSourceMetadata() throws Exception } @Test - public void testResetNoDataSourceMetadata() throws Exception + public void testResetNoDataSourceMetadata() { expect(supervisorRecordSupplier.getPartitionIds(anyObject())).andReturn(Collections.emptySet()).anyTimes(); supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); @@ -2698,8 +2700,7 @@ public void testNoDataIngestionTasks() throws Exception @Test(timeout = 60_000L) - public void testCheckpointForInactiveTaskGroup() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException + public void testCheckpointForInactiveTaskGroup() throws InterruptedException { supervisor = getTestableSupervisor(2, 1, true, "PT1S", null, null, false); //not adding any events @@ -2997,10 +2998,11 @@ public void testCheckpointForUnknownTaskGroup() } @Test(timeout = 60_000L) - public void testCheckpointWithNullTaskGroupId() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException + public void testCheckpointWithNullTaskGroupId() throws InterruptedException { supervisor = getTestableSupervisor(1, 3, true, "PT1S", null, null, false); + supervisor.getStateManager().markRunFinished(); + //not adding any events final Task id1 = createKinesisIndexTask( "id1", @@ -3304,7 +3306,7 @@ public void testSuspendedRunningTasks() throws Exception } @Test - public void testResetSuspended() throws Exception + public void testResetSuspended() { expect(supervisorRecordSupplier.getPartitionIds(anyObject())).andReturn(Collections.emptySet()).anyTimes(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -3373,8 +3375,7 @@ public void testGetCurrentTotalStats() } @Test - public void testDoNotKillCompatibleTasks() - throws InterruptedException, EntryExistsException, ExecutionException, TimeoutException, JsonProcessingException + public void testDoNotKillCompatibleTasks() throws InterruptedException, EntryExistsException { // This supervisor always returns true for isTaskCurrent -> it should not kill its tasks int numReplicas = 2; @@ -3471,8 +3472,7 @@ public void testDoNotKillCompatibleTasks() } @Test - public void testKillIncompatibleTasks() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException, EntryExistsException + public void testKillIncompatibleTasks() throws InterruptedException, EntryExistsException { // This supervisor always returns false for isTaskCurrent -> it should kill its tasks int numReplicas = 2; @@ -3709,7 +3709,7 @@ public void testIsTaskCurrent() verifyAll(); } - private KinesisSupervisor getTestableSupervisor( + private TestableKinesisSupervisor getTestableSupervisor( int replicas, int taskCount, boolean useEarliestOffset, @@ -3787,7 +3787,7 @@ public KinesisIndexTaskClient build( ); } - private KinesisSupervisor getTestableSupervisor( + private TestableKinesisSupervisor getTestableSupervisor( int replicas, int taskCount, boolean useEarliestOffset, @@ -3809,7 +3809,7 @@ private KinesisSupervisor getTestableSupervisor( ); } - private KinesisSupervisor getTestableSupervisor( + private TestableKinesisSupervisor getTestableSupervisor( int replicas, int taskCount, boolean useEarliestOffset, @@ -3892,7 +3892,7 @@ public KinesisIndexTaskClient build( /** * Use when you want to mock the return value of SeekableStreamSupervisor#isTaskCurrent() */ - private KinesisSupervisor getTestableSupervisorCustomIsTaskCurrent( + private TestableKinesisSupervisor getTestableSupervisorCustomIsTaskCurrent( int replicas, int taskCount, boolean useEarliestOffset, @@ -4257,6 +4257,11 @@ protected RecordSupplier setupRecordSupplier() { return supervisorRecordSupplier; } + + private SeekableStreamSupervisorStateManager getStateManager() + { + return stateManager; + } } private class TestableKinesisSupervisorWithCustomIsTaskCurrent extends TestableKinesisSupervisor diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 44cd0d56d242..2a8d670eaff1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -64,6 +64,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamException; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; @@ -280,7 +281,7 @@ public Map getStats() private class RunNotice implements Notice { @Override - public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + public void handle() { long nowTime = System.currentTimeMillis(); if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { @@ -1029,7 +1030,7 @@ public void addTaskGroupToPendingCompletionTaskGroup( } @VisibleForTesting - public void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + public void runInternal() { try { possiblyRegisterListener(); @@ -1831,7 +1832,9 @@ private boolean updatePartitionDataFromStream() } if (partitionIds == null || partitionIds.size() == 0) { - log.warn("No partitions found for stream[%s]", ioConfig.getStream()); + String errMsg = StringUtils.format("No partitions found for stream [%s]", ioConfig.getStream()); + stateManager.storeThrowableEvent(new StreamException(new ISE(errMsg))); + log.warn(errMsg); return false; } @@ -2506,18 +2509,18 @@ private OrderedSequenceNumber getOffsetFromStorageForPartiti resetInternal( createDataSourceMetaDataForReset(ioConfig.getStream(), ImmutableMap.of(partition, sequence)) ); - throw new ISE( + throw new StreamException(new ISE( "Previous sequenceNumber [%s] is no longer available for partition [%s] - automatically resetting sequence", sequence, partition - ); + )); } else { - throw new ISE( + throw new StreamException(new ISE( "Previous sequenceNumber [%s] is no longer available for partition [%s]. You can clear the previous sequenceNumber and start reading from a valid message by using the supervisor's reset API.", sequence, partition - ); + )); } } } @@ -2690,7 +2693,7 @@ private void updateLatestOffsetsFromStream() throws InterruptedException } catch (Exception e) { log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getStream()); - throw new RuntimeException(e); + throw new StreamException(e); } Set> partitions = partitionIds diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index 3a45e23211ce..04d387daded7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -55,6 +55,7 @@ import org.apache.druid.indexing.seekablestream.common.StreamException; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.State; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -207,7 +208,7 @@ public void testConnectingToStreamFail() throws Exception Assert.assertTrue(exceptionEvents.get(0).isStreamException()); Assert.assertEquals(IllegalStateException.class.getName(), exceptionEvents.get(0).getExceptionClass()); Assert.assertEquals( - String.format("%s: %s", IllegalStateException.class.getName(), EXCEPTION_MSG), + StringUtils.format("%s: %s", IllegalStateException.class.getName(), EXCEPTION_MSG), exceptionEvents.get(0).getMessage() ); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); From 34b08616c8fd2557f1452a7e41ac116a162d59a8 Mon Sep 17 00:00:00 2001 From: dclim Date: Mon, 6 May 2019 19:01:44 -0600 Subject: [PATCH 15/22] code review changes and add HTTP health check status --- .../extensions-core/kafka-ingestion.md | 10 ++--- .../extensions-core/kinesis-ingestion.md | 10 ++--- .../indexing/kafka/KafkaRecordSupplier.java | 8 ++-- .../kafka/supervisor/KafkaSupervisor.java | 1 + .../KafkaSupervisorReportPayload.java | 5 ++- .../kinesis/supervisor/KinesisSupervisor.java | 1 + .../KinesisSupervisorReportPayload.java | 5 ++- .../supervisor/SupervisorResource.java | 11 ++++- .../supervisor/SeekableStreamSupervisor.java | 38 ++++++++-------- .../SeekableStreamSupervisorConfig.java | 8 ++-- ...SeekableStreamSupervisorReportPayload.java | 16 ++++++- .../SeekableStreamSupervisorStateManager.java | 11 +++-- ...kableStreamSupervisorStateManagerTest.java | 26 +++++------ .../SeekableStreamSupervisorStateTest.java | 43 ++++++++++++++++++- ...ealthCheckableSupervisorReportPayload.java | 25 +++++++++++ 15 files changed, 156 insertions(+), 62 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/indexing/overlord/supervisor/HealthCheckableSupervisorReportPayload.java diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 18929d92ecad..d0cbf5ba60d0 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -230,7 +230,7 @@ follows: |CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state| |RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse| |SUSPENDED|The supervisor has been suspended| -|SHUTTING_DOWN|Shutdown has been called but the supervisor hasn’t fully shutdown yet| +|STOPPING|The supervisor is stopping| States marked with "first iteration only" only occur on the supervisor's first iteration at startup or after suspension. @@ -369,9 +369,9 @@ Hadoop (see [here](https://github.com/apache/incubator-druid/pull/5102)). |Property|Description|Default| |--------|-----------|-------| -|druid.supervisor.stream.healthinessThreshold|The number of successful iterations before the supervisor flips from an UNHEALTHY to a RUNNING state|3| -|druid.supervisor.stream.unhealthinessThreshold|The number of iterations failed before the supervisor flips from a RUNNING to an UNHEALTHY state|3| -|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state|3| -|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state|3| +|druid.supervisor.stream.healthinessThreshold|The number of successful runs before an unhealthy supervisor is again considered healthy|3| +|druid.supervisor.stream.unhealthinessThreshold|The number of failed runs before the supervisor is considered unhealthy|3| +|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before an unhealthy supervisor is again considered healthy|3| +|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor is considered unhealthy|3| |druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|false| |druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|`max(healthinessThreshold, unhealthinessThreshold)`| diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md index 98c463544ee7..904f89c7cfbe 100644 --- a/docs/content/development/extensions-core/kinesis-ingestion.md +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -244,7 +244,7 @@ follows: |CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state| |RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse| |SUSPENDED|The supervisor has been suspended| -|SHUTTING_DOWN|Shutdown has been called but the supervisor hasn’t fully shutdown yet| +|STOPPING|The supervisor is stopping| States marked with "first iteration only" only occur on the supervisor's first iteration at startup or after suspension. @@ -419,9 +419,9 @@ Then when submitting a supervisor-spec, set `deaggregate` to true. |Property|Description|Default| |--------|-----------|-------| -|druid.supervisor.stream.healthinessThreshold|The number of successful iterations before the supervisor flips from an UNHEALTHY to a RUNNING state|3| -|druid.supervisor.stream.unhealthinessThreshold|The number of iterations failed before the supervisor flips from a RUNNING to an UNHEALTHY state|3| -|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before the supervisor flips from an UNHEALTHY_TASKS to a RUNNING state|3| -|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor flips from a RUNNING to an UNHEALTHY_TASKS state|3| +|druid.supervisor.stream.healthinessThreshold|The number of successful runs before an unhealthy supervisor is again considered healthy|3| +|druid.supervisor.stream.unhealthinessThreshold|The number of failed runs before the supervisor is considered unhealthy|3| +|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before an unhealthy supervisor is again considered healthy|3| +|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor is considered unhealthy|3| |druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|false| |druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|`max(healthinessThreshold, unhealthinessThreshold)`| diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 3329a286a7ec..e40f77a11523 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -156,13 +156,11 @@ public Long getPosition(StreamPartition partition) public Set getPartitionIds(String stream) { return wrapExceptions(() -> { - // use consumer.listTopics() instead of partitionsFor() to force a remote call so we can detect stream issues - Map> topics = consumer.listTopics(); - if (topics == null || topics.get(stream) == null) { + List partitions = consumer.partitionsFor(stream); + if (partitions == null) { throw new ISE("Topic [%s] is not found in KafkaConsumer's list of topics", stream); } - - return topics.get(stream).stream().map(PartitionInfo::partition).collect(Collectors.toSet()); + return partitions.stream().map(PartitionInfo::partition).collect(Collectors.toSet()); }); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 33af1c87539a..a8deb58e835d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -192,6 +192,7 @@ protected SeekableStreamSupervisorReportPayload createReportPaylo includeOffsets ? partitionLag.values().stream().mapToLong(x -> Math.max(x, 0)).sum() : null, includeOffsets ? sequenceLastUpdated : null, spec.isSuspended(), + stateManager.isHealthy(), stateManager.getSupervisorState(), stateManager.getExceptionEvents() ); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index 49a2c9457083..ae8fd12a56b2 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -40,6 +40,7 @@ public KafkaSupervisorReportPayload( @Nullable Long aggregateLag, @Nullable DateTime offsetsLastUpdated, boolean suspended, + boolean healthy, SeekableStreamSupervisorStateManager.State state, List recentErrors ) @@ -55,6 +56,7 @@ public KafkaSupervisorReportPayload( aggregateLag, offsetsLastUpdated, suspended, + healthy, state, recentErrors ); @@ -75,7 +77,8 @@ public String toString() (getMinimumLag() != null ? ", minimumLag=" + getMinimumLag() : "") + (getAggregateLag() != null ? ", aggregateLag=" + getAggregateLag() : "") + (getOffsetsLastUpdated() != null ? ", sequenceLastUpdated=" + getOffsetsLastUpdated() : "") + - ", suspended=" + getSuspended() + + ", suspended=" + isSuspended() + + ", healthy=" + isHealthy() + ", state=" + getState() + ", recentErrors=" + getRecentErrors() + '}'; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index d8b7aee1f719..64612a29347d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -248,6 +248,7 @@ protected SeekableStreamSupervisorReportPayload createReportPayl ioConfig.getReplicas(), ioConfig.getTaskDuration().getMillis() / 1000, spec.isSuspended(), + stateManager.isHealthy(), stateManager.getSupervisorState(), stateManager.getExceptionEvents() ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java index 6a4a4236ba0a..0f135cd06bda 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java @@ -34,6 +34,7 @@ public KinesisSupervisorReportPayload( Integer replicas, Long durationSeconds, boolean suspended, + boolean healthy, SeekableStreamSupervisorStateManager.State state, List recentErrors ) @@ -49,6 +50,7 @@ public KinesisSupervisorReportPayload( null, null, suspended, + healthy, state, recentErrors ); @@ -65,7 +67,8 @@ public String toString() ", durationSeconds=" + getDurationSeconds() + ", active=" + getActiveTasks() + ", publishing=" + getPublishingTasks() + - ", suspended=" + getSuspended() + + ", suspended=" + isSuspended() + + ", healthy=" + isHealthy() + ", state=" + getState() + ", recentErrors=" + getRecentErrors() + '}'; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java index 97e0580376e8..f8ae4b292d8b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -181,7 +181,13 @@ public Response specGetStatus(@PathParam("id") final String id) .build(); } - return Response.ok(spec.get()).build(); + boolean healthyOrNotApplicable = !(spec.get().getPayload() instanceof HealthCheckableSupervisorReportPayload) + || ((HealthCheckableSupervisorReportPayload) spec.get() + .getPayload()).isHealthy(); + + return Response.status(healthyOrNotApplicable ? Response.Status.OK : Response.Status.SERVICE_UNAVAILABLE) + .entity(spec.get()) + .build(); } ); } @@ -311,7 +317,8 @@ public Response specGetAllHistory(@Context final HttpServletRequest req) @Produces(MediaType.APPLICATION_JSON) public Response specGetHistory( @Context final HttpServletRequest req, - @PathParam("id") final String id) + @PathParam("id") final String id + ) { return asLeaderWithSupervisorManager( manager -> { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 2a8d670eaff1..fdcb130642e9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -662,7 +662,7 @@ public void stop(boolean stopGracefully) Preconditions.checkState(lifecycleStarted, "lifecycle not started"); log.info("Beginning shutdown of [%s]", supervisorId); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.SHUTTING_DOWN); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.STOPPING); try { scheduledExec.shutdownNow(); // stop recurring executions @@ -710,7 +710,7 @@ public void stop(boolean stopGracefully) log.info("[%s] has stopped", supervisorId); } catch (Exception e) { - stateManager.storeThrowableEvent(e); + stateManager.recordThrowableEvent(e); log.makeAlert(e, "Exception stopping [%s]", supervisorId) .emit(); } @@ -756,7 +756,7 @@ public void tryInit() notice.handle(); } catch (Throwable e) { - stateManager.storeThrowableEvent(e); + stateManager.recordThrowableEvent(e); log.makeAlert(e, "SeekableStreamSupervisor[%s] failed to handle notice", dataSource) .addData("noticeClass", notice.getClass().getSimpleName()) .emit(); @@ -764,7 +764,7 @@ public void tryInit() } } catch (InterruptedException e) { - stateManager.storeThrowableEvent(e); + stateManager.recordThrowableEvent(e); log.info("SeekableStreamSupervisor[%s] interrupted, exiting", dataSource); } } @@ -788,7 +788,7 @@ public void tryInit() ); } catch (Exception e) { - stateManager.storeThrowableEvent(e); + stateManager.recordThrowableEvent(e); if (recordSupplier != null) { recordSupplier.close(); } @@ -1070,7 +1070,7 @@ public void runInternal() } } catch (Exception e) { - stateManager.storeThrowableEvent(e); + stateManager.recordThrowableEvent(e); log.warn(e, "Exception in supervisor run loop for dataSource [%s]", dataSource); } finally { @@ -1378,7 +1378,7 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) .get(futureTimeoutInSeconds, TimeUnit.SECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { - stateManager.storeThrowableEvent(e); + stateManager.recordThrowableEvent(e); log.warn(e, "Exception while stopping task"); } return false; @@ -1396,7 +1396,7 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) .get(futureTimeoutInSeconds, TimeUnit.SECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { - stateManager.storeThrowableEvent(e); + stateManager.recordThrowableEvent(e); log.warn(e, "Exception while stopping task"); } return false; @@ -1438,7 +1438,7 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) return true; } catch (Throwable t) { - stateManager.storeThrowableEvent(t); + stateManager.recordThrowableEvent(t); log.error(t, "Something bad while discovering task [%s]", taskId); return null; } @@ -1514,7 +1514,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) futures.get(i).get(); } catch (Exception e) { - stateManager.storeThrowableEvent(e); + stateManager.recordThrowableEvent(e); log.error(e, "Problem while getting checkpoints for task [%s], killing the task", taskId); killTask(taskId, "Exception[%s] while getting checkpoints", e.getClass()); taskGroup.tasks.remove(taskId); @@ -1825,7 +1825,7 @@ private boolean updatePartitionDataFromStream() } } catch (Exception e) { - stateManager.storeThrowableEvent(e); + stateManager.recordThrowableEvent(e); log.warn("Could not fetch partitions for topic/stream [%s]: %s", ioConfig.getStream(), e.getMessage()); log.debug(e, "full stack trace"); return false; @@ -1833,7 +1833,7 @@ private boolean updatePartitionDataFromStream() if (partitionIds == null || partitionIds.size() == 0) { String errMsg = StringUtils.format("No partitions found for stream [%s]", ioConfig.getStream()); - stateManager.storeThrowableEvent(new StreamException(new ISE(errMsg))); + stateManager.recordThrowableEvent(new StreamException(new ISE(errMsg))); log.warn(errMsg); return false; } @@ -2024,7 +2024,7 @@ private ListenableFuture> checkpointTas // This will cause us to create a new set of tasks next cycle that will start from the sequences 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) - stateManager.storeCompletedTaskState(TaskState.SUCCESS); + stateManager.recordCompletedTaskState(TaskState.SUCCESS); return Futures.transform( stopTasksInGroup(taskGroup, "task[%s] succeeded in the taskGroup", task.status.getId()), new Function>() @@ -2086,7 +2086,7 @@ public Map apply(List +public abstract class SeekableStreamSupervisorReportPayload implements + HealthCheckableSupervisorReportPayload { private final String dataSource; private final String stream; @@ -44,6 +46,7 @@ public abstract class SeekableStreamSupervisorReportPayload recentErrors; @@ -58,6 +61,7 @@ public SeekableStreamSupervisorReportPayload( @Nullable Long aggregateLag, @Nullable DateTime offsetsLastUpdated, boolean suspended, + boolean healthy, SeekableStreamSupervisorStateManager.State state, List recentErrors ) @@ -74,6 +78,7 @@ public SeekableStreamSupervisorReportPayload( this.aggregateLag = aggregateLag; this.offsetsLastUpdated = offsetsLastUpdated; this.suspended = suspended; + this.healthy = healthy; this.state = state; this.recentErrors = recentErrors; } @@ -114,11 +119,18 @@ public int getReplicas() } @JsonProperty - public boolean getSuspended() + public boolean isSuspended() { return suspended; } + @Override + @JsonProperty + public boolean isHealthy() + { + return healthy; + } + @JsonProperty public long getDurationSeconds() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java index c2d014976c76..0e0fcb4e4c86 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java @@ -51,7 +51,7 @@ public enum State CREATING_TASKS(true, true), RUNNING(true, false), SUSPENDED(true, false), - SHUTTING_DOWN(true, false); + STOPPING(true, false); private final boolean healthy; private final boolean firstRunOnly; @@ -145,7 +145,7 @@ public void maybeSetState(State proposedState) } } - public void storeThrowableEvent(Throwable t) + public void recordThrowableEvent(Throwable t) { recentEventsQueue.add(new ExceptionEvent(t, supervisorConfig.isStoringStackTraces())); @@ -156,7 +156,7 @@ public void storeThrowableEvent(Throwable t) currentRunSuccessful = false; } - public void storeCompletedTaskState(TaskState state) + public void recordCompletedTaskState(TaskState state) { if (state.isSuccess()) { consecutiveSuccessfulTasks++; @@ -193,6 +193,11 @@ public State getSupervisorState() return supervisorState; } + public boolean isHealthy() + { + return supervisorState != null && supervisorState.healthy; + } + public boolean isAtLeastOneSuccessfulRun() { return atLeastOneSuccessfulRun; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java index 732e774c3910..21a87c0fdb8f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java @@ -94,7 +94,7 @@ public void testStreamFailureLostContact() for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); - stateManager.storeThrowableEvent(new StreamException(new IllegalStateException("DOH!"))); + stateManager.recordThrowableEvent(new StreamException(new IllegalStateException("DOH!"))); stateManager.markRunFinished(); } Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, stateManager.getSupervisorState()); @@ -112,7 +112,7 @@ public void testStreamFailureUnableToConnect() stateManager.maybeSetState(State.CONNECTING_TO_STREAM); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(State.CONNECTING_TO_STREAM, stateManager.getSupervisorState()); - stateManager.storeThrowableEvent(new StreamException(new IllegalStateException("DOH!"))); + stateManager.recordThrowableEvent(new StreamException(new IllegalStateException("DOH!"))); stateManager.markRunFinished(); } Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, stateManager.getSupervisorState()); @@ -130,7 +130,7 @@ public void testNonStreamUnhealthiness() stateManager.maybeSetState(State.DISCOVERING_INITIAL_TASKS); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(State.DISCOVERING_INITIAL_TASKS, stateManager.getSupervisorState()); - stateManager.storeThrowableEvent(new NullPointerException("oof")); + stateManager.recordThrowableEvent(new NullPointerException("oof")); stateManager.markRunFinished(); } Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); @@ -148,7 +148,7 @@ public void testTransientUnhealthiness() stateManager.markRunFinished(); for (int j = 1; j < 3; j++) { for (int i = 0; i < config.getUnhealthinessThreshold() - 1; i++) { - stateManager.storeThrowableEvent(new NullPointerException("oof")); + stateManager.recordThrowableEvent(new NullPointerException("oof")); stateManager.markRunFinished(); Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); } @@ -165,7 +165,7 @@ public void testNonTransientTaskUnhealthiness() stateManager.markRunFinished(); for (int i = 0; i < config.getTaskUnhealthinessThreshold(); i++) { Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); - stateManager.storeCompletedTaskState(TaskState.FAILED); + stateManager.recordCompletedTaskState(TaskState.FAILED); stateManager.markRunFinished(); } Assert.assertEquals(State.UNHEALTHY_TASKS, stateManager.getSupervisorState()); @@ -179,8 +179,8 @@ public void testTransientTaskUnhealthiness() stateManager.markRunFinished(); for (int i = 0; i < config.getTaskUnhealthinessThreshold() + 3; i++) { Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); - stateManager.storeCompletedTaskState(TaskState.FAILED); - stateManager.storeCompletedTaskState(TaskState.SUCCESS); + stateManager.recordCompletedTaskState(TaskState.FAILED); + stateManager.recordCompletedTaskState(TaskState.SUCCESS); stateManager.markRunFinished(); } Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); @@ -193,7 +193,7 @@ public void testSupervisorRecoveryWithHealthinessThreshold() // Put into an unhealthy state for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { Assert.assertEquals(State.WAITING_TO_RUN, stateManager.getSupervisorState()); - stateManager.storeThrowableEvent(new Exception("Except the inevitable")); + stateManager.recordThrowableEvent(new Exception("Except the inevitable")); stateManager.markRunFinished(); } Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); @@ -220,7 +220,7 @@ public void testTaskRecoveryWithHealthinessThreshold() // Put into an unhealthy state for (int i = 0; i < config.getTaskUnhealthinessThreshold(); i++) { Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); - stateManager.storeCompletedTaskState(TaskState.FAILED); + stateManager.recordCompletedTaskState(TaskState.FAILED); stateManager.markRunFinished(); } Assert.assertEquals(State.UNHEALTHY_TASKS, stateManager.getSupervisorState()); @@ -228,7 +228,7 @@ public void testTaskRecoveryWithHealthinessThreshold() // Recover after config.healthinessThreshold successful task completions for (int i = 0; i < config.getTaskHealthinessThreshold(); i++) { Assert.assertEquals(State.UNHEALTHY_TASKS, stateManager.getSupervisorState()); - stateManager.storeCompletedTaskState(TaskState.SUCCESS); + stateManager.recordCompletedTaskState(TaskState.SUCCESS); stateManager.markRunFinished(); } Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); @@ -240,8 +240,8 @@ public void testTwoUnhealthyStates() stateManager.markRunFinished(); for (int i = 0; i < Math.max(config.getTaskUnhealthinessThreshold(), config.getUnhealthinessThreshold()); i++) { - stateManager.storeThrowableEvent(new NullPointerException("somebody goofed")); - stateManager.storeCompletedTaskState(TaskState.FAILED); + stateManager.recordThrowableEvent(new NullPointerException("somebody goofed")); + stateManager.recordCompletedTaskState(TaskState.FAILED); stateManager.markRunFinished(); } // UNHEALTHY_SUPERVISOR should take priority over UNHEALTHY_TASKS @@ -258,7 +258,7 @@ public void testGetThrowableEvents() new RuntimeException(new IllegalArgumentException("oof")) ); for (Exception exception : exceptions) { - stateManager.storeThrowableEvent(exception); + stateManager.recordThrowableEvent(exception); stateManager.markRunFinished(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index 04d387daded7..055638d05d8a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -162,18 +162,21 @@ public void testRunning() throws Exception supervisor.start(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); @@ -196,12 +199,14 @@ public void testConnectingToStreamFail() throws Exception supervisor.start(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); List exceptionEvents = supervisor.stateManager.getExceptionEvents(); Assert.assertEquals(1, exceptionEvents.size()); @@ -215,12 +220,14 @@ public void testConnectingToStreamFail() throws Exception supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(2, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); @@ -245,12 +252,16 @@ public void testConnectingToStreamFailRecoveryFailRecovery() throws Exception supervisor.start(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); @@ -272,10 +283,13 @@ public void testConnectingToStreamFailRecoveryFailRecovery() throws Exception Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, supervisor.stateManager.getSupervisorState()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, supervisor.stateManager.getSupervisorState()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); @@ -298,6 +312,7 @@ public void testDiscoveringInitialTasksFailRecoveryFail() throws Exception supervisor.start(); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.DISCOVERING_INITIAL_TASKS, supervisor.stateManager.getSupervisorState()); List exceptionEvents = supervisor.stateManager.getExceptionEvents(); Assert.assertEquals(1, exceptionEvents.size()); @@ -307,39 +322,47 @@ public void testDiscoveringInitialTasksFailRecoveryFail() throws Exception Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.DISCOVERING_INITIAL_TASKS, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(2, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); @@ -362,6 +385,7 @@ public void testCreatingTasksFailRecoveryFail() throws Exception supervisor.start(); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.CREATING_TASKS, supervisor.stateManager.getSupervisorState()); List exceptionEvents = supervisor.stateManager.getExceptionEvents(); Assert.assertEquals(1, exceptionEvents.size()); @@ -371,39 +395,47 @@ public void testCreatingTasksFailRecoveryFail() throws Exception Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.CREATING_TASKS, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(2, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertFalse(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); @@ -424,18 +456,21 @@ public void testSuspended() throws Exception supervisor.start(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.SUSPENDED, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.SUSPENDED, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); @@ -444,7 +479,7 @@ public void testSuspended() throws Exception } @Test - public void testShuttingDown() throws Exception + public void testStopping() throws Exception { expect(spec.isSuspended()).andReturn(false).anyTimes(); expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); @@ -460,19 +495,22 @@ public void testShuttingDown() throws Exception supervisor.start(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); + Assert.assertTrue(supervisor.stateManager.isHealthy()); Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.stop(false); - Assert.assertEquals(State.SHUTTING_DOWN, supervisor.stateManager.getSupervisorState()); + Assert.assertTrue(supervisor.stateManager.isHealthy()); + Assert.assertEquals(State.STOPPING, supervisor.stateManager.getSupervisorState()); verifyAll(); } @@ -808,6 +846,7 @@ protected SeekableStreamSupervisorReportPayload createReportPayl null, null, false, + true, null, null ) diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/HealthCheckableSupervisorReportPayload.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/HealthCheckableSupervisorReportPayload.java new file mode 100644 index 000000000000..054a9864240b --- /dev/null +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/HealthCheckableSupervisorReportPayload.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.overlord.supervisor; + +public interface HealthCheckableSupervisorReportPayload +{ + boolean isHealthy(); +} From 413af77adc6490909b4bdfa0fc69c7b1488009b1 Mon Sep 17 00:00:00 2001 From: dclim Date: Mon, 6 May 2019 20:01:29 -0600 Subject: [PATCH 16/22] fix test failure --- .../supervisor/SeekableStreamSupervisorStateTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index 055638d05d8a..ad5b8163fe93 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -488,6 +488,7 @@ public void testStopping() throws Exception taskRunner.unregisterListener("testSupervisorId"); indexTaskClient.close(); + recordSupplier.close(); replayAll(); From 58d087083992bbb8e1f991e6d56e014a0491069f Mon Sep 17 00:00:00 2001 From: dclim Date: Wed, 22 May 2019 19:58:25 -0600 Subject: [PATCH 17/22] refactor to split into a generic SupervisorStateManager and a specific SeekableStreamSupervisorStateManager --- docs/content/configuration/index.md | 11 + .../extensions-core/kafka-ingestion.md | 25 +- .../extensions-core/kinesis-ingestion.md | 19 +- .../MaterializedViewSupervisor.java | 63 ++-- .../MaterializedViewSupervisorReport.java | 33 +- .../MaterializedViewSupervisorSpec.java | 102 ++++--- .../MaterializedViewSupervisorSpecTest.java | 11 +- .../MaterializedViewSupervisorTest.java | 7 +- .../kafka/supervisor/KafkaSupervisor.java | 7 + .../KafkaSupervisorReportPayload.java | 9 +- .../kafka/supervisor/KafkaSupervisorSpec.java | 8 +- .../supervisor/KafkaSupervisorSpecTest.java | 4 +- .../kafka/supervisor/KafkaSupervisorTest.java | 15 +- .../kinesis/supervisor/KinesisSupervisor.java | 7 + .../KinesisSupervisorReportPayload.java | 9 +- .../supervisor/KinesisSupervisorSpec.java | 8 +- .../supervisor/KinesisSupervisorTest.java | 15 +- .../druid/indexing/overlord/TaskQueue.java | 2 +- .../supervisor/SupervisorManager.java | 6 + .../supervisor/SupervisorResource.java | 29 +- .../supervisor/SeekableStreamSupervisor.java | 16 +- ...SeekableStreamSupervisorReportPayload.java | 27 +- .../SeekableStreamSupervisorSpec.java | 11 +- .../SeekableStreamSupervisorStateManager.java | 225 +++----------- .../supervisor/SupervisorResourceTest.java | 64 +++- ...kableStreamSupervisorStateManagerTest.java | 131 ++++---- .../SeekableStreamSupervisorStateTest.java | 140 +++++---- ...ealthCheckableSupervisorReportPayload.java | 25 -- .../overlord/supervisor/Supervisor.java | 6 + .../overlord/supervisor/SupervisorModule.java | 8 +- .../supervisor/SupervisorStateManager.java | 288 ++++++++++++++++++ .../SupervisorStateManagerConfig.java | 27 +- .../org/apache/druid/cli/CliOverlord.java | 8 +- 33 files changed, 843 insertions(+), 523 deletions(-) delete mode 100644 server/src/main/java/org/apache/druid/indexing/overlord/supervisor/HealthCheckableSupervisorReportPayload.java rename indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java => server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorModule.java (76%) create mode 100644 server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorConfig.java => server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerConfig.java (85%) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 91edb823cd1e..00ccc0743f41 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -942,6 +942,17 @@ There are additional configs for autoscaling (if it is enabled): |`druid.indexer.autoscale.workerVersion`|If set, will only create nodes of set version during autoscaling. Overrides dynamic configuration. |null| |`druid.indexer.autoscale.workerPort`|The port that MiddleManagers will run on.|8080| +##### Supervisors + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.supervisor.healthinessThreshold`|The number of successful runs before an unhealthy supervisor is again considered healthy.|3| +|`druid.supervisor.unhealthinessThreshold`|The number of failed runs before the supervisor is considered unhealthy.|3| +|`druid.supervisor.taskHealthinessThreshold`|The number of consecutive task successes before an unhealthy supervisor is again considered healthy.|3| +|`druid.supervisor.taskUnhealthinessThreshold`|The number of consecutive task failures before the supervisor is considered unhealthy.|3| +|`druid.supervisor.storeStackTrace`|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint.|false| +|`druid.supervisor.maxStoredExceptionEvents`|The maximum number of exception events that can be returned through the supervisor `/status` endpoint.|`max(healthinessThreshold, unhealthinessThreshold)`| + #### Overlord Dynamic Configuration The Overlord can dynamically change worker behavior. diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index d0cbf5ba60d0..cd1c6bcbce9f 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -215,16 +215,16 @@ consumer lag per partition may be reported as negative values if the supervisor response from Kafka. The aggregate lag value will always be >= 0. The status report also contains the supervisor's state and a list of recently thrown exceptions (whose max size can be -controlled using the `druid.supervisor.stream.maxStoredExceptionEvents` config parameter). The list of states is as +controlled using the `druid.supervisor.maxStoredExceptionEvents` config parameter). The list of states is as follows: |State|Description| |-----|-----------| -|UNHEALTHY_SUPERVISOR|The supervisor has encountered errors on the past `druid.supervisor.stream.unhealthinessThreshold` iterations| -|UNHEALTHY_TASKS|The last `druid.supervisor.stream.taskUnhealthinessThreshold` tasks have all failed| +|UNHEALTHY_SUPERVISOR|The supervisor has encountered errors on the past `druid.supervisor.unhealthinessThreshold` iterations| +|UNHEALTHY_TASKS|The last `druid.supervisor.taskUnhealthinessThreshold` tasks have all failed| |UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kafka and has not successfully connected in the past| |LOST_CONTACT_WITH_STREAM|The supervisor is encountering connectivity issues with Kafka but has successfully connected in the past| -|WAITING_TO_RUN (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream| +|PENDING (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream| |CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data| |DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks| |CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state| @@ -240,6 +240,12 @@ States marked with "first iteration only" only occur on the supervisor's first i See [Task Reports: Row Stats](../../ingestion/reports.html#row-stats) for more information. +### Supervisor Health Check + +`GET /druid/indexer/v1/supervisor//health` returns `200 OK` if the supervisor is healthy and +`503 Service Unavailable` if it is unhealthy. Healthiness is determined by the supervisor's `state` (as returned by the +`/status` endpoint) and the `druid.supervisor.*` Overlord configuration thresholds. + ### Updating Existing Supervisors `POST /druid/indexer/v1/supervisor` can be used to update existing supervisor spec. @@ -364,14 +370,3 @@ one can schedule re-indexing tasks be run to merge segments together into new se Details on how to optimize the segment size can be found on [Segment size optimization](../../operations/segment-optimization.html). There is also ongoing work to support automatic segment compaction of sharded segments as well as compaction not requiring Hadoop (see [here](https://github.com/apache/incubator-druid/pull/5102)). - -## Configuration Properties - -|Property|Description|Default| -|--------|-----------|-------| -|druid.supervisor.stream.healthinessThreshold|The number of successful runs before an unhealthy supervisor is again considered healthy|3| -|druid.supervisor.stream.unhealthinessThreshold|The number of failed runs before the supervisor is considered unhealthy|3| -|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before an unhealthy supervisor is again considered healthy|3| -|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor is considered unhealthy|3| -|druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|false| -|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|`max(healthinessThreshold, unhealthinessThreshold)`| diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md index 904f89c7cfbe..8180ef9af367 100644 --- a/docs/content/development/extensions-core/kinesis-ingestion.md +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -229,16 +229,16 @@ managed by the given supervisor. This includes the latest sequence numbers as re Indexing Service, stats about lag are not yet supported. The status report also contains the supervisor's state and a list of recently thrown exceptions (whose max size can be -controlled using the `druid.supervisor.stream.maxStoredExceptionEvents` config parameter). The list of states is as +controlled using the `druid.supervisor.maxStoredExceptionEvents` config parameter). The list of states is as follows: |State|Description| |-----|-----------| -|UNHEALTHY_SUPERVISOR|The supervisor has encountered errors on the past `druid.supervisor.stream.unhealthinessThreshold` iterations| -|UNHEALTHY_TASKS|The last `druid.supervisor.stream.taskUnhealthinessThreshold` tasks have all failed| +|UNHEALTHY_SUPERVISOR|The supervisor has encountered errors on the past `druid.supervisor.unhealthinessThreshold` iterations| +|UNHEALTHY_TASKS|The last `druid.supervisor.taskUnhealthinessThreshold` tasks have all failed| |UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kinesis and has not successfully connected in the past| |LOST_CONTACT_WITH_STREAM|The supervisor is encountering connectivity issues with Kinesis but has successfully connected in the past| -|WAITING_TO_RUN (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream| +|PENDING (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream| |CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data| |DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks| |CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state| @@ -414,14 +414,3 @@ compatible with Apache projects. To enable this feature, add the `amazon-kinesis-client` (tested on version `1.9.2`) jar file ([link](https://mvnrepository.com/artifact/com.amazonaws/amazon-kinesis-client/1.9.2)) under `dist/druid/extensions/druid-kinesis-indexing-service/`. Then when submitting a supervisor-spec, set `deaggregate` to true. - -## Configuration Properties - -|Property|Description|Default| -|--------|-----------|-------| -|druid.supervisor.stream.healthinessThreshold|The number of successful runs before an unhealthy supervisor is again considered healthy|3| -|druid.supervisor.stream.unhealthinessThreshold|The number of failed runs before the supervisor is considered unhealthy|3| -|druid.supervisor.stream.taskHealthinessThreshold|The number of consecutive task successes before an unhealthy supervisor is again considered healthy|3| -|druid.supervisor.stream.taskUnhealthinessThreshold|The number of consecutive task failures before the supervisor is considered unhealthy|3| -|druid.supervisor.stream.storingStackTraces|Whether full stack traces of supervisor exceptions should be stored and returned by the supervisor `/status` endpoint|false| -|druid.supervisor.stream.maxStoredExceptionEvents|The maximum number of exception events that can be returned through the supervisor `/status` endpoint|`max(healthinessThreshold, unhealthinessThreshold)`| diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java index 105afdf8f23f..c1033ae647d4 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java @@ -35,6 +35,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; @@ -77,6 +78,7 @@ public class MaterializedViewSupervisor implements Supervisor private final TaskMaster taskMaster; private final TaskStorage taskStorage; private final MaterializedViewTaskConfig config; + private final SupervisorStateManager stateManager; private final String dataSource; private final String supervisorId; private final int maxTaskCount; @@ -93,7 +95,7 @@ public class MaterializedViewSupervisor implements Supervisor // In the missing intervals, baseDataSource has data but derivedDataSource does not, which means // data in these intervals of derivedDataSource needs to be rebuilt. private Set missInterval = new HashSet<>(); - + public MaterializedViewSupervisor( TaskMaster taskMaster, TaskStorage taskStorage, @@ -111,6 +113,7 @@ public MaterializedViewSupervisor( this.metadataSupervisorManager = metadataSupervisorManager; this.config = config; this.spec = spec; + this.stateManager = new SupervisorStateManager(spec.getSupervisorStateManagerConfig(), spec.isSuspended()); this.dataSource = spec.getDataSourceName(); this.supervisorId = StringUtils.format("MaterializedViewSupervisor-%s", dataSource); this.maxTaskCount = spec.getContext().containsKey("maxTaskCount") @@ -120,17 +123,17 @@ public MaterializedViewSupervisor( ? Long.parseLong(String.valueOf(spec.getContext().get("minDataLagMs"))) : DEFAULT_MIN_DATA_LAG_MS; } - + @Override - public void start() + public void start() { synchronized (stateLock) { Preconditions.checkState(!started, "already started"); - + DataSourceMetadata metadata = metadataStorageCoordinator.getDataSourceMetadata(dataSource); if (null == metadata) { metadataStorageCoordinator.insertDataSourceMetadata( - dataSource, + dataSource, new DerivativeDataSourceMetadata(spec.getBaseDataSource(), spec.getDimensions(), spec.getMetrics()) ); } @@ -175,15 +178,22 @@ public void run() } } catch (Exception e) { + stateManager.recordThrowableEvent(e); log.makeAlert(e, StringUtils.format("uncaught exception in %s.", supervisorId)).emit(); } + finally { + stateManager.markRunFinished(); + } } @Override - public void stop(boolean stopGracefully) + public void stop(boolean stopGracefully) { synchronized (stateLock) { Preconditions.checkState(started, "not started"); + + stateManager.maybeSetState(SupervisorStateManager.BasicState.STOPPING); + // stop all schedulers and threads if (stopGracefully) { synchronized (taskLock) { @@ -214,7 +224,7 @@ public void stop(boolean stopGracefully) } @Override - public SupervisorReport getStatus() + public SupervisorReport getStatus() { return new MaterializedViewSupervisorReport( dataSource, @@ -223,10 +233,19 @@ public SupervisorReport getStatus() spec.getBaseDataSource(), spec.getDimensions(), spec.getMetrics(), - JodaUtils.condenseIntervals(missInterval) + JodaUtils.condenseIntervals(missInterval), + stateManager.isHealthy(), + stateManager.getSupervisorState().getBasicState(), + stateManager.getExceptionEvents() ); } + @Override + public Boolean isHealthy() + { + return stateManager.isHealthy(); + } + @Override public void reset(DataSourceMetadata dataSourceMetadata) { @@ -293,7 +312,7 @@ void checkSegmentsAndSubmitTasks() submitTasks(sortedToBuildVersion, baseSegments); } } - + @VisibleForTesting Pair, Map> getRunningTasks() { @@ -311,7 +330,7 @@ Pair, Map> getRunningTasks() * * @return the left part of Pair: interval -> version, and the right part: interval -> DataSegment list. * Version and DataSegment list can be used to create HadoopIndexTask. - * Derived datasource data in all these intervals need to be rebuilt. + * Derived datasource data in all these intervals need to be rebuilt. */ @VisibleForTesting Pair, Map>> checkSegments() @@ -347,13 +366,13 @@ Pair, Map>> checkSegment // if some intervals are in running tasks and the versions are the same, remove it from toBuildInterval // if some intervals are in running tasks, but the versions are different, stop the task. for (Interval interval : runningVersion.keySet()) { - if (toBuildInterval.containsKey(interval) + if (toBuildInterval.containsKey(interval) && toBuildInterval.get(interval).equals(runningVersion.get(interval)) ) { toBuildInterval.remove(interval); } else if ( - toBuildInterval.containsKey(interval) + toBuildInterval.containsKey(interval) && !toBuildInterval.get(interval).equals(runningVersion.get(interval)) ) { if (taskMaster.getTaskQueue().isPresent()) { @@ -374,7 +393,7 @@ Pair, Map>> checkSegment } private void submitTasks( - SortedMap sortedToBuildVersion, + SortedMap sortedToBuildVersion, Map> baseSegments ) { @@ -397,7 +416,7 @@ private void submitTasks( } } } - + private Pair, Map>> getVersionAndBaseSegments( List snapshot ) @@ -412,7 +431,7 @@ private Pair, Map>> getVersion } return new Pair<>(versions, segments); } - + private Pair, Map>> getMaxCreateDateAndBaseSegments( List> snapshot ) @@ -432,9 +451,9 @@ private Pair, Map>> getMaxCrea continue; } maxCreatedDate.put( - interval, + interval, DateTimes.max( - DateTimes.of(createDate), + DateTimes.of(createDate), DateTimes.of(maxCreatedDate.getOrDefault(interval, DateTimes.MIN.toString())) ).toString() ); @@ -457,8 +476,8 @@ private boolean hasEnoughLag(Interval target, Interval maxInterval) { return minDataLagMs <= (maxInterval.getStartMillis() - target.getStartMillis()); } - - private void clearTasks() + + private void clearTasks() { for (HadoopIndexTask task : runningTasks.values()) { if (taskMaster.getTaskQueue().isPresent()) { @@ -468,7 +487,7 @@ private void clearTasks() runningTasks.clear(); runningVersion.clear(); } - + private void clearSegments() { log.info("Clear all metadata of dataSource %s", dataSource); @@ -476,7 +495,7 @@ private void clearSegments() segmentManager.removeDataSource(dataSource); metadataStorageCoordinator.deleteDataSourceMetadata(dataSource); } - + private void commitDataSourceMetadata(DataSourceMetadata dataSourceMetadata) { if (!metadataStorageCoordinator.insertDataSourceMetadata(dataSource, dataSourceMetadata)) { @@ -485,7 +504,7 @@ private void commitDataSourceMetadata(DataSourceMetadata dataSourceMetadata) dataSource, dataSourceMetadata ); - } + } catch (IOException e) { throw new RuntimeException(e); } diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorReport.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorReport.java index f05f8e0f6d1e..13e51dafd142 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorReport.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorReport.java @@ -19,17 +19,18 @@ package org.apache.druid.indexing.materializedview; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.List; import java.util.Set; -public class MaterializedViewSupervisorReport extends SupervisorReport +public class MaterializedViewSupervisorReport extends SupervisorReport { - public MaterializedViewSupervisorReport( String dataSource, DateTime generationTime, @@ -37,16 +38,26 @@ public MaterializedViewSupervisorReport( String baseDataSource, Set dimensions, Set metrics, - List missTimeline + List missTimeline, + boolean healthy, + SupervisorStateManager.State state, + List recentErrors ) { - super(dataSource, generationTime, "{" + - "dataSource='" + dataSource + '\'' + - ", baseDataSource='" + baseDataSource + '\'' + - ", suspended='" + suspended + "\'" + - ", dimensions=" + dimensions + - ", metrics=" + metrics + - ", missTimeline" + Sets.newHashSet(missTimeline) + - "}"); + super( + dataSource, + generationTime, + ImmutableMap.builder() + .put("dataSource", dataSource) + .put("baseDataSource", baseDataSource) + .put("suspended", suspended) + .put("dimensions", dimensions) + .put("metrics", metrics) + .put("missTimeline", Sets.newHashSet(missTimeline)) + .put("healthy", healthy) + .put("state", state) + .put("recentErrors", recentErrors) + .build() + ); } } diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java index 3a9a1137b319..5a92540d0ca1 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java @@ -38,6 +38,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -58,7 +59,7 @@ import java.util.Map; import java.util.Set; -public class MaterializedViewSupervisorSpec implements SupervisorSpec +public class MaterializedViewSupervisorSpec implements SupervisorSpec { private static final String TASK_PREFIX = "index_materialized_view"; private final String baseDataSource; @@ -81,8 +82,9 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec private final MaterializedViewTaskConfig config; private final AuthorizerMapper authorizerMapper; private final ChatHandlerProvider chatHandlerProvider; + private final SupervisorStateManagerConfig supervisorStateManagerConfig; private final boolean suspended; - + public MaterializedViewSupervisorSpec( @JsonProperty("baseDataSource") String baseDataSource, @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, @@ -102,31 +104,35 @@ public MaterializedViewSupervisorSpec( @JacksonInject IndexerMetadataStorageCoordinator metadataStorageCoordinator, @JacksonInject MaterializedViewTaskConfig config, @JacksonInject AuthorizerMapper authorizerMapper, - @JacksonInject ChatHandlerProvider chatHandlerProvider + @JacksonInject ChatHandlerProvider chatHandlerProvider, + @JacksonInject SupervisorStateManagerConfig supervisorStateManagerConfig ) { - Preconditions.checkArgument(!Strings.isNullOrEmpty(baseDataSource), "baseDataSource cannot be null or empty. Please provide a baseDataSource."); + Preconditions.checkArgument( + !Strings.isNullOrEmpty(baseDataSource), + "baseDataSource cannot be null or empty. Please provide a baseDataSource." + ); this.baseDataSource = baseDataSource; this.dimensionsSpec = Preconditions.checkNotNull( - dimensionsSpec, - "dimensionsSpec cannot be null. Please provide a dimensionsSpec" - ); + dimensionsSpec, + "dimensionsSpec cannot be null. Please provide a dimensionsSpec" + ); this.aggregators = Preconditions.checkNotNull( - aggregators, - "metricsSpec cannot be null. Please provide a metricsSpec" - ); + aggregators, + "metricsSpec cannot be null. Please provide a metricsSpec" + ); this.tuningConfig = Preconditions.checkNotNull( - tuningConfig, - "tuningConfig cannot be null. Please provide tuningConfig" - ); - - this.dataSourceName = dataSourceName == null + tuningConfig, + "tuningConfig cannot be null. Please provide tuningConfig" + ); + + this.dataSourceName = dataSourceName == null ? StringUtils.format( - "%s-%s", - baseDataSource, - DigestUtils.sha1Hex(dimensionsSpec.toString()).substring(0, 8) - ) + "%s-%s", + baseDataSource, + DigestUtils.sha1Hex(dimensionsSpec.toString()).substring(0, 8) + ) : dataSourceName; this.hadoopCoordinates = hadoopCoordinates; this.hadoopDependencyCoordinates = hadoopDependencyCoordinates; @@ -141,6 +147,7 @@ public MaterializedViewSupervisorSpec( this.authorizerMapper = authorizerMapper; this.chatHandlerProvider = chatHandlerProvider; this.config = config; + this.supervisorStateManagerConfig = supervisorStateManagerConfig; this.suspended = suspended != null ? suspended : false; this.metrics = new HashSet<>(); @@ -152,11 +159,11 @@ public MaterializedViewSupervisorSpec( dimensions.add(schema.getName()); } } - + public HadoopIndexTask createTask(Interval interval, String version, List segments) { String taskId = StringUtils.format("%s_%s_%s", TASK_PREFIX, dataSourceName, DateTimes.nowUtc()); - + // generate parser Map parseSpec = new HashMap<>(); parseSpec.put("format", "timeAndDims"); @@ -164,7 +171,7 @@ public HadoopIndexTask createTask(Interval interval, String version, List parser = new HashMap<>(); parser.put("type", "map"); parser.put("parseSpec", parseSpec); - + //generate HadoopTuningConfig HadoopTuningConfig tuningConfigForTask = new HadoopTuningConfig( tuningConfig.getWorkingPath(), @@ -190,7 +197,7 @@ public HadoopIndexTask createTask(Interval interval, String version, List getDimensions() { return dimensions; } - + public Set getMetrics() { return metrics; } - + @JsonProperty("baseDataSource") public String getBaseDataSource() { return baseDataSource; } - + @JsonProperty("dimensionsSpec") public DimensionsSpec getDimensionsSpec() { return dimensionsSpec; } - + @JsonProperty("metricsSpec") public AggregatorFactory[] getMetricsSpec() { @@ -278,33 +285,33 @@ public HadoopTuningConfig getTuningConfig() { return tuningConfig; } - + @JsonProperty("dataSource") public String getDataSourceName() { return dataSourceName; } - + @JsonProperty("hadoopCoordinates") public String getHadoopCoordinates() { return hadoopCoordinates; } - + @JsonProperty("hadoopDependencyCoordinates") public List getSadoopDependencyCoordinates() { return hadoopDependencyCoordinates; } - + @JsonProperty("classpathPrefix") public String getClasspathPrefix() { return classpathPrefix; } - + @JsonProperty("context") - public Map getContext() + public Map getContext() { return context; } @@ -317,13 +324,13 @@ public boolean isSuspended() } @Override - public String getId() + public String getId() { return StringUtils.format("MaterializedViewSupervisor-%s", dataSourceName); } @Override - public Supervisor createSupervisor() + public Supervisor createSupervisor() { return new MaterializedViewSupervisor( taskMaster, @@ -364,7 +371,8 @@ public SupervisorSpec createSuspendedSpec() metadataStorageCoordinator, config, authorizerMapper, - chatHandlerProvider + chatHandlerProvider, + supervisorStateManagerConfig ); } @@ -390,17 +398,23 @@ public SupervisorSpec createRunningSpec() metadataStorageCoordinator, config, authorizerMapper, - chatHandlerProvider + chatHandlerProvider, + supervisorStateManagerConfig ); } + public SupervisorStateManagerConfig getSupervisorStateManagerConfig() + { + return supervisorStateManagerConfig; + } + @Override public String toString() { return "MaterializedViewSupervisorSpec{" + - "baseDataSource=" + baseDataSource + - ", dimensions=" + dimensions + - ", metrics=" + metrics + - '}'; + "baseDataSource=" + baseDataSource + + ", dimensions=" + dimensions + + ", metrics=" + metrics + + '}'; } } diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java index 3e6e1a6199fc..46728e2418cb 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java @@ -29,6 +29,7 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.MetadataSupervisorManager; import org.apache.druid.metadata.SQLMetadataSegmentManager; @@ -74,6 +75,7 @@ public void setup() .addValue(MaterializedViewTaskConfig.class, new MaterializedViewTaskConfig()) .addValue(AuthorizerMapper.class, createMock(AuthorizerMapper.class)) .addValue(ChatHandlerProvider.class, new NoopChatHandlerProvider()) + .addValue(SupervisorStateManagerConfig.class, new SupervisorStateManagerConfig()) ); } @@ -143,7 +145,8 @@ public void testSupervisorSerialization() throws IOException null, new MaterializedViewTaskConfig(), createMock(AuthorizerMapper.class), - new NoopChatHandlerProvider() + new NoopChatHandlerProvider(), + new SupervisorStateManagerConfig() ); MaterializedViewSupervisorSpec spec = objectMapper.readValue(supervisorStr, MaterializedViewSupervisorSpec.class); Assert.assertEquals(expected.getBaseDataSource(), spec.getBaseDataSource()); @@ -239,7 +242,8 @@ public void testEmptyBaseDataSource() null, new MaterializedViewTaskConfig(), createMock(AuthorizerMapper.class), - new NoopChatHandlerProvider() + new NoopChatHandlerProvider(), + new SupervisorStateManagerConfig() ); } @@ -284,7 +288,8 @@ public void testNullBaseDataSource() null, new MaterializedViewTaskConfig(), createMock(AuthorizerMapper.class), - new NoopChatHandlerProvider() + new NoopChatHandlerProvider(), + new SupervisorStateManagerConfig() ); } } diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index 1bf1c39709d5..3b9061de60cd 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -36,6 +36,7 @@ import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueue; import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; @@ -125,7 +126,8 @@ public void setUp() indexerMetadataStorageCoordinator, new MaterializedViewTaskConfig(), createMock(AuthorizerMapper.class), - createMock(ChatHandlerProvider.class) + createMock(ChatHandlerProvider.class), + new SupervisorStateManagerConfig() ); supervisor = (MaterializedViewSupervisor) spec.createSupervisor(); } @@ -282,7 +284,8 @@ public void testSuspendedDoesntRun() indexerMetadataStorageCoordinator, new MaterializedViewTaskConfig(), createMock(AuthorizerMapper.class), - createMock(ChatHandlerProvider.class) + createMock(ChatHandlerProvider.class), + new SupervisorStateManagerConfig() ); MaterializedViewSupervisor supervisor = (MaterializedViewSupervisor) suspended.createSupervisor(); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index a8deb58e835d..5d419a4497f3 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -193,6 +193,7 @@ protected SeekableStreamSupervisorReportPayload createReportPaylo includeOffsets ? sequenceLastUpdated : null, spec.isSuspended(), stateManager.isHealthy(), + stateManager.getSupervisorState().getBasicState(), stateManager.getSupervisorState(), stateManager.getExceptionEvents() ); @@ -384,4 +385,10 @@ public KafkaSupervisorIOConfig getIoConfig() { return spec.getIoConfig(); } + + @Override + public Boolean isHealthy() + { + return stateManager.isHealthy(); + } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index ae8fd12a56b2..768468c933c6 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -19,8 +19,8 @@ package org.apache.druid.indexing.kafka.supervisor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -41,8 +41,9 @@ public KafkaSupervisorReportPayload( @Nullable DateTime offsetsLastUpdated, boolean suspended, boolean healthy, - SeekableStreamSupervisorStateManager.State state, - List recentErrors + SupervisorStateManager.State state, + SupervisorStateManager.State detailedState, + List recentErrors ) { super( @@ -58,6 +59,7 @@ public KafkaSupervisorReportPayload( suspended, healthy, state, + detailedState, recentErrors ); } @@ -80,6 +82,7 @@ public String toString() ", suspended=" + isSuspended() + ", healthy=" + isHealthy() + ", state=" + getState() + + ", detailedState=" + getDetailedState() + ", recentErrors=" + getRecentErrors() + '}'; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index cde8f0b86877..ef6259f64121 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -30,7 +30,7 @@ import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; @@ -55,7 +55,7 @@ public KafkaSupervisorSpec( @JacksonInject ServiceEmitter emitter, @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, - @JacksonInject SeekableStreamSupervisorConfig supervisorConfig + @JacksonInject SupervisorStateManagerConfig supervisorStateManagerConfig ) { super( @@ -98,7 +98,7 @@ public KafkaSupervisorSpec( emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - supervisorConfig + supervisorStateManagerConfig ); } @@ -147,7 +147,7 @@ protected KafkaSupervisorSpec toggleSuspend(boolean suspend) emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - supervisorConfig + supervisorStateManagerConfig ); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java index 729d8a51fd57..1506d9894a27 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java @@ -28,7 +28,7 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.math.expr.ExprMacroTable; @@ -57,7 +57,7 @@ public KafkaSupervisorSpecTest() .addValue(ServiceEmitter.class, new NoopServiceEmitter()) .addValue(DruidMonitorSchedulerConfig.class, null) .addValue(RowIngestionMetersFactory.class, null) - .addValue(SeekableStreamSupervisorConfig.class, null) + .addValue(SupervisorStateManagerConfig.class, null) .addValue(ExprMacroTable.class.getName(), LookupEnabledTestExprMacroTable.INSTANCE) ); mapper.registerModules((Iterable) new KafkaIndexTaskModule().getJacksonModules()); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index e93556a9b9ef..6f9f163b92c6 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -61,11 +61,12 @@ import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; @@ -157,7 +158,7 @@ public class KafkaSupervisorTest extends EasyMockSupport private String topic; private RowIngestionMetersFactory rowIngestionMetersFactory; private ExceptionCapturingServiceEmitter serviceEmitter; - private SeekableStreamSupervisorConfig supervisorConfig; + private SupervisorStateManagerConfig supervisorConfig; private static String getTopic() { @@ -240,7 +241,7 @@ public void setupTest() rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); serviceEmitter = new ExceptionCapturingServiceEmitter(); EmittingLogger.registerEmitter(serviceEmitter); - supervisorConfig = new SeekableStreamSupervisorConfig(); + supervisorConfig = new SupervisorStateManagerConfig(); } @After @@ -1267,7 +1268,7 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertEquals(topic, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(SupervisorStateManager.BasicState.RUNNING, payload.getDetailedState()); Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData publishingReport = payload.getPublishingTasks().get(0); @@ -1375,7 +1376,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertEquals(topic, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(SupervisorStateManager.BasicState.RUNNING, payload.getDetailedState()); Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData publishingReport = payload.getPublishingTasks().get(0); @@ -1514,7 +1515,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception Assert.assertEquals(topic, payload.getStream()); Assert.assertEquals(1, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(SupervisorStateManager.BasicState.RUNNING, payload.getDetailedState()); Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData activeReport = payload.getActiveTasks().get(0); @@ -3238,7 +3239,7 @@ public KafkaIndexTaskClient build( new NoopServiceEmitter(), new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, - new SeekableStreamSupervisorConfig() + new SupervisorStateManagerConfig() ), rowIngestionMetersFactory ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 64612a29347d..39619a268630 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -249,6 +249,7 @@ protected SeekableStreamSupervisorReportPayload createReportPayl ioConfig.getTaskDuration().getMillis() / 1000, spec.isSuspended(), stateManager.isHealthy(), + stateManager.getSupervisorState().getBasicState(), stateManager.getSupervisorState(), stateManager.getExceptionEvents() ); @@ -315,4 +316,10 @@ protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() { return true; } + + @Override + public Boolean isHealthy() + { + return stateManager.isHealthy(); + } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java index 0f135cd06bda..9a4ee86937df 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java @@ -19,8 +19,8 @@ package org.apache.druid.indexing.kinesis.supervisor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import java.util.Collections; import java.util.List; @@ -35,8 +35,9 @@ public KinesisSupervisorReportPayload( Long durationSeconds, boolean suspended, boolean healthy, - SeekableStreamSupervisorStateManager.State state, - List recentErrors + SupervisorStateManager.State state, + SupervisorStateManager.State detailedState, + List recentErrors ) { super( @@ -52,6 +53,7 @@ public KinesisSupervisorReportPayload( suspended, healthy, state, + detailedState, recentErrors ); } @@ -70,6 +72,7 @@ public String toString() ", suspended=" + isSuspended() + ", healthy=" + isHealthy() + ", state=" + getState() + + ", detailedState=" + getDetailedState() + ", recentErrors=" + getRecentErrors() + '}'; } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index 03c70450dcb2..ec72c7de6d7e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -32,7 +32,7 @@ import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; @@ -60,7 +60,7 @@ public KinesisSupervisorSpec( @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, @JacksonInject @Named("kinesis") AWSCredentialsConfig awsCredentialsConfig, - @JacksonInject SeekableStreamSupervisorConfig supervisorConfig + @JacksonInject SupervisorStateManagerConfig supervisorStateManagerConfig ) { super( @@ -109,7 +109,7 @@ public KinesisSupervisorSpec( emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - supervisorConfig + supervisorStateManagerConfig ); this.awsCredentialsConfig = awsCredentialsConfig; } @@ -174,7 +174,7 @@ protected KinesisSupervisorSpec toggleSuspend(boolean suspend) monitorSchedulerConfig, rowIngestionMetersFactory, awsCredentialsConfig, - supervisorConfig + supervisorStateManagerConfig ); } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index c914902c8b91..ddc6ba8b92e8 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -56,12 +56,13 @@ import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; @@ -139,7 +140,7 @@ public class KinesisSupervisorTest extends EasyMockSupport private TaskQueue taskQueue; private RowIngestionMetersFactory rowIngestionMetersFactory; private ExceptionCapturingServiceEmitter serviceEmitter; - private SeekableStreamSupervisorConfig supervisorConfig; + private SupervisorStateManagerConfig supervisorConfig; public KinesisSupervisorTest() { @@ -197,7 +198,7 @@ public void setupTest() rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); serviceEmitter = new ExceptionCapturingServiceEmitter(); EmittingLogger.registerEmitter(serviceEmitter); - supervisorConfig = new SeekableStreamSupervisorConfig(); + supervisorConfig = new SupervisorStateManagerConfig(); } @After @@ -1389,7 +1390,7 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertEquals(stream, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(SupervisorStateManager.BasicState.RUNNING, payload.getDetailedState()); Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData publishingReport = payload.getPublishingTasks().get(0); @@ -1532,7 +1533,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertEquals(stream, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(SupervisorStateManager.BasicState.RUNNING, payload.getDetailedState()); Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData publishingReport = payload.getPublishingTasks().get(0); @@ -1722,7 +1723,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception Assert.assertEquals(stream, payload.getStream()); Assert.assertEquals(1, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); - Assert.assertEquals(SeekableStreamSupervisorStateManager.State.RUNNING, payload.getState()); + Assert.assertEquals(SupervisorStateManager.BasicState.RUNNING, payload.getDetailedState()); Assert.assertEquals(0, payload.getRecentErrors().size()); TaskReportData activeReport = payload.getActiveTasks().get(0); @@ -3781,7 +3782,7 @@ public KinesisIndexTaskClient build( new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, null, - new SeekableStreamSupervisorConfig() + new SupervisorStateManagerConfig() ), rowIngestionMetersFactory ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java index 327dd4dbfc55..b820b23012dd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java @@ -339,7 +339,7 @@ private boolean isTaskPending(Task task) public boolean add(final Task task) throws EntryExistsException { if (taskStorage.getTask(task.getId()).isPresent()) { - throw new EntryExistsException(StringUtils.format("Task %s is already exists", task.getId())); + throw new EntryExistsException(StringUtils.format("Task %s already exists", task.getId())); } giant.lock(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 3168b898de61..56112d15732d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -183,6 +183,12 @@ public Optional>> getSupervisorStats(String id) return supervisor == null ? Optional.absent() : Optional.fromNullable(supervisor.lhs.getStats()); } + public Optional isSupervisorHealthy(String id) + { + Pair supervisor = supervisors.get(id); + return supervisor == null ? Optional.absent() : Optional.fromNullable(supervisor.lhs.isHealthy()); + } + public boolean resetSupervisor(String id, @Nullable DataSourceMetadata dataSourceMetadata) { Preconditions.checkState(started, "SupervisorManager not started"); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java index f8ae4b292d8b..9d97a80aca7b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -181,12 +181,31 @@ public Response specGetStatus(@PathParam("id") final String id) .build(); } - boolean healthyOrNotApplicable = !(spec.get().getPayload() instanceof HealthCheckableSupervisorReportPayload) - || ((HealthCheckableSupervisorReportPayload) spec.get() - .getPayload()).isHealthy(); + return Response.ok(spec.get()).build(); + } + ); + } + + @GET + @Path("/{id}/health") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(SupervisorResourceFilter.class) + public Response specGetHealth(@PathParam("id") final String id) + { + return asLeaderWithSupervisorManager( + manager -> { + Optional healthy = manager.isSupervisorHealthy(id); + if (!healthy.isPresent()) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ImmutableMap.of( + "error", + StringUtils.format("[%s] does not exist or health check not implemented", id) + )) + .build(); + } - return Response.status(healthyOrNotApplicable ? Response.Status.OK : Response.Status.SERVICE_UNAVAILABLE) - .entity(spec.get()) + return Response.status(healthy.get() ? Response.Status.OK : Response.Status.SERVICE_UNAVAILABLE) + .entity(ImmutableMap.of("healthy", healthy.get())) .build(); } ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index fdcb130642e9..87e876a0d050 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -54,6 +54,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; @@ -537,12 +538,7 @@ public SeekableStreamSupervisor( this.exec = Execs.singleThreaded(supervisorId); this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); this.reportingExec = Execs.scheduledSingleThreaded(supervisorId + "-Reporting-%d"); - this.stateManager = new SeekableStreamSupervisorStateManager( - SeekableStreamSupervisorStateManager.State.WAITING_TO_RUN, - spec.isSuspended() ? SeekableStreamSupervisorStateManager.State.SUSPENDED - : SeekableStreamSupervisorStateManager.State.RUNNING, - spec.getSupervisorConfig() - ); + this.stateManager = new SeekableStreamSupervisorStateManager(spec.getSupervisorStateManagerConfig(), spec.isSuspended()); int workerThreads = (this.tuningConfig.getWorkerThreads() != null ? this.tuningConfig.getWorkerThreads() @@ -662,7 +658,7 @@ public void stop(boolean stopGracefully) Preconditions.checkState(lifecycleStarted, "lifecycle not started"); log.info("Beginning shutdown of [%s]", supervisorId); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.STOPPING); + stateManager.maybeSetState(SupervisorStateManager.BasicState.STOPPING); try { scheduledExec.shutdownNow(); // stop recurring executions @@ -1035,12 +1031,12 @@ public void runInternal() try { possiblyRegisterListener(); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CONNECTING_TO_STREAM); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.SeekableStreamState.CONNECTING_TO_STREAM); if (!updatePartitionDataFromStream() && !stateManager.isAtLeastOneSuccessfulRun()) { return; // if we can't connect to the stream and this is the first run, stop and wait to retry the connection } - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.DISCOVERING_INITIAL_TASKS); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.SeekableStreamState.DISCOVERING_INITIAL_TASKS); discoverTasks(); updateTaskStatus(); @@ -1056,7 +1052,7 @@ public void runInternal() if (!spec.isSuspended()) { log.info("[%s] supervisor is running.", dataSource); - stateManager.maybeSetState(SeekableStreamSupervisorStateManager.State.CREATING_TASKS); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.SeekableStreamState.CREATING_TASKS); createNewTasks(); } else { log.info("[%s] supervisor is suspended.", dataSource); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index bc9afe200412..7436488cf116 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.overlord.supervisor.HealthCheckableSupervisorReportPayload; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.java.util.common.IAE; import org.joda.time.DateTime; @@ -31,8 +31,7 @@ import java.util.Map; @JsonInclude(JsonInclude.Include.NON_NULL) -public abstract class SeekableStreamSupervisorReportPayload implements - HealthCheckableSupervisorReportPayload +public abstract class SeekableStreamSupervisorReportPayload { private final String dataSource; private final String stream; @@ -47,8 +46,9 @@ public abstract class SeekableStreamSupervisorReportPayload recentErrors; + private final SupervisorStateManager.State state; + private final SupervisorStateManager.State detailedState; + private final List recentErrors; public SeekableStreamSupervisorReportPayload( String dataSource, @@ -62,8 +62,9 @@ public SeekableStreamSupervisorReportPayload( @Nullable DateTime offsetsLastUpdated, boolean suspended, boolean healthy, - SeekableStreamSupervisorStateManager.State state, - List recentErrors + SupervisorStateManager.State state, + SupervisorStateManager.State detailedState, + List recentErrors ) { this.dataSource = dataSource; @@ -80,6 +81,7 @@ public SeekableStreamSupervisorReportPayload( this.suspended = suspended; this.healthy = healthy; this.state = state; + this.detailedState = detailedState; this.recentErrors = recentErrors; } @@ -124,7 +126,6 @@ public boolean isSuspended() return suspended; } - @Override @JsonProperty public boolean isHealthy() { @@ -174,13 +175,19 @@ public DateTime getOffsetsLastUpdated() } @JsonProperty - public SeekableStreamSupervisorStateManager.State getState() + public SupervisorStateManager.State getState() { return state; } @JsonProperty - public List getRecentErrors() + public SupervisorStateManager.State getDetailedState() + { + return detailedState; + } + + @JsonProperty + public List getRecentErrors() { return recentErrors; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index afd8fb9b7457..469821ac8e68 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -32,6 +32,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; @@ -57,7 +58,7 @@ public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec protected final ServiceEmitter emitter; protected final DruidMonitorSchedulerConfig monitorSchedulerConfig; private final boolean suspended; - protected final SeekableStreamSupervisorConfig supervisorConfig; + protected final SupervisorStateManagerConfig supervisorStateManagerConfig; @JsonCreator public SeekableStreamSupervisorSpec( @@ -74,7 +75,7 @@ public SeekableStreamSupervisorSpec( @JacksonInject ServiceEmitter emitter, @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, - @JacksonInject SeekableStreamSupervisorConfig supervisorConfig + @JacksonInject SupervisorStateManagerConfig supervisorStateManagerConfig ) { this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); @@ -91,7 +92,7 @@ public SeekableStreamSupervisorSpec( this.monitorSchedulerConfig = monitorSchedulerConfig; this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.suspended = suspended != null ? suspended : false; - this.supervisorConfig = supervisorConfig; + this.supervisorStateManagerConfig = supervisorStateManagerConfig; } @JsonProperty @@ -156,9 +157,9 @@ public SeekableStreamSupervisorSpec createRunningSpec() return toggleSuspend(false); } - public SeekableStreamSupervisorConfig getSupervisorConfig() + public SupervisorStateManagerConfig getSupervisorStateManagerConfig() { - return supervisorConfig; + return supervisorStateManagerConfig; } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java index 0e0fcb4e4c86..5b41fc2a9df1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManager.java @@ -20,220 +20,83 @@ package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonPropertyOrder; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; import org.apache.commons.lang.exception.ExceptionUtils; -import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.seekablestream.common.StreamException; -import org.apache.druid.java.util.common.DateTimes; -import org.joda.time.DateTime; -import java.util.ArrayList; -import java.util.Deque; -import java.util.List; -import java.util.concurrent.ConcurrentLinkedDeque; - -public class SeekableStreamSupervisorStateManager +public class SeekableStreamSupervisorStateManager extends SupervisorStateManager { - public enum State + public enum SeekableStreamState implements State { - // Error states - ordered from high to low priority - UNHEALTHY_SUPERVISOR(false, false), - UNHEALTHY_TASKS(false, false), - UNABLE_TO_CONNECT_TO_STREAM(false, false), + UNABLE_TO_CONNECT_TO_STREAM(false, true), LOST_CONTACT_WITH_STREAM(false, false), - // Non-error states - equal priority - WAITING_TO_RUN(true, true), CONNECTING_TO_STREAM(true, true), DISCOVERING_INITIAL_TASKS(true, true), - CREATING_TASKS(true, true), - RUNNING(true, false), - SUSPENDED(true, false), - STOPPING(true, false); + CREATING_TASKS(true, true); private final boolean healthy; private final boolean firstRunOnly; - State(boolean healthy, boolean firstRunOnly) + SeekableStreamState(boolean healthy, boolean firstRunOnly) { this.healthy = healthy; this.firstRunOnly = firstRunOnly; } - } - - private final SeekableStreamSupervisorConfig supervisorConfig; - private final State healthySteadyState; - - private final Deque recentEventsQueue; - - private State supervisorState; - - private boolean atLeastOneSuccessfulRun = false; - private boolean currentRunSuccessful = true; - - // Used to determine if a low consecutiveSuccessfulRuns/consecutiveSuccessfulTasks means that the supervisor is - // recovering from an unhealthy state, or if the supervisor just started and hasn't run many times yet. - private boolean hasHitUnhealthinessThreshold = false; - private boolean hasHitTaskUnhealthinessThreshold = false; - - private int consecutiveFailedRuns = 0; - private int consecutiveSuccessfulRuns = 0; - private int consecutiveFailedTasks = 0; - private int consecutiveSuccessfulTasks = 0; - - public SeekableStreamSupervisorStateManager( - State initialState, - State healthySteadyState, - SeekableStreamSupervisorConfig supervisorConfig - ) - { - Preconditions.checkArgument(supervisorConfig.getMaxStoredExceptionEvents() >= Math.max( - supervisorConfig.getHealthinessThreshold(), - supervisorConfig.getUnhealthinessThreshold() - ), "maxStoredExceptionEvents must be >= to max(healthinessThreshold, unhealthinessThreshold)"); - - this.supervisorState = initialState; - this.supervisorConfig = supervisorConfig; - this.healthySteadyState = healthySteadyState; - - this.recentEventsQueue = new ConcurrentLinkedDeque<>(); - } - - /** - * Certain states are only valid if the supervisor hasn't had a successful iteration. This method checks if there's - * been at least one successful iteration, and if applicable sets supervisor state to an appropriate new state. - */ - public void maybeSetState(State proposedState) - { - // if we're over our unhealthiness threshold, set the state to the appropriate unhealthy state - if (consecutiveFailedRuns >= supervisorConfig.getUnhealthinessThreshold()) { - hasHitUnhealthinessThreshold = true; - supervisorState = recentEventsQueue.getLast().isStreamException() - ? (atLeastOneSuccessfulRun ? State.LOST_CONTACT_WITH_STREAM : State.UNABLE_TO_CONNECT_TO_STREAM) - : State.UNHEALTHY_SUPERVISOR; - return; - } - // if we're over our task unhealthiness threshold, set the state to UNHEALTHY_TASKS - if (consecutiveFailedTasks >= supervisorConfig.getTaskUnhealthinessThreshold()) { - hasHitTaskUnhealthinessThreshold = true; - supervisorState = State.UNHEALTHY_TASKS; - return; - } - - // if we're currently in an unhealthy state and are below our healthiness threshold for either runs and tasks, - // ignore the proposed state; the healthiness threshold only applies if we've had a failure in the past - if (!this.supervisorState.healthy - && ((hasHitUnhealthinessThreshold && consecutiveSuccessfulRuns < supervisorConfig.getHealthinessThreshold()) - || (hasHitTaskUnhealthinessThreshold - && consecutiveSuccessfulTasks < supervisorConfig.getTaskHealthinessThreshold()))) { - return; + @Override + public boolean isHealthy() + { + return healthy; } - // if we're trying to switch to a healthy steady state (i.e. RUNNING or SUSPENDED) but haven't had a successful run - // yet, refuse to switch and prefer the more specific states used for first run (CONNECTING_TO_STREAM, - // DISCOVERING_INITIAL_TASKS, CREATING_TASKS, etc.) - if (healthySteadyState.equals(proposedState) && !atLeastOneSuccessfulRun) { - return; + @Override + public boolean isFirstRunOnly() + { + return firstRunOnly; } - // accept the state if it is not a firstRunOnly state OR we are still on the first run - if (!proposedState.firstRunOnly || !atLeastOneSuccessfulRun) { - supervisorState = proposedState; + @Override + public State getBasicState() + { + return healthy ? BasicState.RUNNING : BasicState.UNHEALTHY_SUPERVISOR; } } - public void recordThrowableEvent(Throwable t) + public SeekableStreamSupervisorStateManager(SupervisorStateManagerConfig supervisorConfig, boolean suspended) { - recentEventsQueue.add(new ExceptionEvent(t, supervisorConfig.isStoringStackTraces())); - - if (recentEventsQueue.size() > supervisorConfig.getMaxStoredExceptionEvents()) { - recentEventsQueue.poll(); - } - - currentRunSuccessful = false; + super(supervisorConfig, suspended); } - public void recordCompletedTaskState(TaskState state) + @Override + protected State getSpecificUnhealthySupervisorState() { - if (state.isSuccess()) { - consecutiveSuccessfulTasks++; - consecutiveFailedTasks = 0; - } else if (state.isFailure()) { - consecutiveFailedTasks++; - consecutiveSuccessfulTasks = 0; + ExceptionEvent event = getRecentEventsQueue().getLast(); + if (event instanceof SeekableStreamExceptionEvent && ((SeekableStreamExceptionEvent) event).isStreamException()) { + return isAtLeastOneSuccessfulRun() + ? SeekableStreamState.LOST_CONTACT_WITH_STREAM + : SeekableStreamState.UNABLE_TO_CONNECT_TO_STREAM; } - } - - public void markRunFinished() - { - atLeastOneSuccessfulRun |= currentRunSuccessful; - - consecutiveSuccessfulRuns = currentRunSuccessful ? consecutiveSuccessfulRuns + 1 : 0; - consecutiveFailedRuns = currentRunSuccessful ? 0 : consecutiveFailedRuns + 1; - // Try to set the state to RUNNING or SUSPENDED, depending on how the supervisor was configured. This will be - // rejected if we haven't had atLeastOneSuccessfulRun (in favor of the more specific states for the initial run) and - // will instead trigger setting the state to an unhealthy one if we are now over the error thresholds. - maybeSetState(healthySteadyState); - - // reset for next run - currentRunSuccessful = true; + return BasicState.UNHEALTHY_SUPERVISOR; } - public List getExceptionEvents() + @Override + protected ExceptionEvent buildExceptionEvent(Throwable t) { - return new ArrayList<>(recentEventsQueue); + return new SeekableStreamExceptionEvent(t, isStoreStackTrace()); } - public State getSupervisorState() + public static class SeekableStreamExceptionEvent extends ExceptionEvent { - return supervisorState; - } - - public boolean isHealthy() - { - return supervisorState != null && supervisorState.healthy; - } - - public boolean isAtLeastOneSuccessfulRun() - { - return atLeastOneSuccessfulRun; - } - - @JsonPropertyOrder({"timestamp", "exceptionClass", "streamException", "message"}) - public static class ExceptionEvent - { - private static final List SKIPPED_EXCEPTION_CLASSES = ImmutableList.of( - RuntimeException.class.getName(), - StreamException.class.getName() - ); - - private final DateTime timestamp; - private final String exceptionClass; private final boolean streamException; - private final String message; // contains full stackTrace if storingStackTraces is true - public ExceptionEvent(Throwable t, boolean storingStackTraces) + public SeekableStreamExceptionEvent(Throwable t, boolean storeStackTrace) { - this.timestamp = DateTimes.nowUtc(); - this.exceptionClass = getMeaningfulExceptionClass(t); - this.streamException = ExceptionUtils.indexOfType(t, StreamException.class) != -1; - this.message = storingStackTraces ? ExceptionUtils.getStackTrace(t) : t.getMessage(); - } + super(t, storeStackTrace); - @JsonProperty - public DateTime getTimestamp() - { - return timestamp; - } - - @JsonProperty - public String getExceptionClass() - { - return exceptionClass; + this.streamException = ExceptionUtils.indexOfType(t, StreamException.class) != -1; } @JsonProperty @@ -242,20 +105,10 @@ public boolean isStreamException() return streamException; } - @JsonProperty - public String getMessage() - { - return message; - } - - private String getMeaningfulExceptionClass(Throwable t) + @Override + protected boolean shouldSkipException(String className) { - return ((List) ExceptionUtils.getThrowableList(t)) - .stream() - .map(x -> x.getClass().getName()) - .filter(x -> !SKIPPED_EXCEPTION_CLASSES.contains(x)) - .findFirst() - .orElse(Exception.class.getName()); + return RuntimeException.class.getName().equals(className) || StreamException.class.getName().equals(className); } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index 3b31dec99323..96afde5ed1d5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -69,7 +69,8 @@ public void setUp() { supervisorResource = new SupervisorResource( taskMaster, - new AuthorizerMapper(null) { + new AuthorizerMapper(null) + { @Override public Authorizer getAuthorizer(String name) { @@ -92,7 +93,8 @@ public Authorizer getAuthorizer(String name) @Test public void testSpecPost() { - SupervisorSpec spec = new TestSupervisorSpec("my-id", null, null) { + SupervisorSpec spec = new TestSupervisorSpec("my-id", null, null) + { @Override public List getDataSources() @@ -132,7 +134,8 @@ public List getDataSources() public void testSpecGetAll() { Set supervisorIds = ImmutableSet.of("id1", "id2"); - SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, null) { + SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, null) + { @Override public List getDataSources() @@ -140,7 +143,8 @@ public List getDataSources() return Collections.singletonList("datasource1"); } }; - SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, null) { + SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, null) + { @Override public List getDataSources() @@ -183,7 +187,8 @@ public void testSpecGetAllFull() { Set supervisorIds = ImmutableSet.of("id1", "id2"); - SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, null) { + SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, null) + { @Override public List getDataSources() @@ -191,7 +196,8 @@ public List getDataSources() return Collections.singletonList("datasource1"); } }; - SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, null) { + SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, null) + { @Override public List getDataSources() @@ -289,10 +295,41 @@ public void testSpecGetStatus() Assert.assertEquals(503, response.getStatus()); } + @Test + public void testSpecGetHealth() + { + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(3); + EasyMock.expect(supervisorManager.isSupervisorHealthy("my-id")).andReturn(Optional.of(true)); + EasyMock.expect(supervisorManager.isSupervisorHealthy("my-id-2")).andReturn(Optional.of(false)); + EasyMock.expect(supervisorManager.isSupervisorHealthy("my-id-3")).andReturn(Optional.absent()); + replayAll(); + + Response response = supervisorResource.specGetHealth("my-id"); + + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("healthy", true), response.getEntity()); + + response = supervisorResource.specGetHealth("my-id-2"); + + Assert.assertEquals(503, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("healthy", false), response.getEntity()); + + response = supervisorResource.specGetHealth("my-id-3"); + + Assert.assertEquals(404, response.getStatus()); + Assert.assertEquals( + ImmutableMap.of("error", "[my-id-3] does not exist or health check not implemented"), + response.getEntity() + ); + + verifyAll(); + } + @Test public void testSpecSuspend() { - TestSupervisorSpec suspended = new TestSupervisorSpec("my-id", null, null, true) { + TestSupervisorSpec suspended = new TestSupervisorSpec("my-id", null, null, true) + { @Override public List getDataSources() { @@ -329,7 +366,8 @@ public List getDataSources() @Test public void testSpecResume() { - TestSupervisorSpec running = new TestSupervisorSpec("my-id", null, null, false) { + TestSupervisorSpec running = new TestSupervisorSpec("my-id", null, null, false) + { @Override public List getDataSources() { @@ -872,8 +910,14 @@ public void testReset() Capture id1 = Capture.newInstance(); Capture id2 = Capture.newInstance(); EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(2); - EasyMock.expect(supervisorManager.resetSupervisor(EasyMock.capture(id1), EasyMock.anyObject(DataSourceMetadata.class))).andReturn(true); - EasyMock.expect(supervisorManager.resetSupervisor(EasyMock.capture(id2), EasyMock.anyObject(DataSourceMetadata.class))).andReturn(false); + EasyMock.expect(supervisorManager.resetSupervisor( + EasyMock.capture(id1), + EasyMock.anyObject(DataSourceMetadata.class) + )).andReturn(true); + EasyMock.expect(supervisorManager.resetSupervisor( + EasyMock.capture(id2), + EasyMock.anyObject(DataSourceMetadata.class) + )).andReturn(false); replayAll(); Response response = supervisorResource.reset("my-id"); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java index 21a87c0fdb8f..f65fccf70231 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateManagerTest.java @@ -23,8 +23,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager.BasicState; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.seekablestream.common.StreamException; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.State; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.SeekableStreamExceptionEvent; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.SeekableStreamState; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Pair; import org.junit.Assert; @@ -39,50 +43,59 @@ public class SeekableStreamSupervisorStateManagerTest { private SeekableStreamSupervisorStateManager stateManager; - private SeekableStreamSupervisorConfig config; + private SupervisorStateManagerConfig config; private ObjectMapper defaultMapper; @Before public void setupTest() { - config = new SeekableStreamSupervisorConfig(); - config.setMaxStoredExceptionEvents(10); - stateManager = new SeekableStreamSupervisorStateManager(State.WAITING_TO_RUN, State.RUNNING, config); + config = new SupervisorStateManagerConfig(10); + stateManager = new SeekableStreamSupervisorStateManager(config, false); defaultMapper = new DefaultObjectMapper(); } @Test public void testHappyPath() { - Assert.assertEquals(State.WAITING_TO_RUN, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.PENDING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.PENDING, stateManager.getSupervisorState().getBasicState()); - stateManager.maybeSetState(State.CONNECTING_TO_STREAM); - Assert.assertEquals(State.CONNECTING_TO_STREAM, stateManager.getSupervisorState()); + stateManager.maybeSetState(SeekableStreamSupervisorStateManager.SeekableStreamState.CONNECTING_TO_STREAM); + Assert.assertEquals(SeekableStreamState.CONNECTING_TO_STREAM, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState().getBasicState()); - stateManager.maybeSetState(State.DISCOVERING_INITIAL_TASKS); - Assert.assertEquals(State.DISCOVERING_INITIAL_TASKS, stateManager.getSupervisorState()); + stateManager.maybeSetState(SeekableStreamState.DISCOVERING_INITIAL_TASKS); + Assert.assertEquals(SeekableStreamState.DISCOVERING_INITIAL_TASKS, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState().getBasicState()); - stateManager.maybeSetState(State.CREATING_TASKS); - Assert.assertEquals(State.CREATING_TASKS, stateManager.getSupervisorState()); + stateManager.maybeSetState(SeekableStreamState.CREATING_TASKS); + Assert.assertEquals(SeekableStreamState.CREATING_TASKS, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState().getBasicState()); stateManager.markRunFinished(); - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState().getBasicState()); - stateManager.maybeSetState(State.WAITING_TO_RUN); - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + stateManager.maybeSetState(BasicState.PENDING); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState().getBasicState()); - stateManager.maybeSetState(State.CONNECTING_TO_STREAM); - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + stateManager.maybeSetState(SeekableStreamState.CONNECTING_TO_STREAM); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState().getBasicState()); - stateManager.maybeSetState(State.DISCOVERING_INITIAL_TASKS); - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + stateManager.maybeSetState(SeekableStreamState.DISCOVERING_INITIAL_TASKS); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState().getBasicState()); - stateManager.maybeSetState(State.CREATING_TASKS); - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + stateManager.maybeSetState(SeekableStreamState.CREATING_TASKS); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState().getBasicState()); stateManager.markRunFinished(); - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState().getBasicState()); } @Test @@ -90,18 +103,19 @@ public void testStreamFailureLostContact() { stateManager.markRunFinished(); // clean run without errors - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); stateManager.recordThrowableEvent(new StreamException(new IllegalStateException("DOH!"))); stateManager.markRunFinished(); } - Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.LOST_CONTACT_WITH_STREAM, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); stateManager.getExceptionEvents().forEach(x -> { - Assert.assertTrue(x.isStreamException()); + Assert.assertTrue(((SeekableStreamExceptionEvent) x).isStreamException()); Assert.assertEquals(IllegalStateException.class.getName(), x.getExceptionClass()); }); } @@ -109,17 +123,18 @@ public void testStreamFailureLostContact() @Test public void testStreamFailureUnableToConnect() { - stateManager.maybeSetState(State.CONNECTING_TO_STREAM); + stateManager.maybeSetState(SeekableStreamState.CONNECTING_TO_STREAM); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - Assert.assertEquals(State.CONNECTING_TO_STREAM, stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.CONNECTING_TO_STREAM, stateManager.getSupervisorState()); stateManager.recordThrowableEvent(new StreamException(new IllegalStateException("DOH!"))); stateManager.markRunFinished(); } - Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.UNABLE_TO_CONNECT_TO_STREAM, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); stateManager.getExceptionEvents().forEach(x -> { - Assert.assertTrue(x.isStreamException()); + Assert.assertTrue(((SeekableStreamExceptionEvent) x).isStreamException()); Assert.assertEquals(IllegalStateException.class.getName(), x.getExceptionClass()); }); } @@ -127,17 +142,18 @@ public void testStreamFailureUnableToConnect() @Test public void testNonStreamUnhealthiness() { - stateManager.maybeSetState(State.DISCOVERING_INITIAL_TASKS); + stateManager.maybeSetState(SeekableStreamState.DISCOVERING_INITIAL_TASKS); for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - Assert.assertEquals(State.DISCOVERING_INITIAL_TASKS, stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.DISCOVERING_INITIAL_TASKS, stateManager.getSupervisorState()); stateManager.recordThrowableEvent(new NullPointerException("oof")); stateManager.markRunFinished(); } - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); stateManager.getExceptionEvents().forEach(x -> { - Assert.assertFalse(x.isStreamException()); + Assert.assertFalse(((SeekableStreamExceptionEvent) x).isStreamException()); Assert.assertEquals(NullPointerException.class.getName(), x.getExceptionClass()); }); } @@ -150,11 +166,12 @@ public void testTransientUnhealthiness() for (int i = 0; i < config.getUnhealthinessThreshold() - 1; i++) { stateManager.recordThrowableEvent(new NullPointerException("oof")); stateManager.markRunFinished(); - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); } stateManager.markRunFinished(); // clean run - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(j * (config.getUnhealthinessThreshold() - 1), stateManager.getExceptionEvents().size()); } } @@ -164,11 +181,12 @@ public void testNonTransientTaskUnhealthiness() { stateManager.markRunFinished(); for (int i = 0; i < config.getTaskUnhealthinessThreshold(); i++) { - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); stateManager.recordCompletedTaskState(TaskState.FAILED); stateManager.markRunFinished(); } - Assert.assertEquals(State.UNHEALTHY_TASKS, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_TASKS, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_TASKS, stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(0, stateManager.getExceptionEvents().size()); } @@ -178,12 +196,13 @@ public void testTransientTaskUnhealthiness() // Only half are failing stateManager.markRunFinished(); for (int i = 0; i < config.getTaskUnhealthinessThreshold() + 3; i++) { - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); stateManager.recordCompletedTaskState(TaskState.FAILED); stateManager.recordCompletedTaskState(TaskState.SUCCESS); stateManager.markRunFinished(); } - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(0, stateManager.getExceptionEvents().size()); } @@ -192,22 +211,22 @@ public void testSupervisorRecoveryWithHealthinessThreshold() { // Put into an unhealthy state for (int i = 0; i < config.getUnhealthinessThreshold(); i++) { - Assert.assertEquals(State.WAITING_TO_RUN, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.PENDING, stateManager.getSupervisorState()); stateManager.recordThrowableEvent(new Exception("Except the inevitable")); stateManager.markRunFinished(); } - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); // Recover after config.healthinessThreshold successful task completions for (int i = 0; i < config.getHealthinessThreshold(); i++) { - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); stateManager.markRunFinished(); } - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); Assert.assertEquals(config.getUnhealthinessThreshold(), stateManager.getExceptionEvents().size()); stateManager.getExceptionEvents().forEach(x -> { - Assert.assertFalse(x.isStreamException()); + Assert.assertFalse(((SeekableStreamExceptionEvent) x).isStreamException()); Assert.assertEquals(Exception.class.getName(), x.getExceptionClass()); }); } @@ -219,19 +238,19 @@ public void testTaskRecoveryWithHealthinessThreshold() // Put into an unhealthy state for (int i = 0; i < config.getTaskUnhealthinessThreshold(); i++) { - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); stateManager.recordCompletedTaskState(TaskState.FAILED); stateManager.markRunFinished(); } - Assert.assertEquals(State.UNHEALTHY_TASKS, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_TASKS, stateManager.getSupervisorState()); // Recover after config.healthinessThreshold successful task completions for (int i = 0; i < config.getTaskHealthinessThreshold(); i++) { - Assert.assertEquals(State.UNHEALTHY_TASKS, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_TASKS, stateManager.getSupervisorState()); stateManager.recordCompletedTaskState(TaskState.SUCCESS); stateManager.markRunFinished(); } - Assert.assertEquals(State.RUNNING, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, stateManager.getSupervisorState()); } @Test @@ -245,7 +264,7 @@ public void testTwoUnhealthyStates() stateManager.markRunFinished(); } // UNHEALTHY_SUPERVISOR should take priority over UNHEALTHY_TASKS - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); } @Test @@ -262,7 +281,7 @@ public void testGetThrowableEvents() stateManager.markRunFinished(); } - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, stateManager.getSupervisorState()); List> expected = ImmutableList.of( Pair.of("java.lang.UnsupportedOperationException", true), @@ -271,12 +290,12 @@ public void testGetThrowableEvents() Pair.of("java.lang.IllegalArgumentException", false) ); - Iterator it = stateManager.getExceptionEvents().iterator(); + Iterator it = stateManager.getExceptionEvents().iterator(); expected.forEach(x -> { - SeekableStreamSupervisorStateManager.ExceptionEvent event = it.next(); + SupervisorStateManager.ExceptionEvent event = it.next(); Assert.assertNotNull(event.getMessage()); Assert.assertEquals(x.lhs, event.getExceptionClass()); - Assert.assertEquals(x.rhs, event.isStreamException()); + Assert.assertEquals(x.rhs, ((SeekableStreamExceptionEvent) event).isStreamException()); }); Assert.assertFalse(it.hasNext()); @@ -285,8 +304,8 @@ public void testGetThrowableEvents() @Test public void testExceptionEventSerde() throws IOException { - SeekableStreamSupervisorStateManager.ExceptionEvent event = - new SeekableStreamSupervisorStateManager.ExceptionEvent(new NullPointerException("msg"), true); + SupervisorStateManager.ExceptionEvent event = + new SupervisorStateManager.ExceptionEvent(new NullPointerException("msg"), true); String serialized = defaultMapper.writeValueAsString(event); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index ad5b8163fe93..1058eb01e073 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -41,6 +41,9 @@ import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerListener; import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager.BasicState; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; @@ -54,7 +57,8 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamException; import org.apache.druid.indexing.seekablestream.common.StreamPartition; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.State; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.SeekableStreamExceptionEvent; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.SeekableStreamState; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.JSONPathSpec; @@ -111,7 +115,7 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport private RecordSupplier recordSupplier; private RowIngestionMetersFactory rowIngestionMetersFactory; - private SeekableStreamSupervisorConfig supervisorConfig; + private SupervisorStateManagerConfig supervisorConfig; @Before public void setupTest() @@ -128,9 +132,9 @@ public void setupTest() rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); - supervisorConfig = new SeekableStreamSupervisorConfig(); + supervisorConfig = new SupervisorStateManagerConfig(); - expect(spec.getSupervisorConfig()).andReturn(supervisorConfig).anyTimes(); + expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); expect(spec.getIoConfig()).andReturn(getIOConfig()).anyTimes(); @@ -163,21 +167,24 @@ public void testRunning() throws Exception supervisor.start(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.PENDING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.PENDING, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); @@ -200,17 +207,18 @@ public void testConnectingToStreamFail() throws Exception supervisor.start(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.PENDING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); - List exceptionEvents = supervisor.stateManager.getExceptionEvents(); + Assert.assertEquals(SeekableStreamState.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); + List exceptionEvents = supervisor.stateManager.getExceptionEvents(); Assert.assertEquals(1, exceptionEvents.size()); - Assert.assertTrue(exceptionEvents.get(0).isStreamException()); + Assert.assertTrue(((SeekableStreamExceptionEvent) exceptionEvents.get(0)).isStreamException()); Assert.assertEquals(IllegalStateException.class.getName(), exceptionEvents.get(0).getExceptionClass()); Assert.assertEquals( StringUtils.format("%s: %s", IllegalStateException.class.getName(), EXCEPTION_MSG), @@ -221,14 +229,16 @@ public void testConnectingToStreamFail() throws Exception supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(2, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); @@ -253,44 +263,52 @@ public void testConnectingToStreamFailRecoveryFailRecovery() throws Exception supervisor.start(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.PENDING, supervisor.stateManager.getSupervisorState()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.CONNECTING_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); - Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState().getBasicState()); supervisor.runInternal(); - Assert.assertEquals(State.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.UNABLE_TO_CONNECT_TO_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState().getBasicState()); supervisor.runInternal(); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); supervisor.runInternal(); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); supervisor.runInternal(); - Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.LOST_CONTACT_WITH_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.LOST_CONTACT_WITH_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState().getBasicState()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.LOST_CONTACT_WITH_STREAM, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.LOST_CONTACT_WITH_STREAM, supervisor.stateManager.getSupervisorState()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); verifyAll(); @@ -313,57 +331,60 @@ public void testDiscoveringInitialTasksFailRecoveryFail() throws Exception supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.DISCOVERING_INITIAL_TASKS, supervisor.stateManager.getSupervisorState()); - List exceptionEvents = supervisor.stateManager.getExceptionEvents(); + Assert.assertEquals(SeekableStreamState.DISCOVERING_INITIAL_TASKS, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); + List exceptionEvents = supervisor.stateManager.getExceptionEvents(); Assert.assertEquals(1, exceptionEvents.size()); - Assert.assertFalse(exceptionEvents.get(0).isStreamException()); + Assert.assertFalse(((SeekableStreamExceptionEvent) exceptionEvents.get(0)).isStreamException()); Assert.assertEquals(IllegalStateException.class.getName(), exceptionEvents.get(0).getExceptionClass()); Assert.assertEquals(EXCEPTION_MSG, exceptionEvents.get(0).getMessage()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.DISCOVERING_INITIAL_TASKS, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.DISCOVERING_INITIAL_TASKS, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(2, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); verifyAll(); @@ -386,57 +407,61 @@ public void testCreatingTasksFailRecoveryFail() throws Exception supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.CREATING_TASKS, supervisor.stateManager.getSupervisorState()); - List exceptionEvents = supervisor.stateManager.getExceptionEvents(); + Assert.assertEquals(SeekableStreamState.CREATING_TASKS, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); + List exceptionEvents = supervisor.stateManager.getExceptionEvents(); Assert.assertEquals(1, exceptionEvents.size()); - Assert.assertFalse(exceptionEvents.get(0).isStreamException()); + Assert.assertFalse(((SeekableStreamExceptionEvent) exceptionEvents.get(0)).isStreamException()); Assert.assertEquals(IllegalStateException.class.getName(), exceptionEvents.get(0).getExceptionClass()); Assert.assertEquals(EXCEPTION_MSG, exceptionEvents.get(0).getMessage()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.CREATING_TASKS, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(SeekableStreamState.CREATING_TASKS, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(2, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertEquals(3, supervisor.stateManager.getExceptionEvents().size()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertFalse(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.UNHEALTHY_SUPERVISOR, supervisor.stateManager.getSupervisorState()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); verifyAll(); @@ -457,21 +482,24 @@ public void testSuspended() throws Exception supervisor.start(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.PENDING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.PENDING, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.SUSPENDED, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.SUSPENDED, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.SUSPENDED, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.SUSPENDED, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.SUSPENDED, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.SUSPENDED, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); @@ -497,21 +525,24 @@ public void testStopping() throws Exception supervisor.start(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.WAITING_TO_RUN, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.PENDING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.PENDING, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.runInternal(); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.RUNNING, supervisor.stateManager.getSupervisorState().getBasicState()); Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty()); Assert.assertTrue(supervisor.stateManager.isAtLeastOneSuccessfulRun()); supervisor.stop(false); Assert.assertTrue(supervisor.stateManager.isHealthy()); - Assert.assertEquals(State.STOPPING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.STOPPING, supervisor.stateManager.getSupervisorState()); + Assert.assertEquals(BasicState.STOPPING, supervisor.stateManager.getSupervisorState().getBasicState()); verifyAll(); } @@ -849,6 +880,7 @@ protected SeekableStreamSupervisorReportPayload createReportPayl false, true, null, + null, null ) { diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/HealthCheckableSupervisorReportPayload.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/HealthCheckableSupervisorReportPayload.java deleted file mode 100644 index 054a9864240b..000000000000 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/HealthCheckableSupervisorReportPayload.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.overlord.supervisor; - -public interface HealthCheckableSupervisorReportPayload -{ - boolean isHealthy(); -} diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java index 03c2a43d9ebb..cf3f4d5fa2e2 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java @@ -44,6 +44,12 @@ default Map> getStats() return ImmutableMap.of(); } + @Nullable + default Boolean isHealthy() + { + return null; // default implementation for interface compatability; returning null since true or false is misleading + } + void reset(DataSourceMetadata dataSourceMetadata); /** diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorModule.java similarity index 76% rename from indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java rename to server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorModule.java index 6feea536842b..d281f55e52c1 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/SeekableStreamSupervisorModule.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorModule.java @@ -18,17 +18,17 @@ */ -package org.apache.druid.guice; +package org.apache.druid.indexing.overlord.supervisor; import com.google.inject.Binder; import com.google.inject.Module; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorConfig; +import org.apache.druid.guice.JsonConfigProvider; -public class SeekableStreamSupervisorModule implements Module +public class SupervisorModule implements Module { @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.supervisor.stream", SeekableStreamSupervisorConfig.class); + JsonConfigProvider.bind(binder, "druid.supervisor", SupervisorStateManagerConfig.class); } } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java new file mode 100644 index 000000000000..76cf8c60927e --- /dev/null +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java @@ -0,0 +1,288 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.overlord.supervisor; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.java.util.common.DateTimes; +import org.joda.time.DateTime; + +import java.util.ArrayList; +import java.util.Deque; +import java.util.List; +import java.util.concurrent.ConcurrentLinkedDeque; + +public class SupervisorStateManager +{ + public interface State + { + /** + * If we are in this state, is the supervisor healthy or unhealthy? + */ + boolean isHealthy(); + + /** + * It may be helpful to provide more detailed state information (e.g. CONNECTING_TO_STREAM, CREATING_TASKS, etc.) + * during the first run of the supervisor so that if the supervisor is unable to complete the run, we have + * information about what stage it was in when it failed. Once the supervisor is stable, we may not be as concerned + * about all the stages it cycles through, and just want to know if it's healthy or unhealthy. This flag indicates + * if the state should only be accepted prior to having completed a successful run. + */ + boolean isFirstRunOnly(); + + default State getBasicState() + { + return this; + } + } + + public enum BasicState implements State + { + UNHEALTHY_SUPERVISOR(false, false), + UNHEALTHY_TASKS(false, false), + + PENDING(true, true), + RUNNING(true, false), + SUSPENDED(true, false), + STOPPING(true, false); + + private final boolean healthy; + private final boolean firstRunOnly; + + BasicState(boolean healthy, boolean firstRunOnly) + { + this.healthy = healthy; + this.firstRunOnly = firstRunOnly; + } + + @Override + public boolean isHealthy() + { + return healthy; + } + + @Override + public boolean isFirstRunOnly() + { + return firstRunOnly; + } + } + + private final SupervisorStateManagerConfig supervisorStateManagerConfig; + private final State healthySteadyState; + + private final Deque recentEventsQueue = new ConcurrentLinkedDeque<>(); + + private State supervisorState = BasicState.PENDING; + + private boolean atLeastOneSuccessfulRun = false; + private boolean currentRunSuccessful = true; + + // Used to determine if a low consecutiveSuccessfulRuns/consecutiveSuccessfulTasks means that the supervisor is + // recovering from an unhealthy state, or if the supervisor just started and hasn't run many times yet. + private boolean hasHitUnhealthinessThreshold = false; + private boolean hasHitTaskUnhealthinessThreshold = false; + + private int consecutiveFailedRuns = 0; + private int consecutiveSuccessfulRuns = 0; + private int consecutiveFailedTasks = 0; + private int consecutiveSuccessfulTasks = 0; + + public SupervisorStateManager(SupervisorStateManagerConfig supervisorStateManagerConfig, boolean suspended) + { + Preconditions.checkArgument(supervisorStateManagerConfig.getMaxStoredExceptionEvents() >= Math.max( + supervisorStateManagerConfig.getHealthinessThreshold(), + supervisorStateManagerConfig.getUnhealthinessThreshold() + ), "maxStoredExceptionEvents must be >= to max(healthinessThreshold, unhealthinessThreshold)"); + + this.supervisorStateManagerConfig = supervisorStateManagerConfig; + this.healthySteadyState = suspended ? BasicState.SUSPENDED : BasicState.RUNNING; + } + + /** + * Certain states are only valid if the supervisor hasn't had a successful iteration. This method checks if there's + * been at least one successful iteration, and if applicable sets supervisor state to an appropriate new state. + */ + public void maybeSetState(State proposedState) + { + // if we're over our unhealthiness threshold, set the state to the appropriate unhealthy state + if (consecutiveFailedRuns >= supervisorStateManagerConfig.getUnhealthinessThreshold()) { + hasHitUnhealthinessThreshold = true; + supervisorState = getSpecificUnhealthySupervisorState(); + return; + } + + // if we're over our task unhealthiness threshold, set the state to UNHEALTHY_TASKS + if (consecutiveFailedTasks >= supervisorStateManagerConfig.getTaskUnhealthinessThreshold()) { + hasHitTaskUnhealthinessThreshold = true; + supervisorState = BasicState.UNHEALTHY_TASKS; + return; + } + + // if we're currently in an unhealthy state and are below our healthiness threshold for either runs and tasks, + // ignore the proposed state; the healthiness threshold only applies if we've had a failure in the past + if (!this.supervisorState.isHealthy() + && ((hasHitUnhealthinessThreshold + && consecutiveSuccessfulRuns < supervisorStateManagerConfig.getHealthinessThreshold()) + || (hasHitTaskUnhealthinessThreshold + && consecutiveSuccessfulTasks < supervisorStateManagerConfig.getTaskHealthinessThreshold()))) { + return; + } + + // if we're trying to switch to a healthy steady state (i.e. RUNNING or SUSPENDED) but haven't had a successful run + // yet, refuse to switch and prefer the more specific states used for first run (CONNECTING_TO_STREAM, + // DISCOVERING_INITIAL_TASKS, CREATING_TASKS, etc.) + if (healthySteadyState.equals(proposedState) && !atLeastOneSuccessfulRun) { + return; + } + + // accept the state if it is not a firstRunOnly state OR we are still on the first run + if (!proposedState.isFirstRunOnly() || !atLeastOneSuccessfulRun) { + supervisorState = proposedState; + } + } + + public void recordThrowableEvent(Throwable t) + { + recentEventsQueue.add(buildExceptionEvent(t)); + + if (recentEventsQueue.size() > supervisorStateManagerConfig.getMaxStoredExceptionEvents()) { + recentEventsQueue.poll(); + } + + currentRunSuccessful = false; + } + + public void recordCompletedTaskState(TaskState state) + { + if (state.isSuccess()) { + consecutiveSuccessfulTasks++; + consecutiveFailedTasks = 0; + } else if (state.isFailure()) { + consecutiveFailedTasks++; + consecutiveSuccessfulTasks = 0; + } + } + + public void markRunFinished() + { + atLeastOneSuccessfulRun |= currentRunSuccessful; + + consecutiveSuccessfulRuns = currentRunSuccessful ? consecutiveSuccessfulRuns + 1 : 0; + consecutiveFailedRuns = currentRunSuccessful ? 0 : consecutiveFailedRuns + 1; + + // Try to set the state to RUNNING or SUSPENDED. This will be rejected if we haven't had atLeastOneSuccessfulRun + // (in favor of the more specific states for the initial run) and will instead trigger setting the state to an + // unhealthy one if we are now over the error thresholds. + maybeSetState(healthySteadyState); + + // reset for next run + currentRunSuccessful = true; + } + + public List getExceptionEvents() + { + return new ArrayList<>(recentEventsQueue); + } + + public State getSupervisorState() + { + return supervisorState; + } + + public boolean isHealthy() + { + return supervisorState != null && supervisorState.isHealthy(); + } + + public boolean isAtLeastOneSuccessfulRun() + { + return atLeastOneSuccessfulRun; + } + + protected Deque getRecentEventsQueue() + { + return recentEventsQueue; + } + + protected boolean isStoreStackTrace() + { + return supervisorStateManagerConfig.isStoreStackTrace(); + } + + protected State getSpecificUnhealthySupervisorState() + { + return BasicState.UNHEALTHY_SUPERVISOR; + } + + protected ExceptionEvent buildExceptionEvent(Throwable t) + { + return new ExceptionEvent(t, isStoreStackTrace()); + } + + public static class ExceptionEvent + { + private final DateTime timestamp; + private final String exceptionClass; + private final String message; // contains full stackTrace if storeStackTrace is true + + public ExceptionEvent(Throwable t, boolean storeStackTrace) + { + this.timestamp = DateTimes.nowUtc(); + this.exceptionClass = getMeaningfulExceptionClass(t); + this.message = storeStackTrace ? ExceptionUtils.getStackTrace(t) : t.getMessage(); + } + + @JsonProperty + public DateTime getTimestamp() + { + return timestamp; + } + + @JsonProperty + public String getExceptionClass() + { + return exceptionClass; + } + + @JsonProperty + public String getMessage() + { + return message; + } + + protected boolean shouldSkipException(String className) + { + return RuntimeException.class.getName().equals(className); + } + + private String getMeaningfulExceptionClass(Throwable t) + { + return ((List) ExceptionUtils.getThrowableList(t)) + .stream() + .map(x -> x.getClass().getName()) + .filter(x -> !shouldSkipException(x)) + .findFirst() + .orElse(Exception.class.getName()); + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorConfig.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerConfig.java similarity index 85% rename from indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorConfig.java rename to server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerConfig.java index 450fb9774d00..5dde7a399a87 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorConfig.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerConfig.java @@ -17,15 +17,14 @@ * under the License. */ -package org.apache.druid.indexing.seekablestream.supervisor; +package org.apache.druid.indexing.overlord.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.annotations.VisibleForTesting; -public class SeekableStreamSupervisorConfig +public class SupervisorStateManagerConfig { @JsonProperty - private boolean storingStackTraces = false; + private boolean storeStackTrace = false; // The number of failed runs before the supervisor is considered unhealthy @JsonProperty @@ -47,9 +46,19 @@ public class SeekableStreamSupervisorConfig @JsonProperty private int maxStoredExceptionEvents = Math.max(unhealthinessThreshold, healthinessThreshold); - public boolean isStoringStackTraces() + public SupervisorStateManagerConfig() { - return storingStackTraces; + + } + + public SupervisorStateManagerConfig(int maxStoredExceptionEvents) + { + this.maxStoredExceptionEvents = maxStoredExceptionEvents; + } + + public boolean isStoreStackTrace() + { + return storeStackTrace; } public int getUnhealthinessThreshold() @@ -76,10 +85,4 @@ public int getMaxStoredExceptionEvents() { return maxStoredExceptionEvents; } - - @VisibleForTesting - void setMaxStoredExceptionEvents(int maxStoredExceptionEvents) - { - this.maxStoredExceptionEvents = maxStoredExceptionEvents; - } } diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 071e17f6ffe7..fc74b886442d 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -50,7 +50,6 @@ import org.apache.druid.guice.ListProvider; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.PolyBind; -import org.apache.druid.guice.SeekableStreamSupervisorModule; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; @@ -91,6 +90,7 @@ import org.apache.druid.indexing.overlord.sampler.SamplerModule; import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.indexing.overlord.supervisor.SupervisorModule; import org.apache.druid.indexing.overlord.supervisor.SupervisorResource; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.logger.Logger; @@ -198,7 +198,9 @@ public void configure(Binder binder) binder.bind(SupervisorManager.class).in(LazySingleton.class); binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); - binder.bind(new TypeLiteral>() {}) + binder.bind(new TypeLiteral>() + { + }) .toProvider(Providers.of(null)); binder.bind(ChatHandlerProvider.class).toProvider(Providers.of(null)); @@ -334,7 +336,7 @@ private void configureOverlordHelpers(Binder binder) }, new IndexingServiceFirehoseModule(), new IndexingServiceTaskLogsModule(), - new SeekableStreamSupervisorModule(), + new SupervisorModule(), new SamplerModule() ); } From d13aba107071961c03c2d38fb93b80ca0429cabf Mon Sep 17 00:00:00 2001 From: dclim Date: Wed, 22 May 2019 20:43:45 -0600 Subject: [PATCH 18/22] fixup after merge --- .../org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java | 1 + .../apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java | 1 + 2 files changed, 2 insertions(+) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java index 5820047a8d20..dce37662364d 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java @@ -160,6 +160,7 @@ public void testSample() null, null, null, + null, null ); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index 62b1226466fa..95f6b4d6241c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -179,6 +179,7 @@ public void testSample() throws Exception null, null, null, + null, null ); From 02dbec26f3515992d922af45fdf18d481c86b5b4 Mon Sep 17 00:00:00 2001 From: dclim Date: Thu, 30 May 2019 22:50:01 -0600 Subject: [PATCH 19/22] code review changes - add additional docs --- .../extensions-core/kafka-ingestion.md | 61 +++++++++++++------ 1 file changed, 42 insertions(+), 19 deletions(-) diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index cd1c6bcbce9f..c070e466517a 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -214,25 +214,48 @@ offsets as reported by Kafka, the consumer lag per partition, as well as the agg consumer lag per partition may be reported as negative values if the supervisor has not received a recent latest offset response from Kafka. The aggregate lag value will always be >= 0. -The status report also contains the supervisor's state and a list of recently thrown exceptions (whose max size can be -controlled using the `druid.supervisor.maxStoredExceptionEvents` config parameter). The list of states is as -follows: - -|State|Description| -|-----|-----------| -|UNHEALTHY_SUPERVISOR|The supervisor has encountered errors on the past `druid.supervisor.unhealthinessThreshold` iterations| -|UNHEALTHY_TASKS|The last `druid.supervisor.taskUnhealthinessThreshold` tasks have all failed| -|UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kafka and has not successfully connected in the past| -|LOST_CONTACT_WITH_STREAM|The supervisor is encountering connectivity issues with Kafka but has successfully connected in the past| -|PENDING (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream| -|CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data| -|DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks| -|CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state| -|RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse| -|SUSPENDED|The supervisor has been suspended| -|STOPPING|The supervisor is stopping| - -States marked with "first iteration only" only occur on the supervisor's first iteration at startup or after suspension. +The status report also contains the supervisor's state and a list of recently thrown exceptions (reported as +`recentErrors`, whose max size can be controlled using the `druid.supervisor.maxStoredExceptionEvents` configuration). +There are two fields related to the supervisor's state - `state` and `detailedState`. The `state` field will always be +one of a small number of generic states that are applicable to any type of supervisor, while the `detailedState` field +will contain a more descriptive, implementation-specific state that may provide more insight into the supervisor's +activities than the generic `state` field. + +The list of possible `state` values are: [`PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`, `UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`] + +The list of `detailedState` values and their corresponding `state` mapping is as follows: + +|Detailed State|Corresponding State|Description| +|--------------|-------------------|-----------| +|UNHEALTHY_SUPERVISOR|UNHEALTHY_SUPERVISOR|The supervisor has encountered errors on the past `druid.supervisor.unhealthinessThreshold` iterations| +|UNHEALTHY_TASKS|UNHEALTHY_TASKS|The last `druid.supervisor.taskUnhealthinessThreshold` tasks have all failed| +|UNABLE_TO_CONNECT_TO_STREAM|UNHEALTHY_SUPERVISOR|The supervisor is encountering connectivity issues with Kafka and has not successfully connected in the past| +|LOST_CONTACT_WITH_STREAM|UNHEALTHY_SUPERVISOR|The supervisor is encountering connectivity issues with Kafka but has successfully connected in the past| +|PENDING (first iteration only)|PENDING|The supervisor has been initialized and hasn't started connecting to the stream| +|CONNECTING_TO_STREAM (first iteration only)|RUNNING|The supervisor is trying to connect to the stream and update partition data| +|DISCOVERING_INITIAL_TASKS (first iteration only)|RUNNING|The supervisor is discovering already-running tasks| +|CREATING_TASKS (first iteration only)|RUNNING|The supervisor is creating tasks and discovering state| +|RUNNING|RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse| +|SUSPENDED|SUSPENDED|The supervisor has been suspended| +|STOPPING|STOPPING|The supervisor is stopping| + +On each iteration of the supervisor's run loop, the supervisor completes the following tasks in sequence: + 1) Fetch the list of partitions from Kafka and determine the starting offset for each partition (either based on the + last processed offset if continuing, or starting from the beginning or ending of the stream if this is a new topic). + 2) Discover any running indexing tasks that are writing to the supervisor's datasource and adopt them if they match + the supervisor's configuration, else signal them to stop. + 3) Send a status request to each supervised task to update our view of the state of the tasks under our supervision. + 4) Handle tasks that have exceeded `taskDuration` and should transition from the reading to publishing state. + 5) Handle tasks that have finished publishing and signal redundant replica tasks to stop. + 6) Handle tasks that have failed and clean up the supervisor's internal state. + 7) Compare the list of healthy tasks to the requested `taskCount` and `replicas` configurations and create additional tasks if required. + +The `detailedState` field will show additional values (those marked with "first iteration only") the first time the +supervisor executes this run loop after startup or after resuming from a suspension. This is intended to surface +initialization-type issues, where the supervisor is unable to reach a stable state (perhaps because it can't connect to +Kafka, it can't read from the Kafka topic, or it can't communicate with existing tasks). Once the supervisor is stable - +that is, once it has completed a full execution without encountering any issues - `detailedState` will show a `RUNNING` +state until it is stopped, suspended, or hits a failure threshold and transitions to an unhealthy state. ### Getting Supervisor Ingestion Stats Report From c515a3a64948c15769aa4c021349fff5563764a6 Mon Sep 17 00:00:00 2001 From: dclim Date: Fri, 31 May 2019 12:41:48 -0600 Subject: [PATCH 20/22] cleanup KafkaIndexTaskTest --- .../indexing/kafka/KafkaIndexTaskTest.java | 21 +++---------------- 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 783af47ca8bb..6c42b3b85627 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -831,15 +831,8 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception // as soon as any segment has more than one record, incremental publishing should happen maxRowsPerSegment = 2; - // Insert data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : records) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } + insertData(); + Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); @@ -2301,15 +2294,7 @@ public void testCanStartFromLaterThanEarliestOffset() throws Exception maxRowsPerSegment = Integer.MAX_VALUE; maxTotalRows = null; - // Insert data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : records) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } + insertData(); Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); From 17c018441056f34276c29b436f5aa49355d9f43e Mon Sep 17 00:00:00 2001 From: dclim Date: Fri, 31 May 2019 15:44:24 -0600 Subject: [PATCH 21/22] add additional documentation for Kinesis indexing --- .../extensions-core/kinesis-ingestion.md | 61 +++++++++++++------ 1 file changed, 42 insertions(+), 19 deletions(-) diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md index 8180ef9af367..0578dd23b3b1 100644 --- a/docs/content/development/extensions-core/kinesis-ingestion.md +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -228,25 +228,48 @@ profile provider (in this order). managed by the given supervisor. This includes the latest sequence numbers as reported by Kinesis. Unlike the Kafka Indexing Service, stats about lag are not yet supported. -The status report also contains the supervisor's state and a list of recently thrown exceptions (whose max size can be -controlled using the `druid.supervisor.maxStoredExceptionEvents` config parameter). The list of states is as -follows: - -|State|Description| -|-----|-----------| -|UNHEALTHY_SUPERVISOR|The supervisor has encountered errors on the past `druid.supervisor.unhealthinessThreshold` iterations| -|UNHEALTHY_TASKS|The last `druid.supervisor.taskUnhealthinessThreshold` tasks have all failed| -|UNABLE_TO_CONNECT_TO_STREAM|The supervisor is encountering connectivity issues with Kinesis and has not successfully connected in the past| -|LOST_CONTACT_WITH_STREAM|The supervisor is encountering connectivity issues with Kinesis but has successfully connected in the past| -|PENDING (first iteration only)|The supervisor has been initialized and hasn't started connecting to the stream| -|CONNECTING_TO_STREAM (first iteration only)|The supervisor is trying to connect to the stream and update partition data| -|DISCOVERING_INITIAL_TASKS (first iteration only)|The supervisor is discovering already-running tasks| -|CREATING_TASKS (first iteration only)|The supervisor is creating tasks and discovering state| -|RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse| -|SUSPENDED|The supervisor has been suspended| -|STOPPING|The supervisor is stopping| - -States marked with "first iteration only" only occur on the supervisor's first iteration at startup or after suspension. +The status report also contains the supervisor's state and a list of recently thrown exceptions (reported as +`recentErrors`, whose max size can be controlled using the `druid.supervisor.maxStoredExceptionEvents` configuration). +There are two fields related to the supervisor's state - `state` and `detailedState`. The `state` field will always be +one of a small number of generic states that are applicable to any type of supervisor, while the `detailedState` field +will contain a more descriptive, implementation-specific state that may provide more insight into the supervisor's +activities than the generic `state` field. + +The list of possible `state` values are: [`PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`, `UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`] + +The list of `detailedState` values and their corresponding `state` mapping is as follows: + +|Detailed State|Corresponding State|Description| +|--------------|-------------------|-----------| +|UNHEALTHY_SUPERVISOR|UNHEALTHY_SUPERVISOR|The supervisor has encountered errors on the past `druid.supervisor.unhealthinessThreshold` iterations| +|UNHEALTHY_TASKS|UNHEALTHY_TASKS|The last `druid.supervisor.taskUnhealthinessThreshold` tasks have all failed| +|UNABLE_TO_CONNECT_TO_STREAM|UNHEALTHY_SUPERVISOR|The supervisor is encountering connectivity issues with Kinesis and has not successfully connected in the past| +|LOST_CONTACT_WITH_STREAM|UNHEALTHY_SUPERVISOR|The supervisor is encountering connectivity issues with Kinesis but has successfully connected in the past| +|PENDING (first iteration only)|PENDING|The supervisor has been initialized and hasn't started connecting to the stream| +|CONNECTING_TO_STREAM (first iteration only)|RUNNING|The supervisor is trying to connect to the stream and update partition data| +|DISCOVERING_INITIAL_TASKS (first iteration only)|RUNNING|The supervisor is discovering already-running tasks| +|CREATING_TASKS (first iteration only)|RUNNING|The supervisor is creating tasks and discovering state| +|RUNNING|RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse| +|SUSPENDED|SUSPENDED|The supervisor has been suspended| +|STOPPING|STOPPING|The supervisor is stopping| + +On each iteration of the supervisor's run loop, the supervisor completes the following tasks in sequence: + 1) Fetch the list of shards from Kinesis and determine the starting sequence number for each shard (either based on the + last processed sequence number if continuing, or starting from the beginning or ending of the stream if this is a new stream). + 2) Discover any running indexing tasks that are writing to the supervisor's datasource and adopt them if they match + the supervisor's configuration, else signal them to stop. + 3) Send a status request to each supervised task to update our view of the state of the tasks under our supervision. + 4) Handle tasks that have exceeded `taskDuration` and should transition from the reading to publishing state. + 5) Handle tasks that have finished publishing and signal redundant replica tasks to stop. + 6) Handle tasks that have failed and clean up the supervisor's internal state. + 7) Compare the list of healthy tasks to the requested `taskCount` and `replicas` configurations and create additional tasks if required. + +The `detailedState` field will show additional values (those marked with "first iteration only") the first time the +supervisor executes this run loop after startup or after resuming from a suspension. This is intended to surface +initialization-type issues, where the supervisor is unable to reach a stable state (perhaps because it can't connect to +Kinesis, it can't read from the stream, or it can't communicate with existing tasks). Once the supervisor is stable - +that is, once it has completed a full execution without encountering any issues - `detailedState` will show a `RUNNING` +state until it is stopped, suspended, or hits a failure threshold and transitions to an unhealthy state. ### Updating Existing Supervisors From e7b492d0f99a217062026d8295e5b31ae4f73897 Mon Sep 17 00:00:00 2001 From: dclim Date: Fri, 31 May 2019 16:25:00 -0600 Subject: [PATCH 22/22] remove unused throws class --- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 57981334b05f..5e7c693aa0c6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -241,7 +241,7 @@ public String toString() */ private interface Notice { - void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException; + void handle() throws ExecutionException, InterruptedException, TimeoutException; } private static class StatsFromTaskResult