From 3769f7be0ceea37d8c6b9f6fd4b7fa19961a4b5b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 11 Dec 2017 21:40:42 +0900 Subject: [PATCH 1/9] PendingSegments cleanup --- .../java/io/druid/indexer}/TaskLocation.java | 2 +- .../main/java/io/druid/indexer/TaskState.java | 47 ++++++ .../java/io/druid/indexer/TaskStatusPlus.java | 94 ++++++++++++ .../MetadataStorageActionHandler.java | 18 ++- .../SQLServerMetadataStorageModule.java | 7 + .../indexing/kafka/KafkaIndexTaskClient.java | 2 +- .../kafka/supervisor/KafkaSupervisor.java | 2 +- .../kafka/KafkaIndexTaskClientTest.java | 2 +- .../indexing/kafka/KafkaIndexTaskTest.java | 55 +++---- .../kafka/supervisor/KafkaSupervisorTest.java | 2 +- .../mysql/MySQLMetadataStorageModule.java | 7 + .../PostgreSQLMetadataStorageModule.java | 7 + .../indexing/common/TaskInfoProvider.java | 1 + .../io/druid/indexing/common/TaskStatus.java | 30 ++-- .../common/actions/SegmentAllocateAction.java | 23 +-- .../indexing/overlord/ForkingTaskRunner.java | 2 +- .../overlord/HeapMemoryTaskStorage.java | 88 ++++++++--- .../IndexerMetadataStorageAdapter.java | 68 +++++++++ .../overlord/MetadataTaskStorage.java | 31 ++-- .../indexing/overlord/RemoteTaskRunner.java | 2 +- .../overlord/RemoteTaskRunnerWorkItem.java | 2 +- .../indexing/overlord/TaskRunnerListener.java | 2 +- .../indexing/overlord/TaskRunnerUtils.java | 2 +- .../indexing/overlord/TaskRunnerWorkItem.java | 2 +- .../druid/indexing/overlord/TaskStorage.java | 14 +- .../overlord/TaskStorageQueryAdapter.java | 14 +- .../overlord/ThreadPoolTaskRunner.java | 2 +- .../overlord/http/OverlordResource.java | 142 ++++++++---------- .../indexing/worker/TaskAnnouncement.java | 7 +- .../indexing/worker/WorkerTaskMonitor.java | 2 +- .../common/task/RealtimeIndexTaskTest.java | 17 ++- .../IndexerMetadataStorageAdapterTest.java | 98 ++++++++++++ ...kRunnerRunPendingTasksConcurrencyTest.java | 19 +-- .../overlord/RemoteTaskRunnerTest.java | 19 +-- .../overlord/RemoteTaskRunnerTestUtils.java | 2 +- .../indexing/overlord/TaskLifecycleTest.java | 27 ++-- .../overlord/TaskLockBoxConcurrencyTest.java | 4 +- .../indexing/overlord/TaskLockboxTest.java | 4 +- ...dingTaskBasedProvisioningStrategyTest.java | 2 +- .../SimpleProvisioningStrategyTest.java | 2 +- .../overlord/http/OverlordResourceTest.java | 87 +++++++---- .../indexing/overlord/http/OverlordTest.java | 31 ++-- ...TestIndexerMetadataStorageCoordinator.java | 6 + .../indexing/worker/TaskAnnouncementTest.java | 2 +- .../worker/WorkerTaskMonitorTest.java | 8 +- .../clients/OverlordResourceTestClient.java | 12 +- .../indexing/IndexingServiceClient.java | 68 +++++++++ .../guice/SQLMetadataStorageDruidModule.java | 6 - .../IndexerMetadataStorageCoordinator.java | 14 ++ .../DerbyMetadataStorageActionHandler.java | 75 +++++++++ ...byMetadataStorageActionHandlerFactory.java | 53 +++++++ .../IndexerSQLMetadataStorageCoordinator.java | 18 +++ .../MySQLMetadataStorageActionHandler.java | 75 +++++++++ ...QLMetadataStorageActionHandlerFactory.java | 52 +++++++ ...ostgreSQLMetadataStorageActionHandler.java | 77 ++++++++++ ...QLMetadataStorageActionHandlerFactory.java | 52 +++++++ .../SQLMetadataStorageActionHandler.java | 107 ++++++++----- ...QLMetadataStorageActionHandlerFactory.java | 25 +-- ...SQLServerMetadataStorageActionHandler.java | 72 +++++++++ ...erMetadataStorageActionHandlerFactory.java | 52 +++++++ .../DerbyMetadataStorageDruidModule.java | 11 +- .../coordinator/CoordinatorDynamicConfig.java | 64 +++++--- ...ruidCoordinatorCleanupPendingSegments.java | 60 ++++++++ .../coordinator/DruidCoordinatorConfig.java | 6 + ...exerSQLMetadataStorageCoordinatorTest.java | 111 +++++++++++++- .../SQLMetadataStorageActionHandlerTest.java | 44 +++++- .../DruidCoordinatorConfigTest.java | 6 +- .../http/CoordinatorDynamicConfigTest.java | 3 +- .../java/io/druid/cli/CliCoordinator.java | 6 + .../main/java/io/druid/cli/CliOverlord.java | 2 + 70 files changed, 1692 insertions(+), 386 deletions(-) rename {indexing-service/src/main/java/io/druid/indexing/common => api/src/main/java/io/druid/indexer}/TaskLocation.java (98%) create mode 100644 api/src/main/java/io/druid/indexer/TaskState.java create mode 100644 api/src/main/java/io/druid/indexer/TaskStatusPlus.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java create mode 100644 server/src/main/java/io/druid/metadata/DerbyMetadataStorageActionHandler.java create mode 100644 server/src/main/java/io/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java create mode 100644 server/src/main/java/io/druid/metadata/MySQLMetadataStorageActionHandler.java create mode 100644 server/src/main/java/io/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java create mode 100644 server/src/main/java/io/druid/metadata/PostgreSQLMetadataStorageActionHandler.java create mode 100644 server/src/main/java/io/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java create mode 100644 server/src/main/java/io/druid/metadata/SQLServerMetadataStorageActionHandler.java create mode 100644 server/src/main/java/io/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java create mode 100644 server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskLocation.java b/api/src/main/java/io/druid/indexer/TaskLocation.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/common/TaskLocation.java rename to api/src/main/java/io/druid/indexer/TaskLocation.java index 6d4d816bc8c1..70fd0b2e3cea 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskLocation.java +++ b/api/src/main/java/io/druid/indexer/TaskLocation.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common; +package io.druid.indexer; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/api/src/main/java/io/druid/indexer/TaskState.java b/api/src/main/java/io/druid/indexer/TaskState.java new file mode 100644 index 000000000000..c4f54a1f716f --- /dev/null +++ b/api/src/main/java/io/druid/indexer/TaskState.java @@ -0,0 +1,47 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexer; + +public enum TaskState +{ + RUNNING, + SUCCESS, + FAILED; + + public boolean isRunnable() + { + return this == RUNNING; + } + + public boolean isComplete() + { + return this != RUNNING; + } + + public boolean isSuccess() + { + return this == SUCCESS; + } + + public boolean isFailure() + { + return this == FAILED; + } +} diff --git a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java new file mode 100644 index 000000000000..a45a9a7865a9 --- /dev/null +++ b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java @@ -0,0 +1,94 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexer; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; + +public class TaskStatusPlus +{ + private final String id; + private final DateTime createdTime; + private final DateTime queueInsertionTime; + private final TaskState state; + private final Long duration; + private final TaskLocation location; + + @JsonCreator + public TaskStatusPlus( + @JsonProperty("id") String id, + @JsonProperty("createdTime") DateTime createdTime, + @JsonProperty("queueInsertionTime") DateTime queueInsertionTime, + @JsonProperty("state") @Nullable TaskState state, + @JsonProperty("duration") @Nullable Long duration, + @JsonProperty("location") TaskLocation location + ) + { + if (state != null && state.isComplete()) { + Preconditions.checkNotNull(duration, "duration"); + } + this.id = Preconditions.checkNotNull(id, "id"); + this.createdTime = Preconditions.checkNotNull(createdTime, "createdTime"); + this.queueInsertionTime = Preconditions.checkNotNull(queueInsertionTime, "queueInsertionTime"); + this.state = state; + this.duration = duration; + this.location = Preconditions.checkNotNull(location, "location"); + } + + @JsonProperty + public String getId() + { + return id; + } + + @JsonProperty + public DateTime getCreatedTime() + { + return createdTime; + } + + @JsonProperty + public DateTime getQueueInsertionTime() + { + return queueInsertionTime; + } + + @JsonProperty + public TaskState getState() + { + return state; + } + + @JsonProperty + public Long getDuration() + { + return duration; + } + + @JsonProperty + public TaskLocation getLocation() + { + return location; + } +} diff --git a/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java b/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java index fe4454ff1577..dc12c2d97cdc 100644 --- a/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java +++ b/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java @@ -85,13 +85,27 @@ void insert( */ List> getActiveEntriesWithStatus(); + default List getInactiveStatusesSince(DateTime timestamp) + { + return getInactiveStatusesSince(timestamp, null); + } + /** - * Return all statuses for inactive entries created on or later than the given timestamp + * Return up to {@code maxNumStatuses} statuses for inactive entries created on or later than the given timestamp * * @param timestamp timestamp + * @param maxNumStatuses maxNumStatuses * @return list of statuses */ - List getInactiveStatusesSince(DateTime timestamp); + List getInactiveStatusesSince(DateTime timestamp, @Nullable Integer maxNumStatuses); + + /** + * Return createdDate and dataSource for the given id + * + * @return a pair of createdDate and dataSource or null if an entry for the given id is not found + */ + @Nullable + Pair getCreatedDateAndDataSource(String entryId); /** * Add a lock to the given entry diff --git a/extensions-contrib/sqlserver-metadata-storage/src/main/java/io/druid/metadata/storage/sqlserver/SQLServerMetadataStorageModule.java b/extensions-contrib/sqlserver-metadata-storage/src/main/java/io/druid/metadata/storage/sqlserver/SQLServerMetadataStorageModule.java index cc79abb5c7ba..a3839f4534d9 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/main/java/io/druid/metadata/storage/sqlserver/SQLServerMetadataStorageModule.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/main/java/io/druid/metadata/storage/sqlserver/SQLServerMetadataStorageModule.java @@ -26,10 +26,12 @@ import io.druid.guice.PolyBind; import io.druid.guice.SQLMetadataStorageDruidModule; import io.druid.initialization.DruidModule; +import io.druid.metadata.MetadataStorageActionHandlerFactory; import io.druid.metadata.MetadataStorageConnector; import io.druid.metadata.MetadataStorageProvider; import io.druid.metadata.NoopMetadataStorageProvider; import io.druid.metadata.SQLMetadataConnector; +import io.druid.metadata.SQLServerMetadataStorageActionHandlerFactory; import java.util.List; @@ -72,5 +74,10 @@ public void configure(Binder binder) .addBinding(TYPE) .to(SQLServerConnector.class) .in(LazySingleton.class); + + PolyBind.optionBinder(binder, Key.get(MetadataStorageActionHandlerFactory.class)) + .addBinding(TYPE) + .to(SQLServerMetadataStorageActionHandlerFactory.class) + .in(LazySingleton.class); } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java index 8e39da6b8f26..30a06de8c17d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -38,7 +38,7 @@ import io.druid.indexing.common.RetryPolicyConfig; import io.druid.indexing.common.RetryPolicyFactory; import io.druid.indexing.common.TaskInfoProvider; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.java.util.common.IAE; import io.druid.java.util.common.IOE; diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 76468e4f50a3..927014d16fa2 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -47,7 +47,7 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.TaskInfoProvider; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.TaskResource; diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java index c4838a84a289..8d328aadbad2 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -32,7 +32,7 @@ import com.metamx.http.client.response.FullResponseHandler; import com.metamx.http.client.response.FullResponseHolder; import io.druid.indexing.common.TaskInfoProvider; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.DateTimes; diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 9d94127d4de3..66227735b36e 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -51,6 +51,7 @@ import io.druid.discovery.DataNodeService; import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.LookupNodeService; +import io.druid.indexer.TaskState; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; @@ -85,9 +86,9 @@ import io.druid.java.util.common.parsers.JSONPathFieldSpec; import io.druid.java.util.common.parsers.JSONPathSpec; import io.druid.math.expr.ExprMacroTable; +import io.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import io.druid.metadata.EntryExistsException; import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; -import io.druid.metadata.SQLMetadataStorageActionHandlerFactory; import io.druid.metadata.TestDerbyConnector; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Druids; @@ -115,8 +116,6 @@ import io.druid.segment.TestHelper; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.indexing.DataSchema; -import io.druid.segment.transform.ExpressionTransform; -import io.druid.segment.transform.TransformSpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.loading.LocalDataSegmentPusher; @@ -127,6 +126,8 @@ import io.druid.segment.realtime.appenderator.AppenderatorImpl; import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import io.druid.segment.transform.ExpressionTransform; +import io.druid.segment.transform.TransformSpec; import io.druid.server.DruidNode; import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.server.coordination.ServerType; @@ -356,7 +357,7 @@ public void testRunAfterDataInserted() throws Exception final ListenableFuture future = runTask(task); // Wait for task to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); @@ -410,7 +411,7 @@ public void testRunBeforeDataInserted() throws Exception } // Wait for task to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); @@ -478,7 +479,7 @@ public void testIncrementalHandOff() throws Exception Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets) || checkpoint2.getPartitionOffsetMap() .equals(currentOffsets)); task.setEndOffsets(currentOffsets, true, false); - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); Assert.assertEquals(1, checkpointRequestsHash.size()); Assert.assertTrue(checkpointRequestsHash.contains( @@ -554,7 +555,7 @@ public void testRunWithMinimumMessageTime() throws Exception } // Wait for task to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); @@ -608,7 +609,7 @@ public void testRunWithMaximumMessageTime() throws Exception } // Wait for task to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); @@ -672,7 +673,7 @@ public void testRunWithTransformSpec() throws Exception } // Wait for task to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics Assert.assertEquals(1, task.getFireDepartmentMetrics().processed()); @@ -720,7 +721,7 @@ public void testRunOnNothing() throws Exception final ListenableFuture future = runTask(task); // Wait for task to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics Assert.assertEquals(0, task.getFireDepartmentMetrics().processed()); @@ -761,7 +762,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception final ListenableFuture future = runTask(task); // Wait for task to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); @@ -814,7 +815,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio // Wait for task to exit Assert.assertEquals( - isIncrementalHandoffSupported ? TaskStatus.Status.SUCCESS : TaskStatus.Status.FAILED, + isIncrementalHandoffSupported ? TaskState.SUCCESS : TaskState.FAILED, future.get().getStatusCode() ); @@ -867,7 +868,7 @@ public void testReportParseExceptions() throws Exception final ListenableFuture future = runTask(task); // Wait for task to exit - Assert.assertEquals(TaskStatus.Status.FAILED, future.get().getStatusCode()); + Assert.assertEquals(TaskState.FAILED, future.get().getStatusCode()); // Check metrics Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); @@ -922,8 +923,8 @@ public void testRunReplicas() throws Exception } // Wait for tasks to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // Check metrics Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); @@ -988,11 +989,11 @@ public void testRunConflicting() throws Exception // Run first task final ListenableFuture future1 = runTask(task1); - Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); // Run second task final ListenableFuture future2 = runTask(task2); - Assert.assertEquals(TaskStatus.Status.FAILED, future2.get().getStatusCode()); + Assert.assertEquals(TaskState.FAILED, future2.get().getStatusCode()); // Check metrics Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); @@ -1057,7 +1058,7 @@ public void testRunConflictingWithoutTransactions() throws Exception // Run first task final ListenableFuture future1 = runTask(task1); - Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); // Check published segments & metadata SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); @@ -1067,7 +1068,7 @@ public void testRunConflictingWithoutTransactions() throws Exception // Run second task final ListenableFuture future2 = runTask(task2); - Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // Check metrics Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); @@ -1119,7 +1120,7 @@ public void testRunOneTaskTwoPartitions() throws Exception } // Wait for tasks to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics Assert.assertEquals(5, task.getFireDepartmentMetrics().processed()); @@ -1198,8 +1199,8 @@ public void testRunTwoTasksTwoPartitions() throws Exception } // Wait for tasks to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // Check metrics Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); @@ -1262,7 +1263,7 @@ public void testRestore() throws Exception task1.stopGracefully(); unlockAppenderatorBasePersistDirForTask(task1); - Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); // Start a new task final KafkaIndexTask task2 = createTask( @@ -1290,7 +1291,7 @@ public void testRestore() throws Exception } // Wait for task to exit - Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // Check metrics Assert.assertEquals(2, task1.getFireDepartmentMetrics().processed()); @@ -1376,7 +1377,7 @@ public void testRunWithPauseAndResume() throws Exception task.resume(); - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); Assert.assertEquals(task.getEndOffsets(), task.getCurrentOffsets()); // Check metrics @@ -1462,7 +1463,7 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception task.resume(); - Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics Assert.assertEquals(4, task.getFireDepartmentMetrics().processed()); @@ -1725,7 +1726,7 @@ private void makeToolboxFactory() throws IOException taskStorage = new MetadataTaskStorage( derbyConnector, new TaskStorageConfig(null), - new SQLMetadataStorageActionHandlerFactory( + new DerbyMetadataStorageActionHandlerFactory( derbyConnector, derby.metadataTablesConfigSupplier().get(), objectMapper diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index cd159edf2731..8f397952c3d0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -35,7 +35,7 @@ import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; import io.druid.indexing.common.TaskInfoProvider; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.RealtimeIndexTask; import io.druid.indexing.common.task.Task; diff --git a/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java b/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java index 1aad097dc772..211d5ca0b0e4 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/io/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java @@ -27,8 +27,10 @@ import io.druid.guice.PolyBind; import io.druid.guice.SQLMetadataStorageDruidModule; import io.druid.initialization.DruidModule; +import io.druid.metadata.MetadataStorageActionHandlerFactory; import io.druid.metadata.MetadataStorageConnector; import io.druid.metadata.MetadataStorageProvider; +import io.druid.metadata.MySQLMetadataStorageActionHandlerFactory; import io.druid.metadata.NoopMetadataStorageProvider; import io.druid.metadata.SQLMetadataConnector; @@ -71,5 +73,10 @@ public void configure(Binder binder) .addBinding(TYPE) .to(MySQLConnector.class) .in(LazySingleton.class); + + PolyBind.optionBinder(binder, Key.get(MetadataStorageActionHandlerFactory.class)) + .addBinding(TYPE) + .to(MySQLMetadataStorageActionHandlerFactory.class) + .in(LazySingleton.class); } } diff --git a/extensions-core/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLMetadataStorageModule.java b/extensions-core/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLMetadataStorageModule.java index 161022b3e10b..8c777b7070d2 100644 --- a/extensions-core/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLMetadataStorageModule.java +++ b/extensions-core/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLMetadataStorageModule.java @@ -27,9 +27,11 @@ import io.druid.guice.PolyBind; import io.druid.guice.SQLMetadataStorageDruidModule; import io.druid.initialization.DruidModule; +import io.druid.metadata.MetadataStorageActionHandlerFactory; import io.druid.metadata.MetadataStorageConnector; import io.druid.metadata.MetadataStorageProvider; import io.druid.metadata.NoopMetadataStorageProvider; +import io.druid.metadata.PostgreSQLMetadataStorageActionHandlerFactory; import io.druid.metadata.SQLMetadataConnector; import java.util.List; @@ -72,5 +74,10 @@ public void configure(Binder binder) .addBinding(TYPE) .to(PostgreSQLConnector.class) .in(LazySingleton.class); + + PolyBind.optionBinder(binder, Key.get(MetadataStorageActionHandlerFactory.class)) + .addBinding(TYPE) + .to(PostgreSQLMetadataStorageActionHandlerFactory.class) + .in(LazySingleton.class); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskInfoProvider.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskInfoProvider.java index 4095554982f1..5dbe96418498 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskInfoProvider.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskInfoProvider.java @@ -20,6 +20,7 @@ package io.druid.indexing.common; import com.google.common.base.Optional; +import io.druid.indexer.TaskLocation; public interface TaskInfoProvider { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java index cedc701bed2e..e3535957d58a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; +import io.druid.indexer.TaskState; /** * Represents the status of a task from the perspective of the coordinator. The task may be ongoing @@ -33,41 +34,34 @@ */ public class TaskStatus { - public enum Status - { - RUNNING, - SUCCESS, - FAILED - } - public static TaskStatus running(String taskId) { - return new TaskStatus(taskId, Status.RUNNING, -1); + return new TaskStatus(taskId, TaskState.RUNNING, -1); } public static TaskStatus success(String taskId) { - return new TaskStatus(taskId, Status.SUCCESS, -1); + return new TaskStatus(taskId, TaskState.SUCCESS, -1); } public static TaskStatus failure(String taskId) { - return new TaskStatus(taskId, Status.FAILED, -1); + return new TaskStatus(taskId, TaskState.FAILED, -1); } - public static TaskStatus fromCode(String taskId, Status code) + public static TaskStatus fromCode(String taskId, TaskState code) { return new TaskStatus(taskId, code, -1); } private final String id; - private final Status status; + private final TaskState status; private final long duration; @JsonCreator - private TaskStatus( + protected TaskStatus( @JsonProperty("id") String id, - @JsonProperty("status") Status status, + @JsonProperty("status") TaskState status, @JsonProperty("duration") long duration ) { @@ -87,7 +81,7 @@ public String getId() } @JsonProperty("status") - public Status getStatusCode() + public TaskState getStatusCode() { return status; } @@ -107,7 +101,7 @@ public long getDuration() @JsonIgnore public boolean isRunnable() { - return status == Status.RUNNING; + return status == TaskState.RUNNING; } /** @@ -130,7 +124,7 @@ public boolean isComplete() @JsonIgnore public boolean isSuccess() { - return status == Status.SUCCESS; + return status == TaskState.SUCCESS; } /** @@ -142,7 +136,7 @@ public boolean isSuccess() @JsonIgnore public boolean isFailure() { - return status == Status.FAILED; + return status == TaskState.FAILED; } public TaskStatus withDuration(long _duration) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java index ef73da4d9305..bc61d23045b8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentAllocateAction.java @@ -166,13 +166,12 @@ public SegmentIdentifier perform( ); final SegmentIdentifier identifier = usedSegmentsForRow.isEmpty() ? - tryAllocateFirstSegment(toolbox, task, rowInterval, skipSegmentLineageCheck) : + tryAllocateFirstSegment(toolbox, task, rowInterval) : tryAllocateSubsequentSegment( toolbox, task, rowInterval, - usedSegmentsForRow.iterator().next(), - skipSegmentLineageCheck + usedSegmentsForRow.iterator().next() ); if (identifier != null) { return identifier; @@ -212,12 +211,8 @@ public SegmentIdentifier perform( } } - private SegmentIdentifier tryAllocateFirstSegment( - TaskActionToolbox toolbox, - Task task, - Interval rowInterval, - boolean skipSegmentLineageCheck - ) throws IOException + private SegmentIdentifier tryAllocateFirstSegment(TaskActionToolbox toolbox, Task task, Interval rowInterval) + throws IOException { // No existing segments for this row, but there might still be nearby ones that conflict with our preferred // segment granularity. Try that first, and then progressively smaller ones if it fails. @@ -227,7 +222,7 @@ private SegmentIdentifier tryAllocateFirstSegment( .collect(Collectors.toList()); for (Interval tryInterval : tryIntervals) { if (tryInterval.contains(rowInterval)) { - final SegmentIdentifier identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false, skipSegmentLineageCheck); + final SegmentIdentifier identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false); if (identifier != null) { return identifier; } @@ -240,8 +235,7 @@ private SegmentIdentifier tryAllocateSubsequentSegment( TaskActionToolbox toolbox, Task task, Interval rowInterval, - DataSegment usedSegment, - boolean skipSegmentLineageCheck + DataSegment usedSegment ) throws IOException { // Existing segment(s) exist for this row; use the interval of the first one. @@ -251,7 +245,7 @@ private SegmentIdentifier tryAllocateSubsequentSegment( } else { // If segment allocation failed here, it is highly likely an unrecoverable error. We log here for easier // debugging. - return tryAllocate(toolbox, task, usedSegment.getInterval(), rowInterval, true, skipSegmentLineageCheck); + return tryAllocate(toolbox, task, usedSegment.getInterval(), rowInterval, true); } } @@ -260,8 +254,7 @@ private SegmentIdentifier tryAllocate( Task task, Interval tryInterval, Interval rowInterval, - boolean logOnFail, - boolean skipSegmentLineageCheck + boolean logOnFail ) throws IOException { log.debug( diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index d6e9e95cf59b..ee103420207d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -44,7 +44,7 @@ import com.metamx.emitter.EmittingLogger; import io.druid.java.util.common.concurrent.Execs; import io.druid.guice.annotations.Self; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java index 5510e6c9f47c..a464aee9b464 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -24,7 +24,6 @@ import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; @@ -35,13 +34,16 @@ import io.druid.indexing.common.config.TaskStorageConfig; import io.druid.indexing.common.task.Task; import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.EntryExistsException; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.util.List; import java.util.Map; import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; /** * Implements an in-heap TaskStorage facility, with no persistence across restarts. This class is not @@ -84,7 +86,7 @@ public void insert(Task task, TaskStatus status) throws EntryExistsException } log.info("Inserting task %s with status: %s", task.getId(), status); - tasks.put(task.getId(), new TaskStuff(task, status, DateTimes.nowUtc())); + tasks.put(task.getId(), new TaskStuff(task, status, DateTimes.nowUtc(), task.getDataSource())); } finally { giant.unlock(); @@ -166,13 +168,11 @@ public List getActiveTasks() } @Override - public List getRecentlyFinishedTaskStatuses() + public List getRecentlyFinishedTaskStatuses(@Nullable Integer maxTaskStatuses) { giant.lock(); try { - final List returns = Lists.newArrayList(); - final long recent = System.currentTimeMillis() - config.getRecentlyFinishedThreshold().getMillis(); final Ordering createdDateDesc = new Ordering() { @Override @@ -181,12 +181,61 @@ public int compare(TaskStuff a, TaskStuff b) return a.getCreatedDate().compareTo(b.getCreatedDate()); } }.reverse(); - for (final TaskStuff taskStuff : createdDateDesc.sortedCopy(tasks.values())) { - if (taskStuff.getStatus().isComplete() && taskStuff.getCreatedDate().getMillis() > recent) { - returns.add(taskStuff.getStatus()); - } - } - return returns; + + return maxTaskStatuses == null ? + getRecentlyFinishedTaskSTatusesSince( + System.currentTimeMillis() - config.getRecentlyFinishedThreshold().getMillis(), + createdDateDesc + ) : + getNRecentlyFinishedTaskStatuses(maxTaskStatuses, createdDateDesc); + } + finally { + giant.unlock(); + } + } + + private List getRecentlyFinishedTaskSTatusesSince(long start, Ordering createdDateDesc) + { + giant.lock(); + + try { + return createdDateDesc + .sortedCopy(tasks.values()) + .stream() + .filter(taskStuff -> taskStuff.getStatus().isComplete() && taskStuff.getCreatedDate().getMillis() > start) + .map(TaskStuff::getStatus) + .collect(Collectors.toList()); + } + finally { + giant.unlock(); + } + } + + private List getNRecentlyFinishedTaskStatuses(int n, Ordering createdDateDesc) + { + giant.lock(); + + try { + return createdDateDesc.sortedCopy(tasks.values()) + .stream() + .limit(n) + .map(TaskStuff::getStatus) + .collect(Collectors.toList()); + } + finally { + giant.unlock(); + } + } + + @Nullable + @Override + public Pair getCreatedDateTimeAndDataSource(String taskId) + { + giant.lock(); + + try { + final TaskStuff taskStuff = tasks.get(taskId); + return taskStuff == null ? null : Pair.of(taskStuff.getCreatedDate(), taskStuff.getDataSource()); } finally { giant.unlock(); @@ -287,16 +336,16 @@ private static class TaskStuff final Task task; final TaskStatus status; final DateTime createdDate; + final String dataSource; - private TaskStuff(Task task, TaskStatus status, DateTime createdDate) + private TaskStuff(Task task, TaskStatus status, DateTime createdDate, String dataSource) { - Preconditions.checkNotNull(task); - Preconditions.checkNotNull(status); Preconditions.checkArgument(task.getId().equals(status.getId())); - this.task = task; - this.status = status; + this.task = Preconditions.checkNotNull(task, "task"); + this.status = Preconditions.checkNotNull(status, "status"); this.createdDate = Preconditions.checkNotNull(createdDate, "createdDate"); + this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); } public Task getTask() @@ -314,9 +363,14 @@ public DateTime getCreatedDate() return createdDate; } + public String getDataSource() + { + return dataSource; + } + private TaskStuff withStatus(TaskStatus _status) { - return new TaskStuff(task, _status, createdDate); + return new TaskStuff(task, _status, createdDate, dataSource); } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java new file mode 100644 index 000000000000..e166aff6df73 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java @@ -0,0 +1,68 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.google.common.base.Preconditions; +import com.google.inject.Inject; +import io.druid.java.util.common.DateTimes; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import java.util.Comparator; +import java.util.Optional; + +public class IndexerMetadataStorageAdapter +{ + private final TaskStorageQueryAdapter taskStorageQueryAdapter; + private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + + @Inject + public IndexerMetadataStorageAdapter( + TaskStorageQueryAdapter taskStorageQueryAdapter, + IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator + ) + { + this.taskStorageQueryAdapter = taskStorageQueryAdapter; + this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; + } + + public int deletePendingSegments(String dataSource, Interval deleteInterval) + { + // Check the given interval overlaps the interval(minCreatedDateOfActiveTasks, MAX) + final Optional minCreatedDateOfActiveTasks = taskStorageQueryAdapter + .getActiveTasks() + .stream() + .map(task -> taskStorageQueryAdapter.getCreatedTime(task.getId())) + .min(Comparator.naturalOrder()); + + final Interval activeTaskInterval = new Interval( + minCreatedDateOfActiveTasks.orElse(DateTimes.MAX), + DateTimes.MAX + ); + + Preconditions.checkArgument( + !deleteInterval.overlaps(activeTaskInterval), + "Cannot delete pendingSegments because there is at least one running task created at %s", + activeTaskInterval.getStart() + ); + + return indexerMetadataStorageCoordinator.deletePendingSegments(dataSource, deleteInterval); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java index 2a3a3813bc42..479b92e92f38 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java @@ -50,6 +50,7 @@ import javax.annotation.Nullable; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public class MetadataTaskStorage implements TaskStorage { @@ -212,25 +213,27 @@ public Task apply(@Nullable Pair input) } @Override - public List getRecentlyFinishedTaskStatuses() + public List getRecentlyFinishedTaskStatuses(@Nullable Integer maxTaskStatuses) { - final DateTime start = DateTimes.nowUtc().minus(config.getRecentlyFinishedThreshold()); - return ImmutableList.copyOf( - Iterables.filter( - handler.getInactiveStatusesSince(start), - new Predicate() - { - @Override - public boolean apply(TaskStatus status) - { - return status.isComplete(); - } - } - ) + handler + .getInactiveStatusesSince( + DateTimes.nowUtc().minus(config.getRecentlyFinishedThreshold()), + maxTaskStatuses + ) + .stream() + .filter(TaskStatus::isComplete) + .collect(Collectors.toList()) ); } + @Nullable + @Override + public Pair getCreatedDateTimeAndDataSource(String taskId) + { + return handler.getCreatedDateAndDataSource(taskId); + } + @Override public void addLock(final String taskid, final TaskLock taskLock) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index 26cdfb718130..ca052d320aaa 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -53,7 +53,7 @@ import io.druid.concurrent.LifecycleLock; import io.druid.curator.CuratorUtils; import io.druid.curator.cache.PathChildrenCacheFactory; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.autoscaling.ProvisioningService; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java index 258f256159d8..3fecb075d81f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java @@ -20,7 +20,7 @@ package io.druid.indexing.overlord; import com.google.common.util.concurrent.SettableFuture; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.worker.Worker; import org.joda.time.DateTime; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerListener.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerListener.java index df79696e1394..79180411bd0d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerListener.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerListener.java @@ -19,7 +19,7 @@ package io.druid.indexing.overlord; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import java.util.concurrent.Executor; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java index c9f9b015d305..2a6d917d4d73 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java @@ -20,7 +20,7 @@ package io.druid.indexing.overlord; import com.metamx.emitter.EmittingLogger; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.java.util.common.Pair; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java index f3f3e3e066f9..2ec4a1473331 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.util.concurrent.ListenableFuture; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.java.util.common.DateTimes; import org.joda.time.DateTime; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java index c32fa844ac00..16e553fec688 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java @@ -24,8 +24,11 @@ import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.actions.TaskAction; import io.druid.indexing.common.task.Task; +import io.druid.java.util.common.Pair; import io.druid.metadata.EntryExistsException; +import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.util.List; public interface TaskStorage @@ -119,13 +122,16 @@ public interface TaskStorage List getActiveTasks(); /** - * Returns a list of recently finished task statuses as stored in the storage facility. No particular order - * is guaranteed, but implementations are encouraged to return tasks in descending order of creation. No particular - * standard of "recent" is guaranteed, and in fact, this method is permitted to simply return nothing. + * Returns up to {@code maxTaskStatuses} statuses of recently finished tasks as stored in the storage facility. No + * particular order is guaranteed, but implementations are encouraged to return tasks in descending order of creation. + * No particular standard of "recent" is guaranteed, and in fact, this method is permitted to simply return nothing. * * @return list of recently finished tasks */ - List getRecentlyFinishedTaskStatuses(); + List getRecentlyFinishedTaskStatuses(@Nullable Integer maxTaskStatuses); + + @Nullable + Pair getCreatedDateTimeAndDataSource(String taskId); /** * Returns a list of locks for a particular task. diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java index 29412ce961b1..469a5e7bff04 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -27,8 +27,11 @@ import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskAction; import io.druid.indexing.common.task.Task; +import io.druid.java.util.common.Pair; import io.druid.timeline.DataSegment; +import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.util.List; import java.util.Set; @@ -50,9 +53,16 @@ public List getActiveTasks() return storage.getActiveTasks(); } - public List getRecentlyFinishedTaskStatuses() + public List getRecentlyFinishedTaskStatuses(@Nullable Integer maxTaskStatuses) { - return storage.getRecentlyFinishedTaskStatuses(); + return storage.getRecentlyFinishedTaskStatuses(maxTaskStatuses); + } + + @Nullable + public DateTime getCreatedTime(String taskId) + { + final Pair pair = storage.getCreatedDateTimeAndDataSource(taskId); + return pair == null ? null : pair.lhs; } public Optional getTask(final String taskid) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java index c80925556cea..b7d7008b0937 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java @@ -35,7 +35,7 @@ import io.druid.java.util.common.concurrent.Execs; import io.druid.concurrent.TaskThreadPriority; import io.druid.guice.annotations.Self; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TaskToolboxFactory; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index bf905677f296..065bb99dcea4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -19,9 +19,9 @@ package io.druid.indexing.overlord.http; -import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.base.Function; import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -35,11 +35,13 @@ import io.druid.audit.AuditInfo; import io.druid.audit.AuditManager; import io.druid.common.config.JacksonConfigManager; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; +import io.druid.indexer.TaskStatusPlus; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.actions.TaskActionHolder; import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.IndexerMetadataStorageAdapter; import io.druid.indexing.overlord.TaskMaster; import io.druid.indexing.overlord.TaskQueue; import io.druid.indexing.overlord.TaskRunner; @@ -58,19 +60,19 @@ import io.druid.server.http.security.StateResourceFilter; import io.druid.server.security.Access; import io.druid.server.security.Action; -import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; import io.druid.tasklogs.TaskLogStreamer; import io.druid.timeline.DataSegment; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; import javax.ws.rs.HeaderParam; @@ -83,12 +85,14 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; /** */ @@ -99,6 +103,7 @@ public class OverlordResource private final TaskMaster taskMaster; private final TaskStorageQueryAdapter taskStorageQueryAdapter; + private final IndexerMetadataStorageAdapter indexerMetadataStorageAdapter; private final TaskLogStreamer taskLogStreamer; private final JacksonConfigManager configManager; private final AuditManager auditManager; @@ -110,6 +115,7 @@ public class OverlordResource public OverlordResource( TaskMaster taskMaster, TaskStorageQueryAdapter taskStorageQueryAdapter, + IndexerMetadataStorageAdapter indexerMetadataStorageAdapter, TaskLogStreamer taskLogStreamer, JacksonConfigManager configManager, AuditManager auditManager, @@ -118,6 +124,7 @@ public OverlordResource( { this.taskMaster = taskMaster; this.taskStorageQueryAdapter = taskStorageQueryAdapter; + this.indexerMetadataStorageAdapter = indexerMetadataStorageAdapter; this.taskLogStreamer = taskLogStreamer; this.configManager = configManager; this.auditManager = auditManager; @@ -402,7 +409,7 @@ public String apply(final TaskRunnerWorkItem workItem) // Would be nice to include the real created date, but the TaskStorage API doesn't yet allow it. new TaskRunnerWorkItem( task.getId(), - SettableFuture.create(), + SettableFuture.create(), DateTimes.EPOCH, DateTimes.EPOCH ) @@ -459,7 +466,10 @@ public Collection apply(TaskRunner taskRunner) @GET @Path("/completeTasks") @Produces(MediaType.APPLICATION_JSON) - public Response getCompleteTasks(@Context final HttpServletRequest req) + public Response getCompleteTasks( + @QueryParam("n") final Integer maxTaskStatuses, + @Context final HttpServletRequest req + ) { Function> raGenerator = taskStatus -> { final String taskId = taskStatus.getId(); @@ -483,33 +493,59 @@ public Response getCompleteTasks(@Context final HttpServletRequest req) final List recentlyFinishedTasks = Lists.newArrayList( AuthorizationUtils.filterAuthorizedResources( req, - taskStorageQueryAdapter.getRecentlyFinishedTaskStatuses(), + taskStorageQueryAdapter.getRecentlyFinishedTaskStatuses(maxTaskStatuses), raGenerator, authorizerMapper ) ); - final List completeTasks = Lists.transform( - recentlyFinishedTasks, - new Function() - { - @Override - public TaskResponseObject apply(TaskStatus taskStatus) - { - // Would be nice to include the real created date, but the TaskStorage API doesn't yet allow it. - return new TaskResponseObject( - taskStatus.getId(), - DateTimes.EPOCH, - DateTimes.EPOCH, - Optional.of(taskStatus), - TaskLocation.unknown() - ); - } - } - ); + final List completeTasks = recentlyFinishedTasks + .stream() + .map(status -> new TaskStatusPlus( + status.getId(), + taskStorageQueryAdapter.getCreatedTime(status.getId()), + DateTimes.EPOCH, + status.getStatusCode(), + status.getDuration(), + TaskLocation.unknown()) + ) + .collect(Collectors.toList()); + return Response.ok(completeTasks).build(); } + @DELETE + @Path("/pendingSegments/{dataSource}") + @Produces(MediaType.APPLICATION_JSON) + public Response killPendingSegments( + @PathParam("dataSource") String dataSource, + @QueryParam("interval") String deleteIntervalString, + @Context HttpServletRequest request + ) + { + final Interval deleteInterval = Intervals.of(deleteIntervalString); + // check auth for dataSource + final Access authResult = AuthorizationUtils.authorizeAllResourceActions( + request, + ImmutableList.of( + new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.READ), + new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.WRITE) + ), + authorizerMapper + ); + + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } + + if (taskMaster.isLeader()) { + final int numDeleted = indexerMetadataStorageAdapter.deletePendingSegments(dataSource, deleteInterval); + return Response.ok().entity(ImmutableMap.of("numDeleted", numDeleted)).build(); + } else { + return Response.status(Status.SERVICE_UNAVAILABLE).build(); + } + } + @GET @Path("/workers") @Produces(MediaType.APPLICATION_JSON) @@ -595,16 +631,17 @@ public Response apply(TaskRunner taskRunner) return Response.ok( Lists.transform( Lists.newArrayList(fn.apply(taskRunner)), - new Function() + new Function() { @Override - public TaskResponseObject apply(TaskRunnerWorkItem workItem) + public TaskStatusPlus apply(TaskRunnerWorkItem workItem) { - return new TaskResponseObject( + return new TaskStatusPlus( workItem.getTaskId(), workItem.getCreatedTime(), workItem.getQueueInsertionTime(), - Optional.absent(), + null, + null, workItem.getLocation() ); } @@ -671,51 +708,4 @@ private Collection securedTaskRunnerWorkItem( ) ); } - - static class TaskResponseObject - { - private final String id; - private final DateTime createdTime; - private final DateTime queueInsertionTime; - private final Optional status; - private final TaskLocation location; - - private TaskResponseObject( - String id, - DateTime createdTime, - DateTime queueInsertionTime, - Optional status, - TaskLocation location - ) - { - this.id = id; - this.createdTime = createdTime; - this.queueInsertionTime = queueInsertionTime; - this.status = status; - this.location = location; - } - - @JsonValue - public Map toJson() - { - final Map data = Maps.newLinkedHashMap(); - data.put("id", id); - if (createdTime.getMillis() > 0) { - data.put("createdTime", createdTime); - } - if (queueInsertionTime.getMillis() > 0) { - data.put("queueInsertionTime", queueInsertionTime); - } - if (status.isPresent()) { - data.put("statusCode", status.get().getStatusCode().toString()); - if (status.get().isComplete()) { - data.put("duration", status.get().getDuration()); - } - } - if (location != null) { - data.put("location", location); - } - return data; - } - } } diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java b/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java index a59e79228051..8c3cd661b268 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java @@ -22,7 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; +import io.druid.indexer.TaskState; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.TaskResource; @@ -50,7 +51,7 @@ public static TaskAnnouncement create(String taskId, TaskResource resource, Task @JsonCreator private TaskAnnouncement( @JsonProperty("id") String taskId, - @JsonProperty("status") TaskStatus.Status status, + @JsonProperty("status") TaskState status, @JsonProperty("taskStatus") TaskStatus taskStatus, @JsonProperty("taskResource") TaskResource taskResource, @JsonProperty("taskLocation") TaskLocation taskLocation @@ -77,7 +78,7 @@ public String getTaskId() // Can be removed when backwards compat is no longer needed @JsonProperty("status") @Deprecated - public TaskStatus.Status getStatus() + public TaskState getStatus() { return taskStatus.getStatusCode(); } diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java index 1d085b22c8f2..040dbf3500ee 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java @@ -28,7 +28,7 @@ import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; import io.druid.java.util.common.concurrent.Execs; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.TaskRunner; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index bc0b0328ecba..bf1406937d81 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -47,6 +47,7 @@ import io.druid.discovery.DataNodeService; import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.LookupNodeService; +import io.druid.indexer.TaskState; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; @@ -374,7 +375,7 @@ public void testBasics() throws Exception // Wait for the task to finish. final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); } @Test(timeout = 60_000L) @@ -445,7 +446,7 @@ public void testTransformSpec() throws Exception // Wait for the task to finish. final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); } @Test(timeout = 60_000L) @@ -572,7 +573,7 @@ public void testNoReportParseExceptions() throws Exception // Wait for the task to finish. final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); } @Test(timeout = 60_000L) @@ -606,7 +607,7 @@ public void testRestore() throws Exception // Wait for the task to finish. The status doesn't really matter, but we'll check it anyway. final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); // Nothing should be published. Assert.assertEquals(Sets.newHashSet(), mdc.getPublished()); @@ -665,7 +666,7 @@ public void testRestore() throws Exception // Wait for the task to finish. final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); } } @@ -760,7 +761,7 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception // Wait for the task to finish. final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); } } @@ -794,7 +795,7 @@ public void testRestoreCorruptData() throws Exception // Wait for the task to finish. The status doesn't really matter, but we'll check it anyway. final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); // Nothing should be published. Assert.assertEquals(Sets.newHashSet(), mdc.getPublished()); @@ -846,7 +847,7 @@ public void testStopBeforeStarting() throws Exception // Wait for the task to finish. final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); } private ListenableFuture runTask(final Task task, final TaskToolbox toolbox) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java new file mode 100644 index 000000000000..92ff99abee0b --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java @@ -0,0 +1,98 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.google.common.collect.ImmutableList; +import io.druid.indexing.common.task.NoopTask; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.Intervals; +import org.easymock.EasyMock; +import org.hamcrest.CoreMatchers; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class IndexerMetadataStorageAdapterTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private TaskStorageQueryAdapter taskStorageQueryAdapter; + private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + private IndexerMetadataStorageAdapter indexerMetadataStorageAdapter; + + @Before + public void setup() + { + indexerMetadataStorageCoordinator = EasyMock.strictMock(IndexerMetadataStorageCoordinator.class); + taskStorageQueryAdapter = EasyMock.strictMock(TaskStorageQueryAdapter.class); + indexerMetadataStorageAdapter = new IndexerMetadataStorageAdapter( + taskStorageQueryAdapter, + indexerMetadataStorageCoordinator + ); + } + + @Test + public void testDeletePendingSegments() + { + EasyMock.expect(taskStorageQueryAdapter.getActiveTasks()) + .andReturn(ImmutableList.of(NoopTask.create("id1", 0), NoopTask.create("id2", 0))); + EasyMock.expect(taskStorageQueryAdapter.getCreatedTime(EasyMock.eq("id1"))) + .andReturn(DateTimes.of("2017-12-01")); + EasyMock.expect(taskStorageQueryAdapter.getCreatedTime(EasyMock.eq("id2"))) + .andReturn(DateTimes.of("2017-12-02")); + + final Interval deleteInterval = Intervals.of("2017-01-01/2017-12-01"); + EasyMock + .expect( + indexerMetadataStorageCoordinator.deletePendingSegments(EasyMock.anyString(), EasyMock.eq(deleteInterval)) + ) + .andReturn(10); + EasyMock.replay(taskStorageQueryAdapter, indexerMetadataStorageCoordinator); + + Assert.assertEquals(10, indexerMetadataStorageAdapter.deletePendingSegments("dataSource", deleteInterval)); + } + + @Test + public void testDeletePendingSegmentsOfRunningTasks() + { + EasyMock.expect(taskStorageQueryAdapter.getActiveTasks()) + .andReturn(ImmutableList.of(NoopTask.create("id1", 0), NoopTask.create("id2", 0))); + EasyMock.expect(taskStorageQueryAdapter.getCreatedTime(EasyMock.eq("id1"))) + .andReturn(DateTimes.of("2017-11-01")); + EasyMock.expect(taskStorageQueryAdapter.getCreatedTime(EasyMock.eq("id2"))) + .andReturn(DateTimes.of("2017-12-02")); + + final Interval deleteInterval = Intervals.of("2017-01-01/2017-12-01"); + EasyMock + .expect( + indexerMetadataStorageCoordinator.deletePendingSegments(EasyMock.anyString(), EasyMock.eq(deleteInterval)) + ) + .andReturn(10); + EasyMock.replay(taskStorageQueryAdapter, indexerMetadataStorageCoordinator); + + expectedException.expect(CoreMatchers.instanceOf(IllegalArgumentException.class)); + expectedException.expectMessage("Cannot delete pendingSegments because there is at least one running task created"); + indexerMetadataStorageAdapter.deletePendingSegments("dataSource", deleteInterval); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerRunPendingTasksConcurrencyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerRunPendingTasksConcurrencyTest.java index 50e492b4ffb6..c016065284e0 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerRunPendingTasksConcurrencyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerRunPendingTasksConcurrencyTest.java @@ -21,6 +21,7 @@ import com.google.common.util.concurrent.ListenableFuture; +import io.druid.indexer.TaskState; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TestTasks; import io.druid.indexing.common.task.Task; @@ -94,8 +95,8 @@ public int getPendingTasksRunnerNumThreads() //simulate completion of task0 and task1 mockWorkerRunningAndCompletionSuccessfulTasks(tasks[0], tasks[1]); - Assert.assertEquals(TaskStatus.Status.SUCCESS, results[0].get().getStatusCode()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, results[1].get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, results[0].get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, results[1].get().getStatusCode()); // now both threads race to run the last 3 tasks. task2 and task3 are being assigned waitForBothWorkersToHaveUnackedTasks(); @@ -104,20 +105,20 @@ public int getPendingTasksRunnerNumThreads() && remoteTaskRunner.getWorkersWithUnacknowledgedTask().containsValue(tasks[3].getId())) { remoteTaskRunner.shutdown("task4"); mockWorkerRunningAndCompletionSuccessfulTasks(tasks[3], tasks[2]); - Assert.assertEquals(TaskStatus.Status.SUCCESS, results[3].get().getStatusCode()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, results[2].get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, results[3].get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, results[2].get().getStatusCode()); } else if (remoteTaskRunner.getWorkersWithUnacknowledgedTask().containsValue(tasks[3].getId()) && remoteTaskRunner.getWorkersWithUnacknowledgedTask().containsValue(tasks[4].getId())) { remoteTaskRunner.shutdown("task2"); mockWorkerRunningAndCompletionSuccessfulTasks(tasks[4], tasks[3]); - Assert.assertEquals(TaskStatus.Status.SUCCESS, results[4].get().getStatusCode()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, results[3].get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, results[4].get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, results[3].get().getStatusCode()); } else if (remoteTaskRunner.getWorkersWithUnacknowledgedTask().containsValue(tasks[4].getId()) && remoteTaskRunner.getWorkersWithUnacknowledgedTask().containsValue(tasks[2].getId())) { remoteTaskRunner.shutdown("task3"); mockWorkerRunningAndCompletionSuccessfulTasks(tasks[4], tasks[2]); - Assert.assertEquals(TaskStatus.Status.SUCCESS, results[4].get().getStatusCode()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, results[2].get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, results[4].get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, results[2].get().getStatusCode()); } else { throw new ISE("two out of three tasks 2,3 and 4 must be waiting for ack."); } @@ -133,7 +134,7 @@ public int getPendingTasksRunnerNumThreads() rtrTestUtils.mockWorkerRunningTask("worker1", tasks[5]); rtrTestUtils.mockWorkerCompleteSuccessfulTask("worker1", tasks[5]); } - Assert.assertEquals(TaskStatus.Status.SUCCESS, results[5].get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, results[5].get().getStatusCode()); } private void mockWorkerRunningAndCompletionSuccessfulTasks(Task t1, Task t2) throws Exception diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java index 98eeea9657a6..c2a199f872d3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -30,6 +30,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.indexer.TaskState; import io.druid.indexing.common.IndexingServiceCondition; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TestRealtimeTask; @@ -105,7 +106,7 @@ public void testRun() throws Exception Assert.assertTrue(workerCompletedTask(result)); Assert.assertEquals(task.getId(), result.get().getId()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, result.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, result.get().getStatusCode()); } @Test @@ -131,7 +132,7 @@ public void testRunExistingTaskThatHasntStartedRunning() throws Exception Assert.assertTrue(workerCompletedTask(result)); Assert.assertEquals(task.getId(), result.get().getId()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, result.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, result.get().getStatusCode()); } @Test @@ -152,7 +153,7 @@ public void testRunExistingTaskThatHasStartedRunning() throws Exception Assert.assertTrue(workerCompletedTask(result)); Assert.assertEquals(task.getId(), result.get().getId()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, result.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, result.get().getStatusCode()); } @Test @@ -314,7 +315,7 @@ public void testStatusRemoved() throws Exception TaskStatus status = future.get(TIMEOUT_SECONDS, TimeUnit.SECONDS); - Assert.assertEquals(status.getStatusCode(), TaskStatus.Status.FAILED); + Assert.assertEquals(status.getStatusCode(), TaskState.FAILED); } @Test @@ -367,7 +368,7 @@ public void testRunWithTaskComplete() throws Exception TaskStatus status = future.get(TIMEOUT_SECONDS, TimeUnit.SECONDS); - Assert.assertEquals(TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); } @Test @@ -385,7 +386,7 @@ public void testWorkerRemoved() throws Exception TaskStatus status = future.get(TIMEOUT_SECONDS, TimeUnit.SECONDS); - Assert.assertEquals(TaskStatus.Status.FAILED, status.getStatusCode()); + Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); RemoteTaskRunnerConfig config = remoteTaskRunner.getRemoteTaskRunnerConfig(); Assert.assertTrue( TestUtils.conditionValid( @@ -421,7 +422,7 @@ public void testWorkerDisabled() throws Exception mockWorkerCompleteSuccessfulTask(task); Assert.assertTrue(workerCompletedTask(result)); Assert.assertEquals(task.getId(), result.get().getId()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, result.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, result.get().getStatusCode()); // Confirm RTR thinks the worker is disabled. Assert.assertEquals("", Iterables.getOnlyElement(remoteTaskRunner.getWorkers()).getWorker().getVersion()); @@ -595,8 +596,8 @@ public boolean isValid() mockWorkerCompleteSuccessfulTask(task); TaskStatus status = future.get(TIMEOUT_SECONDS, TimeUnit.SECONDS); - Assert.assertEquals(status.getStatusCode(), TaskStatus.Status.SUCCESS); - Assert.assertEquals(TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals(status.getStatusCode(), TaskState.SUCCESS); + Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); } @Test diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTestUtils.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTestUtils.java index e9cbd3b01788..435a5b4a6fc7 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTestUtils.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTestUtils.java @@ -28,7 +28,7 @@ import io.druid.curator.PotentiallyGzippedCompressionProvider; import io.druid.curator.cache.PathChildrenCacheFactory; import io.druid.indexing.common.IndexingServiceCondition; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.task.Task; diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 42b51ec5168e..e408bd28c2db 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -47,6 +47,7 @@ import io.druid.discovery.DataNodeService; import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.LookupNodeService; +import io.druid.indexer.TaskState; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; @@ -81,7 +82,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Comparators; -import io.druid.metadata.SQLMetadataStorageActionHandlerFactory; +import io.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import io.druid.metadata.TestDerbyConnector; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.SegmentDescriptor; @@ -406,7 +407,7 @@ private TaskStorage setUpTaskStorage() taskStorage = new MetadataTaskStorage( testDerbyConnector, new TaskStorageConfig(null), - new SQLMetadataStorageActionHandlerFactory( + new DerbyMetadataStorageActionHandlerFactory( testDerbyConnector, derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper @@ -683,8 +684,8 @@ public void testIndexTask() throws Exception final List publishedSegments = byIntervalOrdering.sortedCopy(mdc.getPublished()); final List loggedSegments = byIntervalOrdering.sortedCopy(tsqa.getInsertedSegments(indexTask.getId())); - Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); - Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode()); + Assert.assertEquals("statusCode", TaskState.SUCCESS, status.getStatusCode()); + Assert.assertEquals("merged statusCode", TaskState.SUCCESS, mergedStatus.getStatusCode()); Assert.assertEquals("segments logged vs published", loggedSegments, publishedSegments); Assert.assertEquals("num segments published", 2, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); @@ -735,7 +736,7 @@ public void testIndexTaskFailure() throws Exception final TaskStatus status = runTask(indexTask); - Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode()); + Assert.assertEquals("statusCode", TaskState.FAILED, status.getStatusCode()); Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); } @@ -803,7 +804,7 @@ public DataSegment apply(String input) final Task killTask = new KillTask(null, "test_kill_task", Intervals.of("2011-04-01/P4D"), null); final TaskStatus status = runTask(killTask); - Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("merged statusCode", TaskState.SUCCESS, status.getStatusCode()); Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 3, mdc.getNuked().size()); Assert.assertTrue( @@ -824,7 +825,7 @@ public void testRealtimeishTask() throws Exception final Task rtishTask = new RealtimeishTask(); final TaskStatus status = runTask(rtishTask); - Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("statusCode", TaskState.SUCCESS, status.getStatusCode()); Assert.assertEquals("num segments published", 2, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); } @@ -838,7 +839,7 @@ public void testNoopTask() throws Exception ); final TaskStatus status = runTask(noopTask); - Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("statusCode", TaskState.SUCCESS, status.getStatusCode()); Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); } @@ -852,7 +853,7 @@ public void testNeverReadyTask() throws Exception ); final TaskStatus status = runTask(neverReadyTask); - Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode()); + Assert.assertEquals("statusCode", TaskState.FAILED, status.getStatusCode()); Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); } @@ -896,7 +897,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception final TaskStatus status = runTask(task); - Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("statusCode", TaskState.SUCCESS, status.getStatusCode()); Assert.assertEquals("segments published", 1, mdc.getPublished().size()); Assert.assertEquals("segments nuked", 0, mdc.getNuked().size()); } @@ -930,7 +931,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception final TaskStatus status = runTask(task); - Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode()); + Assert.assertEquals("statusCode", TaskState.FAILED, status.getStatusCode()); Assert.assertEquals("segments published", 0, mdc.getPublished().size()); Assert.assertEquals("segments nuked", 0, mdc.getNuked().size()); } @@ -964,7 +965,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception final TaskStatus status = runTask(task); - Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode()); + Assert.assertEquals("statusCode", TaskState.FAILED, status.getStatusCode()); Assert.assertEquals("segments published", 0, mdc.getPublished().size()); Assert.assertEquals("segments nuked", 0, mdc.getNuked().size()); } @@ -1116,7 +1117,7 @@ public void testResumeTasks() throws Exception final List publishedSegments = byIntervalOrdering.sortedCopy(mdc.getPublished()); final List loggedSegments = byIntervalOrdering.sortedCopy(tsqa.getInsertedSegments(indexTask.getId())); - Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("statusCode", TaskState.SUCCESS, status.getStatusCode()); Assert.assertEquals("segments logged vs published", loggedSegments, publishedSegments); Assert.assertEquals("num segments published", 2, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index 4bc7e2a2a8d6..e84a118ae068 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -29,8 +29,8 @@ import io.druid.indexing.common.task.Task; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; +import io.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import io.druid.metadata.EntryExistsException; -import io.druid.metadata.SQLMetadataStorageActionHandlerFactory; import io.druid.metadata.TestDerbyConnector; import org.joda.time.Interval; import org.junit.After; @@ -65,7 +65,7 @@ public void setup() taskStorage = new MetadataTaskStorage( derbyConnector, new TaskStorageConfig(null), - new SQLMetadataStorageActionHandlerFactory( + new DerbyMetadataStorageActionHandlerFactory( derbyConnector, derby.metadataTablesConfigSupplier().get(), objectMapper diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java index 0271894f219c..11266b60def0 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLockboxTest.java @@ -33,8 +33,8 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; +import io.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import io.druid.metadata.EntryExistsException; -import io.druid.metadata.SQLMetadataStorageActionHandlerFactory; import io.druid.metadata.TestDerbyConnector; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -74,7 +74,7 @@ public void setup() taskStorage = new MetadataTaskStorage( derbyConnector, new TaskStorageConfig(null), - new SQLMetadataStorageActionHandlerFactory( + new DerbyMetadataStorageActionHandlerFactory( derbyConnector, derby.metadataTablesConfigSupplier().get(), objectMapper diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java index a3176d46f482..e6afde258183 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java @@ -28,7 +28,7 @@ import com.metamx.emitter.service.ServiceEventBuilder; import io.druid.common.guava.DSuppliers; import io.druid.java.util.common.concurrent.Execs; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TestTasks; import io.druid.indexing.common.task.NoopTask; diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java index da2218223054..f917d359c64a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java @@ -29,7 +29,7 @@ import com.metamx.emitter.service.ServiceEventBuilder; import io.druid.common.guava.DSuppliers; import io.druid.java.util.common.concurrent.Execs; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TestTasks; import io.druid.indexing.common.task.NoopTask; diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java index 7f033a23b8ac..4c338c9364d5 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java @@ -25,17 +25,20 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListenableFuture; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; +import io.druid.indexer.TaskStatusPlus; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.task.AbstractTask; import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.IndexerMetadataStorageAdapter; import io.druid.indexing.overlord.TaskMaster; import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.overlord.TaskRunnerWorkItem; import io.druid.indexing.overlord.TaskStorageQueryAdapter; +import io.druid.java.util.common.DateTimes; import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; @@ -45,6 +48,7 @@ import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import org.easymock.EasyMock; +import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -56,12 +60,14 @@ import javax.ws.rs.core.Response; import java.util.Collection; import java.util.List; +import java.util.Map; public class OverlordResourceTest { private OverlordResource overlordResource; private TaskMaster taskMaster; - private TaskStorageQueryAdapter tsqa; + private TaskStorageQueryAdapter taskStorageQueryAdapter; + private IndexerMetadataStorageAdapter indexerMetadataStorageAdapter; private HttpServletRequest req; private TaskRunner taskRunner; @@ -73,7 +79,8 @@ public void setUp() throws Exception { taskRunner = EasyMock.createMock(TaskRunner.class); taskMaster = EasyMock.createStrictMock(TaskMaster.class); - tsqa = EasyMock.createStrictMock(TaskStorageQueryAdapter.class); + taskStorageQueryAdapter = EasyMock.createStrictMock(TaskStorageQueryAdapter.class); + indexerMetadataStorageAdapter = EasyMock.createStrictMock(IndexerMetadataStorageAdapter.class); req = EasyMock.createStrictMock(HttpServletRequest.class); EasyMock.expect(taskMaster.getTaskRunner()).andReturn( @@ -102,7 +109,8 @@ public Access authorize(AuthenticationResult authenticationResult, Resource reso overlordResource = new OverlordResource( taskMaster, - tsqa, + taskStorageQueryAdapter, + indexerMetadataStorageAdapter, null, null, null, @@ -130,7 +138,7 @@ public void expectAuthorizationTokenCheck() public void testLeader() { EasyMock.expect(taskMaster.getCurrentLeader()).andReturn("boz").once(); - EasyMock.replay(taskRunner, taskMaster, tsqa, req); + EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); final Response response = overlordResource.getLeader(); Assert.assertEquals("boz", response.getEntity()); @@ -142,7 +150,7 @@ public void testIsLeader() { EasyMock.expect(taskMaster.isLeader()).andReturn(true).once(); EasyMock.expect(taskMaster.isLeader()).andReturn(false).once(); - EasyMock.replay(taskRunner, taskMaster, tsqa, req); + EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); // true final Response response1 = overlordResource.isLeader(); @@ -160,7 +168,7 @@ public void testSecuredGetWaitingTask() throws Exception { expectAuthorizationTokenCheck(); - EasyMock.expect(tsqa.getActiveTasks()).andReturn( + EasyMock.expect(taskStorageQueryAdapter.getActiveTasks()).andReturn( ImmutableList.of( getTaskWithIdAndDatasource("id_1", "allow"), getTaskWithIdAndDatasource("id_2", "allow"), @@ -176,12 +184,12 @@ public void testSecuredGetWaitingTask() throws Exception ) ); - EasyMock.replay(taskRunner, taskMaster, tsqa, req); + EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); - List responseObjects = (List) overlordResource.getWaitingTasks(req) - .getEntity(); + List responseObjects = (List) overlordResource.getWaitingTasks(req) + .getEntity(); Assert.assertEquals(1, responseObjects.size()); - Assert.assertEquals("id_2", responseObjects.get(0).toJson().get("id")); + Assert.assertEquals("id_2", responseObjects.get(0).getId()); } @Test @@ -190,7 +198,7 @@ public void testSecuredGetCompleteTasks() expectAuthorizationTokenCheck(); List tasksIds = ImmutableList.of("id_1", "id_2", "id_3"); - EasyMock.expect(tsqa.getRecentlyFinishedTaskStatuses()).andReturn( + EasyMock.expect(taskStorageQueryAdapter.getRecentlyFinishedTaskStatuses(null)).andReturn( Lists.transform( tasksIds, new Function() @@ -204,23 +212,26 @@ public TaskStatus apply(String input) ) ).once(); - EasyMock.expect(tsqa.getTask(tasksIds.get(0))).andReturn( + EasyMock.expect(taskStorageQueryAdapter.getTask(tasksIds.get(0))).andReturn( Optional.of(getTaskWithIdAndDatasource(tasksIds.get(0), "deny")) ).once(); - EasyMock.expect(tsqa.getTask(tasksIds.get(1))).andReturn( + EasyMock.expect(taskStorageQueryAdapter.getTask(tasksIds.get(1))).andReturn( Optional.of(getTaskWithIdAndDatasource(tasksIds.get(1), "allow")) ).once(); - EasyMock.expect(tsqa.getTask(tasksIds.get(2))).andReturn( + EasyMock.expect(taskStorageQueryAdapter.getTask(tasksIds.get(2))).andReturn( Optional.of(getTaskWithIdAndDatasource(tasksIds.get(2), "allow")) ).once(); - EasyMock.replay(taskRunner, taskMaster, tsqa, req); + EasyMock.expect(taskStorageQueryAdapter.getCreatedTime(EasyMock.anyString())) + .andReturn(DateTimes.EPOCH) + .anyTimes(); + EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); - List responseObjects = (List) overlordResource.getCompleteTasks(req) - .getEntity(); + List responseObjects = (List) overlordResource.getCompleteTasks(null, req) + .getEntity(); Assert.assertEquals(2, responseObjects.size()); - Assert.assertEquals(tasksIds.get(1), responseObjects.get(0).toJson().get("id")); - Assert.assertEquals(tasksIds.get(2), responseObjects.get(1).toJson().get("id")); + Assert.assertEquals(tasksIds.get(1), responseObjects.get(0).getId()); + Assert.assertEquals(tasksIds.get(2), responseObjects.get(1).getId()); } @Test @@ -235,20 +246,20 @@ public void testSecuredGetRunningTasks() new MockTaskRunnerWorkItem(tasksIds.get(1), null) ) ); - EasyMock.expect(tsqa.getTask(tasksIds.get(0))).andReturn( + EasyMock.expect(taskStorageQueryAdapter.getTask(tasksIds.get(0))).andReturn( Optional.of(getTaskWithIdAndDatasource(tasksIds.get(0), "deny")) ).once(); - EasyMock.expect(tsqa.getTask(tasksIds.get(1))).andReturn( + EasyMock.expect(taskStorageQueryAdapter.getTask(tasksIds.get(1))).andReturn( Optional.of(getTaskWithIdAndDatasource(tasksIds.get(1), "allow")) ).once(); - EasyMock.replay(taskRunner, taskMaster, tsqa, req); + EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); - List responseObjects = (List) overlordResource.getRunningTasks(req) - .getEntity(); + List responseObjects = (List) overlordResource.getRunningTasks(req) + .getEntity(); Assert.assertEquals(1, responseObjects.size()); - Assert.assertEquals(tasksIds.get(1), responseObjects.get(0).toJson().get("id")); + Assert.assertEquals(tasksIds.get(1), responseObjects.get(0).getId()); } @Test @@ -258,15 +269,33 @@ public void testSecuredTaskPost() expectedException.expectMessage("Allowed:false, Message:"); expectAuthorizationTokenCheck(); - EasyMock.replay(taskRunner, taskMaster, tsqa, req); + EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); Task task = NoopTask.create(); overlordResource.taskPost(task, req); } + @Test + public void testKillPendingSegments() + { + expectAuthorizationTokenCheck(); + + EasyMock.expect(taskMaster.isLeader()).andReturn(true); + EasyMock + .expect(indexerMetadataStorageAdapter.deletePendingSegments(EasyMock.eq("allow"), EasyMock.anyObject(Interval.class))) + .andReturn(2); + + EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); + + final Map response = (Map) overlordResource + .killPendingSegments("allow", new Interval(DateTimes.MIN, DateTimes.nowUtc()).toString(), req) + .getEntity(); + Assert.assertEquals(2, response.get("numDeleted").intValue()); + } + @After public void tearDown() { - EasyMock.verify(taskRunner, taskMaster, tsqa, req); + EasyMock.verify(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); } private Task getTaskWithIdAndDatasource(String id, String datasource) @@ -306,7 +335,7 @@ public MockTaskRunnerWorkItem( @Override public TaskLocation getLocation() { - return null; + return TaskLocation.unknown(); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java index 09366c827654..b88556cf4fc3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java @@ -28,17 +28,19 @@ import com.google.common.util.concurrent.MoreExecutors; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; -import io.druid.java.util.common.concurrent.Execs; import io.druid.curator.PotentiallyGzippedCompressionProvider; import io.druid.curator.discovery.NoopServiceAnnouncer; import io.druid.discovery.DruidLeaderSelector; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; +import io.druid.indexer.TaskState; +import io.druid.indexer.TaskStatusPlus; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskStorageConfig; import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.HeapMemoryTaskStorage; +import io.druid.indexing.overlord.IndexerMetadataStorageAdapter; import io.druid.indexing.overlord.TaskLockbox; import io.druid.indexing.overlord.TaskMaster; import io.druid.indexing.overlord.TaskRunner; @@ -52,11 +54,10 @@ import io.druid.indexing.overlord.helpers.OverlordHelperManager; import io.druid.indexing.overlord.supervisor.SupervisorManager; import io.druid.java.util.common.Pair; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.server.DruidNode; import io.druid.server.coordinator.CoordinatorOverlordServiceConfig; -import io.druid.server.initialization.IndexerZkConfig; -import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthTestUtils; @@ -162,7 +163,6 @@ public void setUp() throws Exception taskCompletionCountDownLatches[0] = new CountDownLatch(1); taskCompletionCountDownLatches[1] = new CountDownLatch(1); announcementLatch = new CountDownLatch(1); - IndexerZkConfig indexerZkConfig = new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null); setupServerAndCurator(); curator.start(); curator.blockUntilConnected(); @@ -211,10 +211,12 @@ public void testOverlordRun() throws Exception } Assert.assertEquals(taskMaster.getCurrentLeader(), druidNode.getHostAndPort()); + final TaskStorageQueryAdapter taskStorageQueryAdapter = new TaskStorageQueryAdapter(taskStorage); // Test Overlord resource stuff overlordResource = new OverlordResource( taskMaster, - new TaskStorageQueryAdapter(taskStorage), + taskStorageQueryAdapter, + new IndexerMetadataStorageAdapter(taskStorageQueryAdapter, null), null, null, null, @@ -252,7 +254,7 @@ public void testOverlordRun() throws Exception // Simulate completion of task_0 taskCompletionCountDownLatches[Integer.parseInt(taskId_0)].countDown(); // Wait for taskQueue to handle success status of task_0 - waitForTaskStatus(taskId_0, TaskStatus.Status.SUCCESS); + waitForTaskStatus(taskId_0, TaskState.SUCCESS); // Manually insert task in taskStorage // Verifies sync from storage @@ -265,19 +267,22 @@ public void testOverlordRun() throws Exception response = overlordResource.getRunningTasks(req); // 1 task that was manually inserted should be in running state Assert.assertEquals(1, (((List) response.getEntity()).size())); - final OverlordResource.TaskResponseObject taskResponseObject = ((List) response + final TaskStatusPlus taskResponseObject = ((List) response .getEntity()).get(0); - Assert.assertEquals(taskId_1, taskResponseObject.toJson().get("id")); - Assert.assertEquals(TASK_LOCATION, taskResponseObject.toJson().get("location")); + Assert.assertEquals(taskId_1, taskResponseObject.getId()); + Assert.assertEquals(TASK_LOCATION, taskResponseObject.getLocation()); // Simulate completion of task_1 taskCompletionCountDownLatches[Integer.parseInt(taskId_1)].countDown(); // Wait for taskQueue to handle success status of task_1 - waitForTaskStatus(taskId_1, TaskStatus.Status.SUCCESS); + waitForTaskStatus(taskId_1, TaskState.SUCCESS); // should return number of tasks which are not in running state - response = overlordResource.getCompleteTasks(req); + response = overlordResource.getCompleteTasks(null, req); Assert.assertEquals(2, (((List) response.getEntity()).size())); + + response = overlordResource.getCompleteTasks(1, req); + Assert.assertEquals(1, (((List) response.getEntity()).size())); taskMaster.stop(); Assert.assertFalse(taskMaster.isLeader()); EasyMock.verify(taskLockbox, taskActionClientFactory); @@ -287,7 +292,7 @@ public void testOverlordRun() throws Exception * These method will not timeout until the condition is met so calling method should ensure timeout * This method also assumes that the task with given taskId is present * */ - private void waitForTaskStatus(String taskId, TaskStatus.Status status) throws InterruptedException + private void waitForTaskStatus(String taskId, TaskState status) throws InterruptedException { while (true) { Response response = overlordResource.getTaskStatus(taskId); diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index d51957743ef8..a89a3315b929 100644 --- a/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -123,6 +123,12 @@ public SegmentIdentifier allocatePendingSegment( throw new UnsupportedOperationException(); } + @Override + public int deletePendingSegments(String dataSource, Interval deleteInterval) + { + throw new UnsupportedOperationException(); + } + @Override public void deleteSegments(Set segments) { diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java index fcd01e7f6426..e344c05fe334 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java @@ -20,7 +20,7 @@ package io.druid.indexing.worker; import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.indexing.common.TaskLocation; +import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.task.RealtimeIndexTask; diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index eae1e4d6363e..c34a2c6be849 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -25,9 +25,9 @@ import com.google.common.collect.Lists; import com.google.common.io.Files; import io.druid.curator.PotentiallyGzippedCompressionProvider; +import io.druid.indexer.TaskState; import io.druid.indexing.common.IndexingServiceCondition; import io.druid.indexing.common.SegmentLoaderFactory; -import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.TestRealtimeTask; import io.druid.indexing.common.TestTasks; @@ -263,7 +263,7 @@ public boolean isValid() ); Assert.assertEquals(task.getId(), taskAnnouncement.getTaskStatus().getId()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, taskAnnouncement.getTaskStatus().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, taskAnnouncement.getTaskStatus().getStatusCode()); } @Test(timeout = 30_000L) @@ -299,7 +299,7 @@ public boolean isValid() List announcements = workerCuratorCoordinator.getAnnouncements(); Assert.assertEquals(1, announcements.size()); Assert.assertEquals(task.getId(), announcements.get(0).getTaskStatus().getId()); - Assert.assertEquals(TaskStatus.Status.SUCCESS, announcements.get(0).getTaskStatus().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, announcements.get(0).getTaskStatus().getStatusCode()); Assert.assertEquals(DUMMY_NODE.getHost(), announcements.get(0).getTaskLocation().getHost()); Assert.assertEquals(DUMMY_NODE.getPlaintextPort(), announcements.get(0).getTaskLocation().getPort()); } @@ -337,7 +337,7 @@ public boolean isValid() List announcements = workerCuratorCoordinator.getAnnouncements(); Assert.assertEquals(1, announcements.size()); Assert.assertEquals(task.getId(), announcements.get(0).getTaskStatus().getId()); - Assert.assertEquals(TaskStatus.Status.FAILED, announcements.get(0).getTaskStatus().getStatusCode()); + Assert.assertEquals(TaskState.FAILED, announcements.get(0).getTaskStatus().getStatusCode()); } @Test(timeout = 30_000L) diff --git a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java index 9c45f0b848c7..f47d97e4f536 100644 --- a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java +++ b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java @@ -29,7 +29,7 @@ import com.metamx.http.client.Request; import com.metamx.http.client.response.StatusResponseHandler; import com.metamx.http.client.response.StatusResponseHolder; -import io.druid.indexing.common.TaskStatus; +import io.druid.indexer.TaskState; import io.druid.java.util.common.ISE; import io.druid.java.util.common.RetryUtils; import io.druid.java.util.common.StringUtils; @@ -117,7 +117,7 @@ public String call() throws Exception } } - public TaskStatus.Status getTaskStatus(String taskID) + public TaskState getTaskStatus(String taskID) { try { StatusResponseHolder response = makeRequest( @@ -135,7 +135,7 @@ public TaskStatus.Status getTaskStatus(String taskID) ); //TODO: figure out a better way to parse the response... String status = (String) ((Map) responseData.get("status")).get("status"); - return TaskStatus.Status.valueOf(status); + return TaskState.valueOf(status); } catch (Exception e) { throw Throwables.propagate(e); @@ -189,11 +189,11 @@ public void waitUntilTaskCompletes(final String taskID, final int millisEach, fi @Override public Boolean call() throws Exception { - TaskStatus.Status status = getTaskStatus(taskID); - if (status == TaskStatus.Status.FAILED) { + TaskState status = getTaskStatus(taskID); + if (status == TaskState.FAILED) { throw new ISE("Indexer task FAILED"); } - return status == TaskStatus.Status.SUCCESS; + return status == TaskState.SUCCESS; } }, true, diff --git a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java index 71a03a5a41df..aafdd5a78248 100644 --- a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java @@ -19,18 +19,31 @@ package io.druid.client.indexing; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.inject.Inject; +import com.metamx.http.client.response.FullResponseHolder; import io.druid.discovery.DruidLeaderClient; +import io.druid.indexer.TaskStatusPlus; +import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.jackson.JacksonUtils; import io.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import javax.ws.rs.core.MediaType; +import java.io.IOException; import java.util.Iterator; import java.util.List; +import java.util.Map; public class IndexingServiceClient { @@ -80,6 +93,61 @@ public void upgradeSegments(String dataSource, Interval interval) runQuery(new ClientConversionQuery(dataSource, interval)); } + @Nullable + public TaskStatusPlus getLastCompleteTask() + { + try { + final FullResponseHolder responseHolder = druidLeaderClient.go( + druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/completeTasks?n=1") + ); + + if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE("Error while fetching the status of the last complete task"); + } + + final List taskStatusPluses = jsonMapper.readValue( + responseHolder.getContent(), + new TypeReference>() + { + } + ); + + return taskStatusPluses.isEmpty() ? null : taskStatusPluses.get(0); + } + catch (IOException | InterruptedException e) { + throw new RuntimeException(e); + } + } + + public int killPendingSegments(String dataSource, DateTime end) + { + final String endPoint = StringUtils.format( + "/druid/indexer/v1/pendingSegments/%s?interval=%s", + dataSource, + new Interval(DateTimes.MIN, end) + ); + try { + final FullResponseHolder responseHolder = druidLeaderClient.go( + druidLeaderClient.makeRequest(HttpMethod.DELETE, endPoint) + ); + + if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE("Error while killing pendingSegments of dataSource[%s] created until [%s]", dataSource, end); + } + + final Map resultMap = jsonMapper.readValue( + responseHolder.getContent(), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + + final Object numDeletedObject = resultMap.get("numDeleted"); + return (Integer) Preconditions.checkNotNull(numDeletedObject, "numDeletedObject"); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + private void runQuery(Object queryObject) { try { diff --git a/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java b/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java index 78b9dc1ed5eb..5eef6967afe3 100644 --- a/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java +++ b/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java @@ -44,7 +44,6 @@ import io.druid.metadata.SQLMetadataSegmentManagerProvider; import io.druid.metadata.SQLMetadataSegmentPublisher; import io.druid.metadata.SQLMetadataSegmentPublisherProvider; -import io.druid.metadata.SQLMetadataStorageActionHandlerFactory; import io.druid.metadata.SQLMetadataSupervisorManager; import io.druid.server.audit.AuditManagerProvider; import io.druid.server.audit.SQLAuditManager; @@ -121,11 +120,6 @@ public void configure(Binder binder) .to(SQLMetadataSegmentPublisherProvider.class) .in(LazySingleton.class); - PolyBind.optionBinder(binder, Key.get(MetadataStorageActionHandlerFactory.class)) - .addBinding(type) - .to(SQLMetadataStorageActionHandlerFactory.class) - .in(LazySingleton.class); - PolyBind.optionBinder(binder, Key.get(IndexerMetadataStorageCoordinator.class)) .addBinding(type) .to(IndexerSQLMetadataStorageCoordinator.class) diff --git a/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index ee96b335a0ce..17917d009e84 100644 --- a/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -96,6 +96,17 @@ SegmentIdentifier allocatePendingSegment( boolean skipSegmentLineageCheck ) throws IOException; + /** + * Delete pending segments created in the given interval for the given dataSource from the pending segments table. + * The {@code created_date} field of the pending segments table is checked to find segments to be deleted. + * + * @param dataSource dataSource + * @param deleteInterval interval to check the {@code created_date} of pendingSegments + * + * @return number of deleted pending segments + */ + int deletePendingSegments(String dataSource, Interval deleteInterval); + /** * Attempts to insert a set of segments to the metadata storage. Returns the set of segments actually added (segments * with identifiers already in the metadata storage will not be added). @@ -121,6 +132,9 @@ SegmentPublishResult announceHistoricalSegments( DataSourceMetadata endMetadata ) throws IOException; + /** + * Read dataSource metadata. Returns null if there is no metadata. + */ DataSourceMetadata getDataSourceMetadata(String dataSource); /** diff --git a/server/src/main/java/io/druid/metadata/DerbyMetadataStorageActionHandler.java b/server/src/main/java/io/druid/metadata/DerbyMetadataStorageActionHandler.java new file mode 100644 index 000000000000..62acf4965ee4 --- /dev/null +++ b/server/src/main/java/io/druid/metadata/DerbyMetadataStorageActionHandler.java @@ -0,0 +1,75 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.java.util.common.StringUtils; +import org.joda.time.DateTime; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.Query; + +import javax.annotation.Nullable; +import java.util.Map; + +public class DerbyMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler +{ + DerbyMetadataStorageActionHandler( + SQLMetadataConnector connector, + ObjectMapper jsonMapper, + MetadataStorageActionHandlerTypes types, + String entryTypeName, + String entryTable, + String logTable, + String lockTable + ) + { + super(connector, jsonMapper, types, entryTypeName, entryTable, logTable, lockTable); + } + + @Override + protected Query> createInactiveStatusesSinceQuery( + Handle handle, DateTime timestamp, @Nullable Integer maxNumStatuses + ) + { + String sql = StringUtils.format( + "SELECT " + + " id, " + + " status_payload " + + "FROM " + + " %s " + + "WHERE " + + " active = FALSE AND created_date >= :start " + + "ORDER BY created_date DESC", + getEntryTable() + ); + + if (maxNumStatuses != null) { + sql += " FETCH FIRST :n ROWS ONLY"; + } + + Query> query = handle.createQuery(sql).bind("start", timestamp.toString()); + + if (maxNumStatuses != null) { + query = query.bind("n", maxNumStatuses); + } + return query; + } +} diff --git a/server/src/main/java/io/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java b/server/src/main/java/io/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java new file mode 100644 index 000000000000..ba38cec1528f --- /dev/null +++ b/server/src/main/java/io/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java @@ -0,0 +1,53 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; + +public class DerbyMetadataStorageActionHandlerFactory extends SQLMetadataStorageActionHandlerFactory +{ + @Inject + public DerbyMetadataStorageActionHandlerFactory( + SQLMetadataConnector connector, + MetadataStorageTablesConfig config, + ObjectMapper jsonMapper + ) + { + super(connector, config, jsonMapper); + } + + @Override + public MetadataStorageActionHandler create( + final String entryType, + MetadataStorageActionHandlerTypes payloadTypes + ) + { + return new DerbyMetadataStorageActionHandler<>( + connector, + jsonMapper, + payloadTypes, + entryType, + config.getEntryTable(entryType), + config.getLogTable(entryType), + config.getLockTable(entryType) + ); + } +} diff --git a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index ca6a4e8a1a8a..b0320d773312 100644 --- a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -614,6 +614,24 @@ public SegmentIdentifier inTransaction(Handle handle, TransactionStatus transact ); } + @Override + public int deletePendingSegments(String dataSource, Interval deleteInterval) + { + return connector.getDBI().inTransaction( + (handle, status) -> handle + .createStatement( + StringUtils.format( + "delete from %s where datasource = :dataSource and created_date >= :start and created_date < :end", + dbTables.getPendingSegmentsTable() + ) + ) + .bind("dataSource", dataSource) + .bind("start", deleteInterval.getStart().toString()) + .bind("end", deleteInterval.getEnd().toString()) + .execute() + ); + } + /** * Attempts to insert a single segment to the database. If the segment already exists, will do nothing; although, * this checking is imperfect and callers must be prepared to retry their entire transaction on exceptions. diff --git a/server/src/main/java/io/druid/metadata/MySQLMetadataStorageActionHandler.java b/server/src/main/java/io/druid/metadata/MySQLMetadataStorageActionHandler.java new file mode 100644 index 000000000000..770d980983c8 --- /dev/null +++ b/server/src/main/java/io/druid/metadata/MySQLMetadataStorageActionHandler.java @@ -0,0 +1,75 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.java.util.common.StringUtils; +import org.joda.time.DateTime; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.Query; + +import javax.annotation.Nullable; +import java.util.Map; + +public class MySQLMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler +{ + MySQLMetadataStorageActionHandler( + SQLMetadataConnector connector, + ObjectMapper jsonMapper, + MetadataStorageActionHandlerTypes types, + String entryTypeName, + String entryTable, + String logTable, + String lockTable + ) + { + super(connector, jsonMapper, types, entryTypeName, entryTable, logTable, lockTable); + } + + @Override + protected Query> createInactiveStatusesSinceQuery( + Handle handle, DateTime timestamp, @Nullable Integer maxNumStatuses + ) + { + String sql = StringUtils.format( + "SELECT " + + " id, " + + " status_payload " + + "FROM " + + " %s " + + "WHERE " + + " active = FALSE AND created_date >= :start " + + "ORDER BY created_date DESC", + getEntryTable() + ); + + if (maxNumStatuses != null) { + sql += " LIMIT :n"; + } + + Query> query = handle.createQuery(sql).bind("start", timestamp.toString()); + + if (maxNumStatuses != null) { + query = query.bind("n", maxNumStatuses); + } + return query; + } +} diff --git a/server/src/main/java/io/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java b/server/src/main/java/io/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java new file mode 100644 index 000000000000..0f1742c2d9a7 --- /dev/null +++ b/server/src/main/java/io/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java @@ -0,0 +1,52 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; + +public class MySQLMetadataStorageActionHandlerFactory extends SQLMetadataStorageActionHandlerFactory +{ + @Inject + public MySQLMetadataStorageActionHandlerFactory( + SQLMetadataConnector connector, + MetadataStorageTablesConfig config, + ObjectMapper jsonMapper + ) + { + super(connector, config, jsonMapper); + } + + @Override + public MetadataStorageActionHandler create( + String entryType, MetadataStorageActionHandlerTypes payloadTypes + ) + { + return new MySQLMetadataStorageActionHandler<>( + connector, + jsonMapper, + payloadTypes, + entryType, + config.getEntryTable(entryType), + config.getLogTable(entryType), + config.getLockTable(entryType) + ); + } +} diff --git a/server/src/main/java/io/druid/metadata/PostgreSQLMetadataStorageActionHandler.java b/server/src/main/java/io/druid/metadata/PostgreSQLMetadataStorageActionHandler.java new file mode 100644 index 000000000000..d1a2dfb0af3c --- /dev/null +++ b/server/src/main/java/io/druid/metadata/PostgreSQLMetadataStorageActionHandler.java @@ -0,0 +1,77 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.java.util.common.StringUtils; +import org.joda.time.DateTime; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.Query; + +import javax.annotation.Nullable; +import java.util.Map; + +public class PostgreSQLMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler +{ + public PostgreSQLMetadataStorageActionHandler( + SQLMetadataConnector connector, + ObjectMapper jsonMapper, + MetadataStorageActionHandlerTypes types, + String entryTypeName, + String entryTable, + String logTable, + String lockTable + ) + { + super(connector, jsonMapper, types, entryTypeName, entryTable, logTable, lockTable); + } + + @Override + protected Query> createInactiveStatusesSinceQuery( + Handle handle, + DateTime timestamp, + @Nullable Integer maxNumStatuses + ) + { + String sql = StringUtils.format( + "SELECT " + + " id, " + + " status_payload " + + "FROM " + + " %s " + + "WHERE " + + " active = FALSE AND created_date >= :start " + + "ORDER BY created_date DESC", + getEntryTable() + ); + + if (maxNumStatuses != null) { + sql += " LIMIT :n"; + } + + Query> query = handle.createQuery(sql).bind("start", timestamp.toString()); + + if (maxNumStatuses != null) { + query = query.bind("n", maxNumStatuses); + } + return query; + } +} diff --git a/server/src/main/java/io/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java b/server/src/main/java/io/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java new file mode 100644 index 000000000000..6a802761707c --- /dev/null +++ b/server/src/main/java/io/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java @@ -0,0 +1,52 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; + +public class PostgreSQLMetadataStorageActionHandlerFactory extends SQLMetadataStorageActionHandlerFactory +{ + @Inject + public PostgreSQLMetadataStorageActionHandlerFactory( + SQLMetadataConnector connector, + MetadataStorageTablesConfig config, + ObjectMapper jsonMapper + ) + { + super(connector, config, jsonMapper); + } + + @Override + public MetadataStorageActionHandler create( + String entryType, MetadataStorageActionHandlerTypes payloadTypes + ) + { + return new PostgreSQLMetadataStorageActionHandler<>( + connector, + jsonMapper, + payloadTypes, + entryType, + config.getEntryTable(entryType), + config.getLogTable(entryType), + config.getLockTable(entryType) + ); + } +} diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java index c29b58809009..d196d16020e6 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java @@ -34,6 +34,7 @@ import org.skife.jdbi.v2.FoldController; import org.skife.jdbi.v2.Folder3; import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.Query; import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.exceptions.CallbackFailedException; import org.skife.jdbi.v2.exceptions.StatementException; @@ -49,7 +50,7 @@ import java.util.Map; import java.util.Map.Entry; -public class SQLMetadataStorageActionHandler +public abstract class SQLMetadataStorageActionHandler implements MetadataStorageActionHandler { private static final EmittingLogger log = new EmittingLogger(SQLMetadataStorageActionHandler.class); @@ -88,6 +89,26 @@ public SQLMetadataStorageActionHandler( this.lockTable = lockTable; } + protected SQLMetadataConnector getConnector() + { + return connector; + } + + protected ObjectMapper getJsonMapper() + { + return jsonMapper; + } + + protected TypeReference getStatusType() + { + return statusType; + } + + protected String getEntryTable() + { + return entryTable; + } + @Override public void insert( final String id, @@ -268,47 +289,61 @@ public Pair map(int index, ResultSet r, StatementContext } @Override - public List getInactiveStatusesSince(final DateTime timestamp) + public List getInactiveStatusesSince(DateTime timestamp, @Nullable Integer maxNumStatuses) { - return connector.retryWithHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) throws Exception - { - return handle - .createQuery( - StringUtils.format( - "SELECT id, status_payload FROM %s WHERE active = FALSE AND created_date >= :start ORDER BY created_date DESC", - entryTable - ) - ).bind("start", timestamp.toString()) - .map( - new ResultSetMapper() - { - @Override - public StatusType map(int index, ResultSet r, StatementContext ctx) throws SQLException - { - try { - return jsonMapper.readValue( - r.getBytes("status_payload"), - statusType - ); - } - catch (IOException e) { - log.makeAlert(e, "Failed to parse status payload") - .addData("entry", r.getString("id")) - .emit(); - throw new SQLException(e); - } - } + return getConnector().retryWithHandle( + handle -> { + final Query> query = createInactiveStatusesSinceQuery(handle, timestamp, maxNumStatuses); + + return query + .map( + (ResultSetMapper) (index, r, ctx) -> { + try { + return getJsonMapper().readValue( + r.getBytes("status_payload"), + getStatusType() + ); } - ).list(); - } + catch (IOException e) { + log.makeAlert(e, "Failed to parse status payload") + .addData("entry", r.getString("id")) + .emit(); + throw new SQLException(e); + } + } + ).list(); } ); } + protected abstract Query> createInactiveStatusesSinceQuery( + Handle handle, + DateTime timestamp, + @Nullable Integer maxNumStatuses + ); + + @Override + @Nullable + public Pair getCreatedDateAndDataSource(String entryId) + { + return connector.retryWithHandle( + handle -> handle + .createQuery( + StringUtils.format( + "SELECT created_date, datasource FROM %s WHERE id = :entryId", + entryTable + ) + ) + .bind("entryId", entryId) + .map( + (index, resultSet, ctx) -> Pair.of( + new DateTime(resultSet.getString("created_date")), resultSet.getString("datasource") + ) + ) + .first() + ); + } + @Override public boolean addLock(final String entryId, final LockType lock) { diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandlerFactory.java b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandlerFactory.java index b4298de2ccc9..bdbe7aa5d6e4 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandlerFactory.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandlerFactory.java @@ -22,11 +22,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; -public class SQLMetadataStorageActionHandlerFactory implements MetadataStorageActionHandlerFactory +public abstract class SQLMetadataStorageActionHandlerFactory implements MetadataStorageActionHandlerFactory { - private final SQLMetadataConnector connector; - private final MetadataStorageTablesConfig config; - private final ObjectMapper jsonMapper; + protected final SQLMetadataConnector connector; + protected final MetadataStorageTablesConfig config; + protected final ObjectMapper jsonMapper; @Inject public SQLMetadataStorageActionHandlerFactory( @@ -39,21 +39,4 @@ public SQLMetadataStorageActionHandlerFactory( this.config = config; this.jsonMapper = jsonMapper; } - - @Override - public MetadataStorageActionHandler create( - final String entryType, - MetadataStorageActionHandlerTypes payloadTypes - ) - { - return new SQLMetadataStorageActionHandler<>( - connector, - jsonMapper, - payloadTypes, - entryType, - config.getEntryTable(entryType), - config.getLogTable(entryType), - config.getLockTable(entryType) - ); - } } diff --git a/server/src/main/java/io/druid/metadata/SQLServerMetadataStorageActionHandler.java b/server/src/main/java/io/druid/metadata/SQLServerMetadataStorageActionHandler.java new file mode 100644 index 000000000000..199adff5656f --- /dev/null +++ b/server/src/main/java/io/druid/metadata/SQLServerMetadataStorageActionHandler.java @@ -0,0 +1,72 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.java.util.common.StringUtils; +import org.joda.time.DateTime; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.Query; + +import javax.annotation.Nullable; +import java.util.Map; + +public class SQLServerMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler +{ + public SQLServerMetadataStorageActionHandler( + SQLMetadataConnector connector, + ObjectMapper jsonMapper, + MetadataStorageActionHandlerTypes types, + String entryTypeName, + String entryTable, + String logTable, + String lockTable + ) + { + super(connector, jsonMapper, types, entryTypeName, entryTable, logTable, lockTable); + } + + @Override + protected Query> createInactiveStatusesSinceQuery( + Handle handle, DateTime timestamp, @Nullable Integer maxNumStatuses + ) + { + String sql = maxNumStatuses == null ? "SELECT " : "SELECT TOP :n "; + + sql += StringUtils.format( + " id, " + + " status_payload " + + "FROM " + + " %s " + + "WHERE " + + " active = FALSE AND created_date >= :start " + + "ORDER BY created_date DESC", + getEntryTable() + ); + + Query> query = handle.createQuery(sql).bind("start", timestamp.toString()); + + if (maxNumStatuses != null) { + query = query.bind("n", maxNumStatuses); + } + return query; + } +} diff --git a/server/src/main/java/io/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java b/server/src/main/java/io/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java new file mode 100644 index 000000000000..0704fea34287 --- /dev/null +++ b/server/src/main/java/io/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java @@ -0,0 +1,52 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; + +public class SQLServerMetadataStorageActionHandlerFactory extends SQLMetadataStorageActionHandlerFactory +{ + @Inject + public SQLServerMetadataStorageActionHandlerFactory( + SQLMetadataConnector connector, + MetadataStorageTablesConfig config, + ObjectMapper jsonMapper + ) + { + super(connector, config, jsonMapper); + } + + @Override + public MetadataStorageActionHandler create( + String entryType, MetadataStorageActionHandlerTypes payloadTypes + ) + { + return new SQLServerMetadataStorageActionHandler<>( + connector, + jsonMapper, + payloadTypes, + entryType, + config.getEntryTable(entryType), + config.getLogTable(entryType), + config.getLockTable(entryType) + ); + } +} diff --git a/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorageDruidModule.java b/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorageDruidModule.java index 8cba73bed84f..850d7d688372 100644 --- a/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorageDruidModule.java +++ b/server/src/main/java/io/druid/metadata/storage/derby/DerbyMetadataStorageDruidModule.java @@ -24,7 +24,9 @@ import io.druid.guice.LazySingleton; import io.druid.guice.PolyBind; import io.druid.guice.SQLMetadataStorageDruidModule; +import io.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import io.druid.metadata.MetadataStorage; +import io.druid.metadata.MetadataStorageActionHandlerFactory; import io.druid.metadata.MetadataStorageConnector; import io.druid.metadata.MetadataStorageProvider; import io.druid.metadata.NoopMetadataStorageProvider; @@ -32,13 +34,13 @@ public class DerbyMetadataStorageDruidModule extends SQLMetadataStorageDruidModule { + public static final String TYPE = "derby"; + public DerbyMetadataStorageDruidModule() { super(TYPE); } - public static final String TYPE = "derby"; - @Override public void configure(Binder binder) { @@ -61,5 +63,10 @@ public void configure(Binder binder) .addBinding(TYPE) .to(DerbyConnector.class) .in(LazySingleton.class); + + PolyBind.optionBinder(binder, Key.get(MetadataStorageActionHandlerFactory.class)) + .addBinding(TYPE) + .to(DerbyMetadataStorageActionHandlerFactory.class) + .in(LazySingleton.class); } } diff --git a/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java index f15909e86e98..d23928bc9f03 100644 --- a/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -25,6 +25,7 @@ import java.util.Collection; import java.util.HashSet; +import java.util.Objects; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -42,6 +43,7 @@ public class CoordinatorDynamicConfig private final boolean emitBalancingStats; private final boolean killAllDataSources; private final Set killDataSourceWhitelist; + private final Set killPendingSegmentsSkipList; /** * The maximum number of segments that could be queued for loading to any given server. * Default values is 0 with the meaning of "unbounded" (any number of @@ -65,6 +67,7 @@ public CoordinatorDynamicConfig( // coordinator console can not send array of strings in the update request. // See https://github.com/druid-io/druid/issues/3055 @JsonProperty("killDataSourceWhitelist") Object killDataSourceWhitelist, + @JsonProperty("killPendingSegmentsSkipList") Object killPendingSegmentsSkipList, @JsonProperty("killAllDataSources") boolean killAllDataSources, @JsonProperty("maxSegmentsInNodeLoadingQueue") int maxSegmentsInNodeLoadingQueue ) @@ -78,7 +81,8 @@ public CoordinatorDynamicConfig( this.balancerComputeThreads = Math.max(balancerComputeThreads, 1); this.emitBalancingStats = emitBalancingStats; this.killAllDataSources = killAllDataSources; - this.killDataSourceWhitelist = parseKillDataSourceWhitelist(killDataSourceWhitelist); + this.killDataSourceWhitelist = parseJsonStringOrArray(killDataSourceWhitelist); + this.killPendingSegmentsSkipList = parseJsonStringOrArray(killPendingSegmentsSkipList); this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; if (this.killAllDataSources && !this.killDataSourceWhitelist.isEmpty()) { @@ -86,10 +90,10 @@ public CoordinatorDynamicConfig( } } - private Set parseKillDataSourceWhitelist(Object killDataSourceWhitelist) + private static Set parseJsonStringOrArray(Object jsonStringOrArray) { - if (killDataSourceWhitelist instanceof String) { - String[] list = ((String) killDataSourceWhitelist).split(","); + if (jsonStringOrArray instanceof String) { + String[] list = ((String) jsonStringOrArray).split(","); Set result = new HashSet<>(); for (String item : list) { String trimmed = item.trim(); @@ -98,8 +102,8 @@ private Set parseKillDataSourceWhitelist(Object killDataSourceWhitelist) } } return result; - } else if (killDataSourceWhitelist instanceof Collection) { - return ImmutableSet.copyOf(((Collection) killDataSourceWhitelist)); + } else if (jsonStringOrArray instanceof Collection) { + return ImmutableSet.copyOf(((Collection) jsonStringOrArray)); } else { return ImmutableSet.of(); } @@ -159,6 +163,12 @@ public Set getKillDataSourceWhitelist() return killDataSourceWhitelist; } + @JsonProperty + public Set getKillPendingSegmentsSkipList() + { + return killPendingSegmentsSkipList; + } + @JsonProperty public boolean isKillAllDataSources() { @@ -184,6 +194,7 @@ public String toString() ", balancerComputeThreads=" + balancerComputeThreads + ", emitBalancingStats=" + emitBalancingStats + ", killDataSourceWhitelist=" + killDataSourceWhitelist + + ", killPendingSegmentsSkipList=" + killPendingSegmentsSkipList + ", killAllDataSources=" + killAllDataSources + ", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue + '}'; @@ -231,27 +242,30 @@ public boolean equals(Object o) if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) { return false; } - return !(killDataSourceWhitelist != null - ? !killDataSourceWhitelist.equals(that.killDataSourceWhitelist) - : that.killDataSourceWhitelist != null); + if (!Objects.equals(killDataSourceWhitelist, that.killDataSourceWhitelist)) { + return false; + } + return Objects.equals(killPendingSegmentsSkipList, that.killPendingSegmentsSkipList); } @Override public int hashCode() { - int result = (int) (millisToWaitBeforeDeleting ^ (millisToWaitBeforeDeleting >>> 32)); - result = 31 * result + (int) (mergeBytesLimit ^ (mergeBytesLimit >>> 32)); - result = 31 * result + mergeSegmentsLimit; - result = 31 * result + maxSegmentsToMove; - result = 31 * result + replicantLifetime; - result = 31 * result + replicationThrottleLimit; - result = 31 * result + balancerComputeThreads; - result = 31 * result + (emitBalancingStats ? 1 : 0); - result = 31 * result + (killAllDataSources ? 1 : 0); - result = 31 * result + (killDataSourceWhitelist != null ? killDataSourceWhitelist.hashCode() : 0); - result = 31 * result + maxSegmentsInNodeLoadingQueue; - return result; + return Objects.hash( + millisToWaitBeforeDeleting, + mergeBytesLimit, + mergeSegmentsLimit, + maxSegmentsToMove, + replicantLifetime, + replicationThrottleLimit, + balancerComputeThreads, + emitBalancingStats, + killAllDataSources, + maxSegmentsInNodeLoadingQueue, + killDataSourceWhitelist, + killPendingSegmentsSkipList + ); } public static Builder builder() @@ -281,6 +295,7 @@ public static class Builder private Boolean emitBalancingStats; private Integer balancerComputeThreads; private Object killDataSourceWhitelist; + private Object killPendingSegmentsSkipList; private Boolean killAllDataSources; private Integer maxSegmentsInNodeLoadingQueue; @@ -299,6 +314,7 @@ public Builder( @JsonProperty("balancerComputeThreads") Integer balancerComputeThreads, @JsonProperty("emitBalancingStats") Boolean emitBalancingStats, @JsonProperty("killDataSourceWhitelist") Object killDataSourceWhitelist, + @JsonProperty("killPendingSegmentsSkipList") Object killPendingSegmentsSkipList, @JsonProperty("killAllDataSources") Boolean killAllDataSources, @JsonProperty("maxSegmentsInNodeLoadingQueue") Integer maxSegmentsInNodeLoadingQueue ) @@ -313,6 +329,7 @@ public Builder( this.emitBalancingStats = emitBalancingStats; this.killAllDataSources = killAllDataSources; this.killDataSourceWhitelist = killDataSourceWhitelist; + this.killPendingSegmentsSkipList = killPendingSegmentsSkipList; this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; } @@ -382,9 +399,6 @@ public Builder withMaxSegmentsInNodeLoadingQueue(int maxSegmentsInNodeLoadingQue return this; } - - - public CoordinatorDynamicConfig build() { return new CoordinatorDynamicConfig( @@ -397,6 +411,7 @@ public CoordinatorDynamicConfig build() balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads, emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats, killDataSourceWhitelist, + killPendingSegmentsSkipList, killAllDataSources == null ? DEFAULT_KILL_ALL_DATA_SOURCES : killAllDataSources, maxSegmentsInNodeLoadingQueue == null ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE : maxSegmentsInNodeLoadingQueue ); @@ -414,6 +429,7 @@ public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads, emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats, killDataSourceWhitelist == null ? defaults.getKillDataSourceWhitelist() : killDataSourceWhitelist, + killPendingSegmentsSkipList == null ? defaults.getKillPendingSegmentsSkipList() : killPendingSegmentsSkipList, killAllDataSources == null ? defaults.isKillAllDataSources() : killAllDataSources, maxSegmentsInNodeLoadingQueue == null ? defaults.getMaxSegmentsInNodeLoadingQueue() : maxSegmentsInNodeLoadingQueue ); diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java new file mode 100644 index 000000000000..6697fb1d887c --- /dev/null +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java @@ -0,0 +1,60 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.coordinator; + +import com.google.inject.Inject; +import io.druid.client.ImmutableDruidDataSource; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.indexer.TaskStatusPlus; +import io.druid.java.util.common.logger.Logger; +import io.druid.server.coordinator.helper.DruidCoordinatorHelper; + +public class DruidCoordinatorCleanupPendingSegments implements DruidCoordinatorHelper +{ + private static final Logger log = new Logger(DruidCoordinatorCleanupPendingSegments.class); + + private final IndexingServiceClient indexingServiceClient; + + @Inject + public DruidCoordinatorCleanupPendingSegments(IndexingServiceClient indexingServiceClient) + { + this.indexingServiceClient = indexingServiceClient; + } + + @Override + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) + { + final TaskStatusPlus taskStatusPlus = indexingServiceClient.getLastCompleteTask(); + + if (taskStatusPlus != null) { + for (ImmutableDruidDataSource dataSource : params.getDataSources()) { + if (!params.getCoordinatorDynamicConfig().getKillPendingSegmentsSkipList().contains(dataSource.getName())) { + log.info( + "Kill pendingSegments created until [%s] for dataSource[%s]", + dataSource, + taskStatusPlus.getCreatedTime() + ); + indexingServiceClient.killPendingSegments(dataSource.getName(), taskStatusPlus.getCreatedTime()); + } + } + } + return params; + } +} diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java index ba834620013d..5976b5d59c7c 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java @@ -57,6 +57,12 @@ public boolean isKillSegments() return false; } + @Config("druid.coordinator.kill.pendingSegments.on") + public boolean isKillPendingSegments() + { + return false; + } + @Config("druid.coordinator.kill.period") @Default("P1D") public abstract Duration getCoordinatorKillPeriod(); diff --git a/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 19d8a665b554..2c47b4b9a4b2 100644 --- a/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -27,17 +27,21 @@ import io.druid.indexing.overlord.ObjectMetadata; import io.druid.indexing.overlord.SegmentPublishResult; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NumberedShardSpec; +import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.tweak.HandleCallback; import org.skife.jdbi.v2.util.StringMapper; @@ -53,6 +57,9 @@ public class IndexerSQLMetadataStorageCoordinatorTest @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + private final ObjectMapper mapper = new DefaultObjectMapper(); private final DataSegment defaultSegment = new DataSegment( "fooDataSource", @@ -172,10 +179,11 @@ public class IndexerSQLMetadataStorageCoordinatorTest public void setUp() { derbyConnector = derbyConnectorRule.getConnector(); - mapper.registerSubtypes(LinearShardSpec.class); + mapper.registerSubtypes(LinearShardSpec.class, NumberedShardSpec.class); derbyConnector.createDataSourceTable(); derbyConnector.createTaskTables(); derbyConnector.createSegmentTable(); + derbyConnector.createPendingSegmentsTable(); metadataUpdateCounter.set(0); coordinator = new IndexerSQLMetadataStorageCoordinator( mapper, @@ -826,4 +834,105 @@ private void additionalNumberedShardTest(Set segments) throws IOExc // Should not update dataSource metadata. Assert.assertEquals(0, metadataUpdateCounter.get()); } + + @Test + public void testAllocatePendingSegment() throws IOException + { + final String dataSource = "ds"; + final Interval interval = Intervals.of("2017-01-01/2017-02-01"); + final SegmentIdentifier identifier = coordinator.allocatePendingSegment( + dataSource, + "seq", + null, + interval, + "version", + false + ); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version", identifier.toString()); + + final SegmentIdentifier identifier1 = coordinator.allocatePendingSegment( + dataSource, + "seq", + identifier.toString(), + interval, + identifier.getVersion(), + false + ); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_1", identifier1.toString()); + + final SegmentIdentifier identifier2 = coordinator.allocatePendingSegment( + dataSource, + "seq", + identifier1.toString(), + interval, + identifier1.getVersion(), + false + ); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_2", identifier2.toString()); + + final SegmentIdentifier identifier3 = coordinator.allocatePendingSegment( + dataSource, + "seq", + identifier1.toString(), + interval, + identifier1.getVersion(), + false + ); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_2", identifier3.toString()); + Assert.assertEquals(identifier2, identifier3); + + final SegmentIdentifier identifier4 = coordinator.allocatePendingSegment( + dataSource, + "seq1", + null, + interval, + "version", + false + ); + + Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_3", identifier4.toString()); + } + + @Test + public void testDeletePendingSegment() throws IOException, InterruptedException + { + final String dataSource = "ds"; + final Interval interval = Intervals.of("2017-01-01/2017-02-01"); + String prevSegmentId = null; + + final DateTime begin = DateTimes.nowUtc(); + + for (int i = 0; i < 10; i++) { + final SegmentIdentifier identifier = coordinator.allocatePendingSegment( + dataSource, + "seq", + prevSegmentId, + interval, + "version", + false + ); + prevSegmentId = identifier.toString(); + } + Thread.sleep(100); + + final DateTime secondBegin = DateTimes.nowUtc(); + for (int i = 0; i < 5; i++) { + final SegmentIdentifier identifier = coordinator.allocatePendingSegment( + dataSource, + "seq", + prevSegmentId, + interval, + "version", + false + ); + prevSegmentId = identifier.toString(); + } + + final int numDeleted = coordinator.deletePendingSegments(dataSource, new Interval(begin, secondBegin)); + Assert.assertEquals(10, numDeleted); + } } diff --git a/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java index e86fa2a77bae..1dbdabef177f 100644 --- a/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/io/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -28,6 +28,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Pair; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.jackson.JacksonUtils; import org.junit.Assert; import org.junit.Before; @@ -36,6 +37,7 @@ import org.junit.rules.ExpectedException; import java.util.HashSet; +import java.util.List; import java.util.Map; public class SQLMetadataStorageActionHandlerTest @@ -59,13 +61,11 @@ public void setUp() throws Exception final String logTable = "logs"; final String lockTable = "locks"; - connector.createEntryTable(entryTable); connector.createLockTable(lockTable, entryType); connector.createLogTable(logTable, entryType); - - handler = new SQLMetadataStorageActionHandler<>( + handler = new DerbyMetadataStorageActionHandler<>( connector, jsonMapper, new MetadataStorageActionHandlerTypes, Map, Map, Map>() @@ -179,6 +179,44 @@ public void testEntryAndStatus() throws Exception ); } + @Test + public void testGetRecentStatuses() throws EntryExistsException + { + for (int i = 1; i < 11; i++) { + final String entryId = "abcd_" + i; + final Map entry = ImmutableMap.of("a", i); + final Map status = ImmutableMap.of("count", i * 10); + + handler.insert(entryId, DateTimes.of(StringUtils.format("2014-01-%02d", i)), "test", entry, false, status); + } + + final List> statuses = handler.getInactiveStatusesSince(DateTimes.of("2014-01-01"), 7); + Assert.assertEquals(7, statuses.size()); + int i = 10; + for (Map status : statuses) { + Assert.assertEquals(ImmutableMap.of("count", i-- * 10), status); + } + } + + @Test + public void testGetRecentStatuses2() throws EntryExistsException + { + for (int i = 1; i < 6; i++) { + final String entryId = "abcd_" + i; + final Map entry = ImmutableMap.of("a", i); + final Map status = ImmutableMap.of("count", i * 10); + + handler.insert(entryId, DateTimes.of(StringUtils.format("2014-01-%02d", i)), "test", entry, false, status); + } + + final List> statuses = handler.getInactiveStatusesSince(DateTimes.of("2014-01-01"), 10); + Assert.assertEquals(5, statuses.size()); + int i = 5; + for (Map status : statuses) { + Assert.assertEquals(ImmutableMap.of("count", i-- * 10), status); + } + } + @Test(timeout = 10_000L) public void testRepeatInsert() throws Exception { diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorConfigTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorConfigTest.java index 5836515d0683..87752595ba88 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorConfigTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorConfigTest.java @@ -19,13 +19,12 @@ package io.druid.server.coordinator; +import io.druid.java.util.common.config.Config; import org.joda.time.Duration; import org.junit.Assert; import org.junit.Test; import org.skife.config.ConfigurationObjectFactory; -import io.druid.java.util.common.config.Config; - import java.util.Properties; /** @@ -46,6 +45,7 @@ public void testDeserialization() throws Exception Assert.assertFalse(config.isMergeSegments()); Assert.assertFalse(config.isConvertSegments()); Assert.assertFalse(config.isKillSegments()); + Assert.assertFalse(config.isKillPendingSegments()); Assert.assertEquals(86400000, config.getCoordinatorKillPeriod().getMillis()); Assert.assertEquals(-1000, config.getCoordinatorKillDurationToRetain().getMillis()); Assert.assertEquals(0, config.getCoordinatorKillMaxSegments()); @@ -64,6 +64,7 @@ public void testDeserialization() throws Exception props.setProperty("druid.coordinator.kill.period", "PT1s"); props.setProperty("druid.coordinator.kill.durationToRetain", "PT1s"); props.setProperty("druid.coordinator.kill.maxSegments", "10000"); + props.setProperty("druid.coordinator.kill.pendingSegments.on", "true"); props.setProperty("druid.coordinator.load.timeout", "PT1s"); props.setProperty("druid.coordinator.console.static", "test"); props.setProperty("druid.coordinator.loadqueuepeon.repeatDelay", "PT0.100s"); @@ -77,6 +78,7 @@ public void testDeserialization() throws Exception Assert.assertTrue(config.isMergeSegments()); Assert.assertTrue(config.isConvertSegments()); Assert.assertTrue(config.isKillSegments()); + Assert.assertTrue(config.isKillPendingSegments()); Assert.assertEquals(new Duration("PT1s"), config.getCoordinatorKillPeriod()); Assert.assertEquals(new Duration("PT1s"), config.getCoordinatorKillDurationToRetain()); Assert.assertEquals(10000, config.getCoordinatorKillMaxSegments()); diff --git a/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java index 5f17c37bd2b8..3e9fd1e59717 100644 --- a/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java @@ -183,7 +183,8 @@ public void testUpdate() Assert.assertEquals( current, - new CoordinatorDynamicConfig.Builder(null, null, null, null, null, null, null, null, null, null, null).build(current) + new CoordinatorDynamicConfig.Builder(null, null, null, null, null, null, null, null, null, null, null, null) + .build(current) ); } diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index f2107d54c621..e4f8e1002f56 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -57,6 +57,7 @@ import io.druid.server.audit.AuditManagerProvider; import io.druid.server.coordinator.BalancerStrategyFactory; import io.druid.server.coordinator.DruidCoordinator; +import io.druid.server.coordinator.DruidCoordinatorCleanupPendingSegments; import io.druid.server.coordinator.DruidCoordinatorConfig; import io.druid.server.coordinator.LoadQueueTaskMaster; import io.druid.server.coordinator.helper.DruidCoordinatorHelper; @@ -86,6 +87,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Objects; import java.util.Properties; import java.util.concurrent.Executors; @@ -207,6 +209,10 @@ public void configure(Binder binder) "druid.coordinator.kill.on", Predicates.equalTo("true"), DruidCoordinatorSegmentKiller.class + ).addConditionBinding( + "druid.coordinator.kill.pendingSegments.on", + predicate -> Objects.equals(predicate, "true"), + DruidCoordinatorCleanupPendingSegments.class ); binder.bind(DiscoverySideEffectsProvider.Child.class).annotatedWith(Coordinator.class).toProvider( diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 7bba5f4e2b21..c92aa005fd2c 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -58,6 +58,7 @@ import io.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; import io.druid.indexing.overlord.ForkingTaskRunnerFactory; import io.druid.indexing.overlord.HeapMemoryTaskStorage; +import io.druid.indexing.overlord.IndexerMetadataStorageAdapter; import io.druid.indexing.overlord.MetadataTaskStorage; import io.druid.indexing.overlord.RemoteTaskRunnerFactory; import io.druid.indexing.overlord.TaskLockbox; @@ -176,6 +177,7 @@ public void configure(Binder binder) binder.bind(TaskActionToolbox.class).in(LazySingleton.class); binder.bind(TaskLockbox.class).in(LazySingleton.class); binder.bind(TaskStorageQueryAdapter.class).in(LazySingleton.class); + binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class); binder.bind(SupervisorManager.class).in(LazySingleton.class); binder.bind(ChatHandlerProvider.class).toProvider(Providers.of(null)); From 68a39be72bb23bbedf6e8ab629c214b00717f2d0 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 11 Dec 2017 22:26:06 +0900 Subject: [PATCH 2/9] fix build --- .../io/druid/metadata/SQLMetadataStorageActionHandler.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java index d196d16020e6..90b508f05052 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java @@ -28,6 +28,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.metamx.emitter.EmittingLogger; +import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import org.joda.time.DateTime; @@ -337,7 +338,7 @@ public Pair getCreatedDateAndDataSource(String entryId) .bind("entryId", entryId) .map( (index, resultSet, ctx) -> Pair.of( - new DateTime(resultSet.getString("created_date")), resultSet.getString("datasource") + DateTimes.of(resultSet.getString("created_date")), resultSet.getString("datasource") ) ) .first() From dbc2b77cf9ab1948a563a5b45e9f81db0ceaa248 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 14 Dec 2017 12:19:55 +0900 Subject: [PATCH 3/9] address comments --- .../indexing/IndexingServiceClient.java | 31 +++++++++++++++---- ...ruidCoordinatorCleanupPendingSegments.java | 22 ++++++++++--- 2 files changed, 43 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java index aafdd5a78248..39aa1e037df3 100644 --- a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java @@ -93,32 +93,51 @@ public void upgradeSegments(String dataSource, Interval interval) runQuery(new ClientConversionQuery(dataSource, interval)); } - @Nullable - public TaskStatusPlus getLastCompleteTask() + public List getRunningTasks() + { + return getTasks("runningTasks"); + } + + public List getPendingTasks() + { + return getTasks("pendingTasks"); + } + + public List getWaitingTasks() + { + return getTasks("waitingTasks"); + } + + private List getTasks(String endpointSuffix) { try { final FullResponseHolder responseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/completeTasks?n=1") + druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/%s", endpointSuffix)) ); if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) { throw new ISE("Error while fetching the status of the last complete task"); } - final List taskStatusPluses = jsonMapper.readValue( + return jsonMapper.readValue( responseHolder.getContent(), new TypeReference>() { } ); - - return taskStatusPluses.isEmpty() ? null : taskStatusPluses.get(0); } catch (IOException | InterruptedException e) { throw new RuntimeException(e); } } + @Nullable + public TaskStatusPlus getLastCompleteTask() + { + final List completeTaskStatuses = getTasks("completeTasks?n=1"); + return completeTaskStatuses.isEmpty() ? null : completeTaskStatuses.get(0); + } + public int killPendingSegments(String dataSource, DateTime end) { final String endPoint = StringUtils.format( diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java index 6697fb1d887c..49cad8126d30 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java @@ -26,6 +26,10 @@ import io.druid.java.util.common.logger.Logger; import io.druid.server.coordinator.helper.DruidCoordinatorHelper; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; + public class DruidCoordinatorCleanupPendingSegments implements DruidCoordinatorHelper { private static final Logger log = new Logger(DruidCoordinatorCleanupPendingSegments.class); @@ -41,17 +45,27 @@ public DruidCoordinatorCleanupPendingSegments(IndexingServiceClient indexingServ @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - final TaskStatusPlus taskStatusPlus = indexingServiceClient.getLastCompleteTask(); + final List taskStatusPluses = new ArrayList<>(); + taskStatusPluses.addAll(indexingServiceClient.getRunningTasks()); + taskStatusPluses.addAll(indexingServiceClient.getPendingTasks()); + taskStatusPluses.addAll(indexingServiceClient.getWaitingTasks()); + + final TaskStatusPlus completeTaskStatus = indexingServiceClient.getLastCompleteTask(); + if (completeTaskStatus != null) { + taskStatusPluses.add(completeTaskStatus); + } + taskStatusPluses.sort(Comparator.comparing(TaskStatusPlus::getCreatedTime)); - if (taskStatusPlus != null) { + if (!taskStatusPluses.isEmpty()) { + final TaskStatusPlus firstCreatedTaskStatus = taskStatusPluses.get(0); for (ImmutableDruidDataSource dataSource : params.getDataSources()) { if (!params.getCoordinatorDynamicConfig().getKillPendingSegmentsSkipList().contains(dataSource.getName())) { log.info( "Kill pendingSegments created until [%s] for dataSource[%s]", dataSource, - taskStatusPlus.getCreatedTime() + firstCreatedTaskStatus.getCreatedTime() ); - indexingServiceClient.killPendingSegments(dataSource.getName(), taskStatusPlus.getCreatedTime()); + indexingServiceClient.killPendingSegments(dataSource.getName(), firstCreatedTaskStatus.getCreatedTime()); } } } From 2ebdaed345ddfca47c7696f44ea2f6cfdb3a99b4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 14 Dec 2017 12:21:51 +0900 Subject: [PATCH 4/9] address comments --- .../io/druid/indexing/overlord/HeapMemoryTaskStorage.java | 4 ++-- .../io/druid/server/coordinator/CoordinatorDynamicConfig.java | 2 ++ .../coordinator/DruidCoordinatorCleanupPendingSegments.java | 4 ++-- services/src/main/java/io/druid/cli/CliCoordinator.java | 3 +-- 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java index a464aee9b464..bdc718bd85f1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -183,7 +183,7 @@ public int compare(TaskStuff a, TaskStuff b) }.reverse(); return maxTaskStatuses == null ? - getRecentlyFinishedTaskSTatusesSince( + getRecentlyFinishedTaskStatusesSince( System.currentTimeMillis() - config.getRecentlyFinishedThreshold().getMillis(), createdDateDesc ) : @@ -194,7 +194,7 @@ public int compare(TaskStuff a, TaskStuff b) } } - private List getRecentlyFinishedTaskSTatusesSince(long start, Ordering createdDateDesc) + private List getRecentlyFinishedTaskStatusesSince(long start, Ordering createdDateDesc) { giant.lock(); diff --git a/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java index d23928bc9f03..65bea6a5848c 100644 --- a/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -43,6 +43,8 @@ public class CoordinatorDynamicConfig private final boolean emitBalancingStats; private final boolean killAllDataSources; private final Set killDataSourceWhitelist; + + // The pending segments of the dataSources in this list are not killed. private final Set killPendingSegmentsSkipList; /** * The maximum number of segments that could be queued for loading to any given server. diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java index 49cad8126d30..e158d5000a04 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java @@ -62,8 +62,8 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) if (!params.getCoordinatorDynamicConfig().getKillPendingSegmentsSkipList().contains(dataSource.getName())) { log.info( "Kill pendingSegments created until [%s] for dataSource[%s]", - dataSource, - firstCreatedTaskStatus.getCreatedTime() + firstCreatedTaskStatus.getCreatedTime(), + dataSource ); indexingServiceClient.killPendingSegments(dataSource.getName(), firstCreatedTaskStatus.getCreatedTime()); } diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index e4f8e1002f56..af706fdeb4a3 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -87,7 +87,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Objects; import java.util.Properties; import java.util.concurrent.Executors; @@ -211,7 +210,7 @@ public void configure(Binder binder) DruidCoordinatorSegmentKiller.class ).addConditionBinding( "druid.coordinator.kill.pendingSegments.on", - predicate -> Objects.equals(predicate, "true"), + Predicates.equalTo("true"), DruidCoordinatorCleanupPendingSegments.class ); From 017fce3bfc20f0d3ac60af930704315e812d1942 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 18 Dec 2017 11:02:08 +0900 Subject: [PATCH 5/9] fix potential npe --- .../indexing/overlord/IndexerMetadataStorageAdapter.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java index e166aff6df73..4048dc33193c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java @@ -49,7 +49,11 @@ public int deletePendingSegments(String dataSource, Interval deleteInterval) final Optional minCreatedDateOfActiveTasks = taskStorageQueryAdapter .getActiveTasks() .stream() - .map(task -> taskStorageQueryAdapter.getCreatedTime(task.getId())) + .map(task -> Preconditions.checkNotNull( + taskStorageQueryAdapter.getCreatedTime(task.getId()), + "Can't find the createdTime for task[%s]", + task.getId() + )) .min(Comparator.naturalOrder()); final Interval activeTaskInterval = new Interval( From 3ac2603662088f21e72cb12a5198114aa3305e80 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 19 Dec 2017 14:12:15 +0900 Subject: [PATCH 6/9] address comments --- .../IndexerMetadataStorageAdapter.java | 2 +- .../overlord/http/OverlordResource.java | 7 +- .../IndexerSQLMetadataStorageCoordinator.java | 2 +- ...ruidCoordinatorCleanupPendingSegments.java | 66 ++++++++++++++----- .../java/io/druid/server/security/Access.java | 7 +- .../main/java/io/druid/cli/CliOverlord.java | 1 - 6 files changed, 57 insertions(+), 28 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java index 4048dc33193c..dc7ea1730838 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapter.java @@ -63,7 +63,7 @@ public int deletePendingSegments(String dataSource, Interval deleteInterval) Preconditions.checkArgument( !deleteInterval.overlaps(activeTaskInterval), - "Cannot delete pendingSegments because there is at least one running task created at %s", + "Cannot delete pendingSegments because there is at least one active task created at %s", activeTaskInterval.getStart() ); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 065bb99dcea4..4fa507e164b1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -120,7 +120,7 @@ public OverlordResource( JacksonConfigManager configManager, AuditManager auditManager, AuthorizerMapper authorizerMapper - ) throws Exception + ) { this.taskMaster = taskMaster; this.taskStorageQueryAdapter = taskStorageQueryAdapter; @@ -153,7 +153,7 @@ public Response taskPost( ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(authResult.getMessage()); } return asLeaderWith( @@ -504,6 +504,7 @@ public Response getCompleteTasks( .map(status -> new TaskStatusPlus( status.getId(), taskStorageQueryAdapter.getCreatedTime(status.getId()), + // Would be nice to include the real queue insertion time, but the TaskStorage API doesn't yet allow it. DateTimes.EPOCH, status.getStatusCode(), status.getDuration(), @@ -535,7 +536,7 @@ public Response killPendingSegments( ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(authResult.getMessage()); } if (taskMaster.isLeader()) { diff --git a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index b0320d773312..52067ab960c4 100644 --- a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -387,7 +387,7 @@ public SegmentIdentifier allocatePendingSegment( final Interval interval, final String maxVersion, final boolean skipSegmentLineageCheck - ) throws IOException + ) { Preconditions.checkNotNull(dataSource, "dataSource"); Preconditions.checkNotNull(sequenceName, "sequenceName"); diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java index e158d5000a04..40af4260324e 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java @@ -19,20 +19,25 @@ package io.druid.server.coordinator; +import com.google.common.base.Preconditions; import com.google.inject.Inject; import io.druid.client.ImmutableDruidDataSource; import io.druid.client.indexing.IndexingServiceClient; import io.druid.indexer.TaskStatusPlus; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.logger.Logger; import io.druid.server.coordinator.helper.DruidCoordinatorHelper; +import org.joda.time.DateTime; +import org.joda.time.Period; import java.util.ArrayList; -import java.util.Comparator; import java.util.List; public class DruidCoordinatorCleanupPendingSegments implements DruidCoordinatorHelper { private static final Logger log = new Logger(DruidCoordinatorCleanupPendingSegments.class); + private static final Period KEEP_PENDING_SEGMENTS_OFFSET = new Period("P1D"); private final IndexingServiceClient indexingServiceClient; @@ -45,28 +50,53 @@ public DruidCoordinatorCleanupPendingSegments(IndexingServiceClient indexingServ @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - final List taskStatusPluses = new ArrayList<>(); - taskStatusPluses.addAll(indexingServiceClient.getRunningTasks()); - taskStatusPluses.addAll(indexingServiceClient.getPendingTasks()); - taskStatusPluses.addAll(indexingServiceClient.getWaitingTasks()); + final List createdTimes = new ArrayList<>(); + createdTimes.add( + indexingServiceClient + .getRunningTasks() + .stream() + .map(TaskStatusPlus::getCreatedTime) + .min(Comparators.naturalNullsFirst()) + .orElse(DateTimes.nowUtc()) // If there is no running tasks, this returns the current time. + ); + createdTimes.add( + indexingServiceClient + .getPendingTasks() + .stream() + .map(TaskStatusPlus::getCreatedTime) + .min(Comparators.naturalNullsFirst()) + .orElse(DateTimes.nowUtc()) // If there is no pending tasks, this returns the current time. + ); + createdTimes.add( + indexingServiceClient + .getWaitingTasks() + .stream() + .map(TaskStatusPlus::getCreatedTime) + .min(Comparators.naturalNullsFirst()) + .orElse(DateTimes.nowUtc()) // If there is no waiting tasks, this returns the current time. + ); final TaskStatusPlus completeTaskStatus = indexingServiceClient.getLastCompleteTask(); if (completeTaskStatus != null) { - taskStatusPluses.add(completeTaskStatus); + createdTimes.add(completeTaskStatus.getCreatedTime()); } - taskStatusPluses.sort(Comparator.comparing(TaskStatusPlus::getCreatedTime)); + createdTimes.sort(Comparators.naturalNullsFirst()); - if (!taskStatusPluses.isEmpty()) { - final TaskStatusPlus firstCreatedTaskStatus = taskStatusPluses.get(0); - for (ImmutableDruidDataSource dataSource : params.getDataSources()) { - if (!params.getCoordinatorDynamicConfig().getKillPendingSegmentsSkipList().contains(dataSource.getName())) { - log.info( - "Kill pendingSegments created until [%s] for dataSource[%s]", - firstCreatedTaskStatus.getCreatedTime(), - dataSource - ); - indexingServiceClient.killPendingSegments(dataSource.getName(), firstCreatedTaskStatus.getCreatedTime()); - } + // There should be at least one createdTime because the current time is added to the 'createdTimes' list if there + // is no running/pending/waiting tasks. + Preconditions.checkState(!createdTimes.isEmpty(), "Failed to gather createdTimes of tasks"); + + // If there is no running/pending/waiting/complete tasks, pendingSegmentsCleanupEndTime is + // (DateTimes.nowUtc() - KEEP_PENDING_SEGMENTS_OFFSET). + final DateTime pendingSegmentsCleanupEndTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); + for (ImmutableDruidDataSource dataSource : params.getDataSources()) { + if (!params.getCoordinatorDynamicConfig().getKillPendingSegmentsSkipList().contains(dataSource.getName())) { + log.info( + "Killed [%d] pendingSegments created until [%s] for dataSource[%s]", + indexingServiceClient.killPendingSegments(dataSource.getName(), pendingSegmentsCleanupEndTime), + pendingSegmentsCleanupEndTime, + dataSource + ); } } return params; diff --git a/server/src/main/java/io/druid/server/security/Access.java b/server/src/main/java/io/druid/server/security/Access.java index c7bc32b9dd6e..bb76b12de8f5 100644 --- a/server/src/main/java/io/druid/server/security/Access.java +++ b/server/src/main/java/io/druid/server/security/Access.java @@ -26,7 +26,7 @@ public class Access public final static Access OK = new Access(true); private final boolean allowed; - private String message; + private final String message; public Access(boolean allowed) { @@ -44,10 +44,9 @@ public boolean isAllowed() return allowed; } - public Access setMessage(String message) + public String getMessage() { - this.message = message; - return this; + return message; } @Override diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index ee15a29ea844..5315a8a6552e 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -318,7 +318,6 @@ public void initialize(Server server, Injector injector) ) ); - final AuthConfig authConfig = injector.getInstance(AuthConfig.class); final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); final AuthenticatorMapper authenticatorMapper = injector.getInstance(AuthenticatorMapper.class); From 599b349d667e907a387891807031d3b13200f8bf Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 20 Dec 2017 20:33:59 +0900 Subject: [PATCH 7/9] fix build --- services/src/main/java/io/druid/cli/CliOverlord.java | 1 - 1 file changed, 1 deletion(-) diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 5315a8a6552e..ae075abd4b84 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -90,7 +90,6 @@ import io.druid.server.http.RedirectInfo; import io.druid.server.initialization.jetty.JettyServerInitUtils; import io.druid.server.initialization.jetty.JettyServerInitializer; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticationUtils; import io.druid.server.security.Authenticator; import io.druid.server.security.AuthenticatorMapper; From f8b4a7e2db31b4b50db6e475d6bce1a09331cc0f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 20 Dec 2017 20:56:53 +0900 Subject: [PATCH 8/9] fix test --- .../indexing/overlord/IndexerMetadataStorageAdapterTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java index 92ff99abee0b..872c85e5b816 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java @@ -92,7 +92,7 @@ public void testDeletePendingSegmentsOfRunningTasks() EasyMock.replay(taskStorageQueryAdapter, indexerMetadataStorageCoordinator); expectedException.expect(CoreMatchers.instanceOf(IllegalArgumentException.class)); - expectedException.expectMessage("Cannot delete pendingSegments because there is at least one running task created"); + expectedException.expectMessage("Cannot delete pendingSegments because there is at least one active task created"); indexerMetadataStorageAdapter.deletePendingSegments("dataSource", deleteInterval); } } From 5ae915c6502a033efaf7f85f8625103092549050 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 20 Dec 2017 21:26:39 +0900 Subject: [PATCH 9/9] fix test --- .../io/druid/indexing/overlord/http/OverlordResourceTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java index 4c338c9364d5..78242e8f8603 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java @@ -266,7 +266,6 @@ public void testSecuredGetRunningTasks() public void testSecuredTaskPost() { expectedException.expect(ForbiddenException.class); - expectedException.expectMessage("Allowed:false, Message:"); expectAuthorizationTokenCheck(); EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req);