You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/05/10 07:09:03 UTC

[GitHub] [druid] maytasm opened a new pull request #11227: Add feature to automatically remove datasource metadata based on retention period

maytasm opened a new pull request #11227:
URL: https://github.com/apache/druid/pull/11227


   Add feature to automatically remove datasource metadata based on retention period
   
   ### Description
   
   We currently already have tasklog auto cleanup (https://github.com/apache/druid/pull/3677) and audit logs auto cleanup (https://github.com/apache/druid/pull/11084). This PR adds a similar auto cleanup based on duration (time to retained) but for the datasource metadata table to auto clean up datasource that is no longer active -- meaning that the datasource does not have active supervisor running (Note: datasource metadata only exists for datasource created from supervisor).
   
   This is useful when Druid user has a high churn of task / datasource in a short amount of time causing the metadata store size to grow uncontrollably. 
   
   This PR has:
   - [x] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [x] added documentation for new or modified features or behaviors.
   - [x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [x] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [x] been tested in a test Druid cluster.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] suneet-s commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629573919



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/duty/KillDatasourceMetadata.java
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.coordinator.duty;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.supervisor.NoopSupervisorSpec;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.metadata.MetadataSupervisorManager;
+import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * CoordinatorDuty for automatic deletion of datasource metadata from the datasource table in metadata storage.
+ * (Note: datasource metadata only exists for datasource created from supervisor).
+ */
+public class KillDatasourceMetadata implements CoordinatorDuty
+{
+  private static final Logger log = new Logger(KillDatasourceMetadata.class);
+
+  private final long period;
+  private final long retainDuration;
+  private long lastKillTime = 0;
+
+  private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
+  private final MetadataSupervisorManager metadataSupervisorManager;
+
+  @Inject
+  public KillDatasourceMetadata(
+      DruidCoordinatorConfig config,
+      IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
+      MetadataSupervisorManager metadataSupervisorManager
+  )
+  {
+    this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
+    this.metadataSupervisorManager = metadataSupervisorManager;
+    this.period = config.getCoordinatorDatasourceKillPeriod().getMillis();
+    Preconditions.checkArgument(
+        this.period >= config.getCoordinatorMetadataStoreManagementPeriod().getMillis(),
+        "Coordinator datasource metadata kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod"
+    );
+    this.retainDuration = config.getCoordinatorDatasourceKillDurationToRetain().getMillis();
+    Preconditions.checkArgument(this.retainDuration >= 0, "Coordinator datasource metadata kill retainDuration must be >= 0");
+    log.debug(
+        "Datasource Metadata Kill Task scheduling enabled with period [%s], retainDuration [%s]",
+        this.period,
+        this.retainDuration
+    );
+  }
+
+  @Override
+  public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
+  {
+    if ((lastKillTime + period) < System.currentTimeMillis()) {
+      lastKillTime = System.currentTimeMillis();
+      long timestamp = System.currentTimeMillis() - retainDuration;

Review comment:
       nit: use consistent timestamp for all calculations
   
   ```suggestion
       long currentTimeMillis = System.currentTimeMillis();
       if ((lastKillTime + period) < currentTimeMillis) {
         lastKillTime = currentTimeMillis;
         long timestamp = currentTimeMillis - retainDuration;
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629610774



##########
File path: server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java
##########
@@ -267,6 +267,15 @@ SegmentPublishResult announceHistoricalSegments(
    */
   boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata dataSourceMetadata);
 
+  /**
+   * Remove datasource metadata created before the given timestamp and not in given excludeDatasources set.
+   *
+   * @param timestamp timestamp in milliseconds
+   * @param excludeDatasources set of datasource names to exclude from removal

Review comment:
       excludeDatasources should not be null. It can be empty which means no exclusion 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629610611



##########
File path: server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
##########
@@ -1423,4 +1424,40 @@ public boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata me
             .execute()
     );
   }
+
+  @Override
+  public int removeDataSourceMetadataOlderThan(long timestamp, @Nullable Set<String> excludeDatasources)
+  {
+    DateTime dateTime = DateTimes.utc(timestamp);
+    List<String> datasources = connector.getDBI().withHandle(
+        handle -> handle
+            .createQuery(
+                StringUtils.format(
+                    "SELECT dataSource FROM %1$s WHERE created_date < '%2$s'",
+                    dbTables.getDataSourceTable(),
+                    dateTime.toString()
+                )
+            )
+            .mapTo(String.class)
+            .list()
+    );
+    return connector.getDBI().withHandle(
+        handle -> {
+          final PreparedBatch batch = handle.prepareBatch(
+              StringUtils.format(
+                  "DELETE FROM %1$s WHERE dataSource = :dataSource AND created_date < '%2$s'",
+                  dbTables.getDataSourceTable(),
+                  dateTime.toString()
+              )
+          );
+          for (String datasourceMetadataInDb : datasources) {
+            if (!excludeDatasources.contains(datasourceMetadataInDb)) {

Review comment:
       excludeDatasources should be `@NotNull`. Fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629682360



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/duty/KillDatasourceMetadata.java
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.coordinator.duty;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.supervisor.NoopSupervisorSpec;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.metadata.MetadataSupervisorManager;
+import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * CoordinatorDuty for automatic deletion of datasource metadata from the datasource table in metadata storage.
+ * (Note: datasource metadata only exists for datasource created from supervisor).
+ */
+public class KillDatasourceMetadata implements CoordinatorDuty
+{
+  private static final Logger log = new Logger(KillDatasourceMetadata.class);
+
+  private final long period;
+  private final long retainDuration;
+  private long lastKillTime = 0;
+
+  private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
+  private final MetadataSupervisorManager metadataSupervisorManager;
+
+  @Inject
+  public KillDatasourceMetadata(
+      DruidCoordinatorConfig config,
+      IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
+      MetadataSupervisorManager metadataSupervisorManager
+  )
+  {
+    this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
+    this.metadataSupervisorManager = metadataSupervisorManager;
+    this.period = config.getCoordinatorDatasourceKillPeriod().getMillis();
+    Preconditions.checkArgument(
+        this.period >= config.getCoordinatorMetadataStoreManagementPeriod().getMillis(),
+        "Coordinator datasource metadata kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod"
+    );
+    this.retainDuration = config.getCoordinatorDatasourceKillDurationToRetain().getMillis();
+    Preconditions.checkArgument(this.retainDuration >= 0, "Coordinator datasource metadata kill retainDuration must be >= 0");
+    log.debug(
+        "Datasource Metadata Kill Task scheduling enabled with period [%s], retainDuration [%s]",
+        this.period,
+        this.retainDuration
+    );
+  }
+
+  @Override
+  public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
+  {
+    if ((lastKillTime + period) < System.currentTimeMillis()) {
+      lastKillTime = System.currentTimeMillis();
+      long timestamp = System.currentTimeMillis() - retainDuration;

Review comment:
       Note that I intentionally didnt specific in the docs that retainDuration have to be less than current timestamp (although we do check against this condition to protect ourself from unexpected behavior), since this is a very unlikely scenario and I don't want to make the docs to be unnecessary verbose.  




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] suneet-s commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629580751



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/duty/KillDatasourceMetadata.java
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.coordinator.duty;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.supervisor.NoopSupervisorSpec;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.metadata.MetadataSupervisorManager;
+import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * CoordinatorDuty for automatic deletion of datasource metadata from the datasource table in metadata storage.
+ * (Note: datasource metadata only exists for datasource created from supervisor).
+ */
+public class KillDatasourceMetadata implements CoordinatorDuty
+{
+  private static final Logger log = new Logger(KillDatasourceMetadata.class);
+
+  private final long period;
+  private final long retainDuration;
+  private long lastKillTime = 0;
+
+  private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
+  private final MetadataSupervisorManager metadataSupervisorManager;
+
+  @Inject
+  public KillDatasourceMetadata(
+      DruidCoordinatorConfig config,
+      IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
+      MetadataSupervisorManager metadataSupervisorManager
+  )
+  {
+    this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
+    this.metadataSupervisorManager = metadataSupervisorManager;
+    this.period = config.getCoordinatorDatasourceKillPeriod().getMillis();
+    Preconditions.checkArgument(
+        this.period >= config.getCoordinatorMetadataStoreManagementPeriod().getMillis(),
+        "Coordinator datasource metadata kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod"
+    );
+    this.retainDuration = config.getCoordinatorDatasourceKillDurationToRetain().getMillis();
+    Preconditions.checkArgument(this.retainDuration >= 0, "Coordinator datasource metadata kill retainDuration must be >= 0");
+    log.debug(
+        "Datasource Metadata Kill Task scheduling enabled with period [%s], retainDuration [%s]",
+        this.period,
+        this.retainDuration
+    );
+  }
+
+  @Override
+  public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
+  {
+    if ((lastKillTime + period) < System.currentTimeMillis()) {
+      lastKillTime = System.currentTimeMillis();
+      long timestamp = System.currentTimeMillis() - retainDuration;
+      // Datasource metadata only exists for datasource with supervisor
+      // To determine if datasource metadata is still active, we check if the supervisor for that particular datasource
+      // is still active or not
+      Map<String, SupervisorSpec> allSupervisor = metadataSupervisorManager.getLatest();
+      Set<String> allDatasourceWithActiveSupervisor = allSupervisor.values()
+                                                                   .stream()
+                                                                   // Terminated supervisor will have it's latest supervisorSpec as NoopSupervisorSpec
+                                                                   // (NoopSupervisorSpec is used as a tombstone marker)

Review comment:
       This logic is very similar to `SQLMetadataSupervisorManager#removeTerminatedSupervisorsOlderThan`
   
   Should that logic be moved out of the metadata store layer and pulled into the `KillSupervisors` class instead?
   
   Should this logic be shared so that other callers can easily find the "active" supervisors?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] suneet-s commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629569200



##########
File path: server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
##########
@@ -1423,4 +1424,40 @@ public boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata me
             .execute()
     );
   }
+
+  @Override
+  public int removeDataSourceMetadataOlderThan(long timestamp, @Nullable Set<String> excludeDatasources)
+  {
+    DateTime dateTime = DateTimes.utc(timestamp);
+    List<String> datasources = connector.getDBI().withHandle(
+        handle -> handle
+            .createQuery(
+                StringUtils.format(
+                    "SELECT dataSource FROM %1$s WHERE created_date < '%2$s'",
+                    dbTables.getDataSourceTable(),
+                    dateTime.toString()
+                )
+            )
+            .mapTo(String.class)
+            .list()
+    );
+    return connector.getDBI().withHandle(
+        handle -> {
+          final PreparedBatch batch = handle.prepareBatch(
+              StringUtils.format(
+                  "DELETE FROM %1$s WHERE dataSource = :dataSource AND created_date < '%2$s'",

Review comment:
       Why did you choose to build the delete statements one at a time instead of doing a batch delete?
   
   I think we could encapsulate the `excludeDatasources` logic in a where clause of this delete statement instead.
   
   Something like `DELETE FROM datasources where created_date < "date" and datasource not in ("excludeDataSources")`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629671791



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/duty/KillDatasourceMetadata.java
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.coordinator.duty;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.supervisor.NoopSupervisorSpec;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.metadata.MetadataSupervisorManager;
+import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * CoordinatorDuty for automatic deletion of datasource metadata from the datasource table in metadata storage.
+ * (Note: datasource metadata only exists for datasource created from supervisor).
+ */
+public class KillDatasourceMetadata implements CoordinatorDuty
+{
+  private static final Logger log = new Logger(KillDatasourceMetadata.class);
+
+  private final long period;
+  private final long retainDuration;
+  private long lastKillTime = 0;
+
+  private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
+  private final MetadataSupervisorManager metadataSupervisorManager;
+
+  @Inject
+  public KillDatasourceMetadata(
+      DruidCoordinatorConfig config,
+      IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
+      MetadataSupervisorManager metadataSupervisorManager
+  )
+  {
+    this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
+    this.metadataSupervisorManager = metadataSupervisorManager;
+    this.period = config.getCoordinatorDatasourceKillPeriod().getMillis();
+    Preconditions.checkArgument(
+        this.period >= config.getCoordinatorMetadataStoreManagementPeriod().getMillis(),
+        "Coordinator datasource metadata kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod"
+    );
+    this.retainDuration = config.getCoordinatorDatasourceKillDurationToRetain().getMillis();
+    Preconditions.checkArgument(this.retainDuration >= 0, "Coordinator datasource metadata kill retainDuration must be >= 0");
+    log.debug(
+        "Datasource Metadata Kill Task scheduling enabled with period [%s], retainDuration [%s]",
+        this.period,
+        this.retainDuration
+    );
+  }
+
+  @Override
+  public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
+  {
+    if ((lastKillTime + period) < System.currentTimeMillis()) {
+      lastKillTime = System.currentTimeMillis();
+      long timestamp = System.currentTimeMillis() - retainDuration;
+      // Datasource metadata only exists for datasource with supervisor
+      // To determine if datasource metadata is still active, we check if the supervisor for that particular datasource
+      // is still active or not
+      Map<String, SupervisorSpec> allSupervisor = metadataSupervisorManager.getLatest();
+      Set<String> allDatasourceWithActiveSupervisor = allSupervisor.values()
+                                                                   .stream()
+                                                                   // Terminated supervisor will have it's latest supervisorSpec as NoopSupervisorSpec
+                                                                   // (NoopSupervisorSpec is used as a tombstone marker)

Review comment:
       I added some convenience methods in  SQLMetadataSupervisorManager to getLatestTerminatedOnly and getLatestActiveOnly supervisors




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629610611



##########
File path: server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
##########
@@ -1423,4 +1424,40 @@ public boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata me
             .execute()
     );
   }
+
+  @Override
+  public int removeDataSourceMetadataOlderThan(long timestamp, @Nullable Set<String> excludeDatasources)
+  {
+    DateTime dateTime = DateTimes.utc(timestamp);
+    List<String> datasources = connector.getDBI().withHandle(
+        handle -> handle
+            .createQuery(
+                StringUtils.format(
+                    "SELECT dataSource FROM %1$s WHERE created_date < '%2$s'",
+                    dbTables.getDataSourceTable(),
+                    dateTime.toString()
+                )
+            )
+            .mapTo(String.class)
+            .list()
+    );
+    return connector.getDBI().withHandle(
+        handle -> {
+          final PreparedBatch batch = handle.prepareBatch(
+              StringUtils.format(
+                  "DELETE FROM %1$s WHERE dataSource = :dataSource AND created_date < '%2$s'",
+                  dbTables.getDataSourceTable(),
+                  dateTime.toString()
+              )
+          );
+          for (String datasourceMetadataInDb : datasources) {
+            if (!excludeDatasources.contains(datasourceMetadataInDb)) {

Review comment:
       excludeDatasources should be @NotNull. Fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] suneet-s commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629576496



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/duty/KillDatasourceMetadata.java
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.coordinator.duty;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.supervisor.NoopSupervisorSpec;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.metadata.MetadataSupervisorManager;
+import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * CoordinatorDuty for automatic deletion of datasource metadata from the datasource table in metadata storage.
+ * (Note: datasource metadata only exists for datasource created from supervisor).
+ */
+public class KillDatasourceMetadata implements CoordinatorDuty
+{
+  private static final Logger log = new Logger(KillDatasourceMetadata.class);
+
+  private final long period;
+  private final long retainDuration;
+  private long lastKillTime = 0;
+
+  private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
+  private final MetadataSupervisorManager metadataSupervisorManager;
+
+  @Inject
+  public KillDatasourceMetadata(
+      DruidCoordinatorConfig config,
+      IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
+      MetadataSupervisorManager metadataSupervisorManager
+  )
+  {
+    this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
+    this.metadataSupervisorManager = metadataSupervisorManager;
+    this.period = config.getCoordinatorDatasourceKillPeriod().getMillis();
+    Preconditions.checkArgument(
+        this.period >= config.getCoordinatorMetadataStoreManagementPeriod().getMillis(),
+        "Coordinator datasource metadata kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod"
+    );
+    this.retainDuration = config.getCoordinatorDatasourceKillDurationToRetain().getMillis();
+    Preconditions.checkArgument(this.retainDuration >= 0, "Coordinator datasource metadata kill retainDuration must be >= 0");
+    log.debug(
+        "Datasource Metadata Kill Task scheduling enabled with period [%s], retainDuration [%s]",
+        this.period,
+        this.retainDuration
+    );
+  }
+
+  @Override
+  public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
+  {
+    if ((lastKillTime + period) < System.currentTimeMillis()) {
+      lastKillTime = System.currentTimeMillis();
+      long timestamp = System.currentTimeMillis() - retainDuration;

Review comment:
       Additional question - I notice this pattern in a few other co-ordinator duties.
   
   Are there any additional safeguards we need for a very large `retainDuration`? What happens if `timestamp` is calculated to be negative?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629620902



##########
File path: server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
##########
@@ -1423,4 +1424,40 @@ public boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata me
             .execute()
     );
   }
+
+  @Override
+  public int removeDataSourceMetadataOlderThan(long timestamp, @Nullable Set<String> excludeDatasources)
+  {
+    DateTime dateTime = DateTimes.utc(timestamp);
+    List<String> datasources = connector.getDBI().withHandle(
+        handle -> handle
+            .createQuery(
+                StringUtils.format(
+                    "SELECT dataSource FROM %1$s WHERE created_date < '%2$s'",
+                    dbTables.getDataSourceTable(),
+                    dateTime.toString()
+                )
+            )
+            .mapTo(String.class)
+            .list()
+    );
+    return connector.getDBI().withHandle(
+        handle -> {
+          final PreparedBatch batch = handle.prepareBatch(
+              StringUtils.format(
+                  "DELETE FROM %1$s WHERE dataSource = :dataSource AND created_date < '%2$s'",

Review comment:
       This is to prevent the IN clause being too large




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629655853



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/duty/KillDatasourceMetadata.java
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.coordinator.duty;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.inject.Inject;
+import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.supervisor.NoopSupervisorSpec;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.metadata.MetadataSupervisorManager;
+import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * CoordinatorDuty for automatic deletion of datasource metadata from the datasource table in metadata storage.
+ * (Note: datasource metadata only exists for datasource created from supervisor).
+ */
+public class KillDatasourceMetadata implements CoordinatorDuty
+{
+  private static final Logger log = new Logger(KillDatasourceMetadata.class);
+
+  private final long period;
+  private final long retainDuration;
+  private long lastKillTime = 0;
+
+  private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
+  private final MetadataSupervisorManager metadataSupervisorManager;
+
+  @Inject
+  public KillDatasourceMetadata(
+      DruidCoordinatorConfig config,
+      IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
+      MetadataSupervisorManager metadataSupervisorManager
+  )
+  {
+    this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
+    this.metadataSupervisorManager = metadataSupervisorManager;
+    this.period = config.getCoordinatorDatasourceKillPeriod().getMillis();
+    Preconditions.checkArgument(
+        this.period >= config.getCoordinatorMetadataStoreManagementPeriod().getMillis(),
+        "Coordinator datasource metadata kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod"
+    );
+    this.retainDuration = config.getCoordinatorDatasourceKillDurationToRetain().getMillis();
+    Preconditions.checkArgument(this.retainDuration >= 0, "Coordinator datasource metadata kill retainDuration must be >= 0");
+    log.debug(
+        "Datasource Metadata Kill Task scheduling enabled with period [%s], retainDuration [%s]",
+        this.period,
+        this.retainDuration
+    );
+  }
+
+  @Override
+  public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
+  {
+    if ((lastKillTime + period) < System.currentTimeMillis()) {
+      lastKillTime = System.currentTimeMillis();
+      long timestamp = System.currentTimeMillis() - retainDuration;

Review comment:
       Added safeguard so that we never get calculated `timestamp` to be negative




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629618963



##########
File path: server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
##########
@@ -1423,4 +1424,40 @@ public boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata me
             .execute()
     );
   }
+
+  @Override
+  public int removeDataSourceMetadataOlderThan(long timestamp, @Nullable Set<String> excludeDatasources)
+  {
+    DateTime dateTime = DateTimes.utc(timestamp);
+    List<String> datasources = connector.getDBI().withHandle(
+        handle -> handle
+            .createQuery(
+                StringUtils.format(
+                    "SELECT dataSource FROM %1$s WHERE created_date < '%2$s'",
+                    dbTables.getDataSourceTable(),
+                    dateTime.toString()
+                )
+            )
+            .mapTo(String.class)
+            .list()
+    );
+    return connector.getDBI().withHandle(

Review comment:
       Added try catch block




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] suneet-s commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629565383



##########
File path: server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java
##########
@@ -267,6 +267,15 @@ SegmentPublishResult announceHistoricalSegments(
    */
   boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata dataSourceMetadata);
 
+  /**
+   * Remove datasource metadata created before the given timestamp and not in given excludeDatasources set.
+   *
+   * @param timestamp timestamp in milliseconds
+   * @param excludeDatasources set of datasource names to exclude from removal

Review comment:
       What does a null `excludeDatasources` mean?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] suneet-s commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629564966



##########
File path: server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
##########
@@ -1423,4 +1424,40 @@ public boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata me
             .execute()
     );
   }
+
+  @Override
+  public int removeDataSourceMetadataOlderThan(long timestamp, @Nullable Set<String> excludeDatasources)
+  {
+    DateTime dateTime = DateTimes.utc(timestamp);
+    List<String> datasources = connector.getDBI().withHandle(
+        handle -> handle
+            .createQuery(
+                StringUtils.format(
+                    "SELECT dataSource FROM %1$s WHERE created_date < '%2$s'",
+                    dbTables.getDataSourceTable(),
+                    dateTime.toString()
+                )
+            )
+            .mapTo(String.class)
+            .list()
+    );
+    return connector.getDBI().withHandle(
+        handle -> {
+          final PreparedBatch batch = handle.prepareBatch(
+              StringUtils.format(
+                  "DELETE FROM %1$s WHERE dataSource = :dataSource AND created_date < '%2$s'",
+                  dbTables.getDataSourceTable(),
+                  dateTime.toString()
+              )
+          );
+          for (String datasourceMetadataInDb : datasources) {
+            if (!excludeDatasources.contains(datasourceMetadataInDb)) {

Review comment:
       possible NPE - `excludeDatasources` is marked as nullable in the function definition




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629652513



##########
File path: server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
##########
@@ -1423,4 +1424,40 @@ public boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata me
             .execute()
     );
   }
+
+  @Override
+  public int removeDataSourceMetadataOlderThan(long timestamp, @Nullable Set<String> excludeDatasources)
+  {
+    DateTime dateTime = DateTimes.utc(timestamp);
+    List<String> datasources = connector.getDBI().withHandle(
+        handle -> handle
+            .createQuery(
+                StringUtils.format(
+                    "SELECT dataSource FROM %1$s WHERE created_date < '%2$s'",
+                    dbTables.getDataSourceTable(),
+                    dateTime.toString()
+                )
+            )
+            .mapTo(String.class)
+            .list()
+    );
+    return connector.getDBI().withHandle(
+        handle -> {
+          final PreparedBatch batch = handle.prepareBatch(
+              StringUtils.format(
+                  "DELETE FROM %1$s WHERE dataSource = :dataSource AND created_date < '%2$s'",

Review comment:
       Also moved the filtering of the datasources to outside the handle block.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm merged pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
maytasm merged pull request #11227:
URL: https://github.com/apache/druid/pull/11227


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] suneet-s commented on a change in pull request #11227: Add feature to automatically remove datasource metadata based on retention period

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #11227:
URL: https://github.com/apache/druid/pull/11227#discussion_r629566613



##########
File path: server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
##########
@@ -1423,4 +1424,40 @@ public boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata me
             .execute()
     );
   }
+
+  @Override
+  public int removeDataSourceMetadataOlderThan(long timestamp, @Nullable Set<String> excludeDatasources)
+  {
+    DateTime dateTime = DateTimes.utc(timestamp);
+    List<String> datasources = connector.getDBI().withHandle(
+        handle -> handle
+            .createQuery(
+                StringUtils.format(
+                    "SELECT dataSource FROM %1$s WHERE created_date < '%2$s'",
+                    dbTables.getDataSourceTable(),
+                    dateTime.toString()
+                )
+            )
+            .mapTo(String.class)
+            .list()
+    );
+    return connector.getDBI().withHandle(

Review comment:
       What happens if an exception is thrown while trying to delete the  datasources? `withHandle` will throw a `CallbackFailedException` - is this handled somewhere else in the code?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org