From 40e479ed65ac1bad0dabe9d6f1574c041bc27cfe Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 6 Apr 2022 19:57:29 +0530 Subject: [PATCH 01/16] Optimize overlord GET /tasks memory usage --- .../apache/druid/indexer/TaskStatusPlus.java | 10 +- .../MetadataStorageActionHandler.java | 21 ++- .../MetadataStorageActionHandlerFactory.java | 2 +- .../overlord/HeapMemoryTaskStorage.java | 11 ++ .../overlord/MetadataTaskStorage.java | 27 ++- .../druid/indexing/overlord/TaskStorage.java | 18 ++ .../overlord/TaskStorageQueryAdapter.java | 10 + .../overlord/http/OverlordResource.java | 106 ++++++++++- .../DerbyMetadataStorageActionHandler.java | 4 +- ...byMetadataStorageActionHandlerFactory.java | 2 +- .../MySQLMetadataStorageActionHandler.java | 4 +- ...QLMetadataStorageActionHandlerFactory.java | 4 +- ...ostgreSQLMetadataStorageActionHandler.java | 4 +- ...QLMetadataStorageActionHandlerFactory.java | 4 +- .../SQLMetadataStorageActionHandler.java | 171 +++++++++++++++++- ...SQLServerMetadataStorageActionHandler.java | 4 +- ...erMetadataStorageActionHandlerFactory.java | 4 +- .../SQLMetadataStorageActionHandlerTest.java | 2 +- 18 files changed, 379 insertions(+), 29 deletions(-) diff --git a/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java b/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java index 75bbefa0eaf3..0aba225ab36c 100644 --- a/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java +++ b/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java @@ -35,7 +35,7 @@ public class TaskStatusPlus private final DateTime createdTime; private final DateTime queueInsertionTime; private final TaskState statusCode; - private final RunnerTaskState runnerTaskState; + private RunnerTaskState runnerTaskState; private final Long duration; private final TaskLocation location; private final String dataSource; @@ -197,6 +197,14 @@ public String getErrorMsg() return errorMsg; } + public void setRunnerTaskState(RunnerTaskState runnerTaskState) + { + if (this.runnerTaskState != null) { + throw new IllegalAccessError("Cannot set runnerTaskState since it has already been set"); + } + this.runnerTaskState = runnerTaskState; + } + @Override public boolean equals(Object o) { diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index b4d50aaa3d2e..292225a73ff9 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -21,6 +21,7 @@ import com.google.common.base.Optional; import org.apache.druid.indexer.TaskInfo; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.metadata.TaskLookup.TaskLookupType; import org.joda.time.DateTime; @@ -30,7 +31,7 @@ import java.util.List; import java.util.Map; -public interface MetadataStorageActionHandler +public interface MetadataStorageActionHandler { /** * Creates a new entry. @@ -83,6 +84,24 @@ void insert( @Nullable TaskInfo getTaskInfo(String entryId); + /** + * Returns a list of {@link TaskStatusPlus} from metadata store matching the given filters. + * + * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. + * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata + * store. For complete tasks, additional filters in {@code CompleteTaskLookup} can be applied. + * All lookups should be processed atomically if there are more than one lookup is given. + * + * @param taskLookups task lookup type and filters. + * @param datasource datasource filter + * @param runnerWorkItems map of id to work items for running tasks + */ + List getTaskStatusPlusList( + Map taskLookups, + @Nullable String datasource, + Map runnerWorkItems + ); + /** * Returns a list of {@link TaskInfo} from metadata store that matches to the given filters. * diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerFactory.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerFactory.java index a81ecfdbf1c4..4770c0dc885c 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerFactory.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerFactory.java @@ -21,7 +21,7 @@ public interface MetadataStorageActionHandlerFactory { - MetadataStorageActionHandler create( + MetadataStorageActionHandler create( String entryType, MetadataStorageActionHandlerTypes payloadTypes ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index de9ebb728125..5365982bd3d8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -30,6 +30,7 @@ import com.google.inject.Inject; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.config.TaskStorageConfig; @@ -233,6 +234,16 @@ public List> getTaskInfos( return tasks; } + @Override + public List getTaskStatusPlusList( + Map taskLookups, + @Nullable String datasource, + Map runnerWorkItems + ) + { + return Collections.emptyList(); + } + private List> getRecentlyCreatedAlreadyFinishedTaskInfoSince( DateTime start, @Nullable Integer n, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 86597d5cfd25..590244926bf9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -29,6 +29,7 @@ import com.google.inject.Inject; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.config.TaskStorageConfig; @@ -95,7 +96,7 @@ public TypeReference getLockType() private final MetadataStorageConnector metadataStorageConnector; private final TaskStorageConfig config; - private final MetadataStorageActionHandler handler; + private final MetadataStorageActionHandler handler; private static final EmittingLogger log = new EmittingLogger(MetadataTaskStorage.class); @@ -220,6 +221,30 @@ public List getActiveTasksByDatasource(String datasource) return tasksBuilder.build(); } + @Override + public List getTaskStatusPlusList( + Map taskLookups, + @Nullable String datasource, + Map runnerWorkItems + ) + { + Map theTaskLookups = Maps.newHashMapWithExpectedSize(taskLookups.size()); + for (Entry entry : taskLookups.entrySet()) { + if (entry.getKey() == TaskLookupType.COMPLETE) { + CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); + theTaskLookups.put( + entry.getKey(), + completeTaskLookup.hasTaskCreatedTimeFilter() + ? completeTaskLookup + : completeTaskLookup.withDurationBeforeNow(config.getRecentlyFinishedThreshold()) + ); + } else { + theTaskLookups.put(entry.getKey(), entry.getValue()); + } + } + return Collections.unmodifiableList(handler.getTaskStatusPlusList(theTaskLookups, datasource, runnerWorkItems)); + } + @Override public List> getTaskInfos( Map taskLookups, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java index cf858ebcc300..7a71711ea103 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java @@ -22,6 +22,7 @@ import com.google.common.base.Optional; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.task.Task; @@ -150,6 +151,23 @@ public interface TaskStorage */ List getActiveTasksByDatasource(String datasource); + /** + * Returns a list of tasks' relevant status info stored in the storage facility as {@link TaskStatusPlus}. No + * particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. + * + * The returned list can contain active tasks and complete tasks depending on the {@code taskLookups} parameter. + * See {@link TaskLookup} for more details of active and complete tasks. + * + * @param taskLookups lookup types and filters + * @param datasource datasource filter + * @param runnerWorkItems + */ + List getTaskStatusPlusList( + Map taskLookups, + @Nullable String datasource, + Map runnerWorkItems + ); + /** * Returns a list of tasks stored in the storage facility as {@link TaskInfo}. No * particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java index f4338c909674..fa65d735208d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -23,6 +23,7 @@ import com.google.inject.Inject; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.actions.SegmentInsertAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskAction; @@ -102,6 +103,15 @@ public List> getTaskInfos( return storage.getTaskInfos(taskLookups, dataSource); } + public List getTaskStatusPlusList( + Map taskLookups, + @Nullable String datasource, + Map runnerWorkItems + ) + { + return storage.getTaskStatusPlusList(taskLookups, datasource, runnerWorkItems); + } + public Optional getTask(final String taskid) { return storage.getTask(taskid); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index 638fbf2df4e9..83d6f2ab458a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -690,7 +690,7 @@ public Response getTasks( taskMaster.getTaskRunner(), taskRunner -> { final List authorizedList = securedTaskStatusPlus( - getTasks( + getTaskStatusPlusList( taskRunner, TaskStateLookup.fromString(state), dataSource, @@ -706,6 +706,58 @@ public Response getTasks( ); } + private List getTaskStatusPlusList( + TaskRunner taskRunner, + TaskStateLookup state, + @Nullable String dataSource, + @Nullable String createdTimeInterval, + @Nullable Integer maxCompletedTasks, + @Nullable String type + ) + { + final Duration createdTimeDuration; + if (createdTimeInterval != null) { + final Interval theInterval = Intervals.of(StringUtils.replace(createdTimeInterval, "_", "/")); + createdTimeDuration = theInterval.toDuration(); + } else { + createdTimeDuration = null; + } + + // This way, we can use the snapshot from taskStorage as the source of truth for the set of tasks to process + // and use the snapshot from taskRunner as a reference for potential task state updates happened + // after the first snapshotting. + final Map runnerWorkItems = getTaskRunnerWorkItems( + taskRunner, + state, + dataSource, + type + ); + Stream taskStatusPlusStream = getTaskStatusPlusStreamFromStorage( + state, + dataSource, + createdTimeDuration, + maxCompletedTasks, + type, + runnerWorkItems + ); + + if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING) { + // We are interested in only those tasks which are in taskRunner. + taskStatusPlusStream = taskStatusPlusStream + .filter(taskStatusPlus -> runnerWorkItems.containsKey(taskStatusPlus.getId())); + } + final List taskStatusPlusList = taskStatusPlusStream.collect(Collectors.toList()); + for (TaskStatusPlus taskStatusPlus : taskStatusPlusList) { + if (taskStatusPlus.getRunnerStatusCode() == null) { + // this is racy for remoteTaskRunner + RunnerTaskState runnerTaskState = taskRunner.getRunnerTaskState(taskStatusPlus.getId()); + taskStatusPlus.setRunnerTaskState(runnerTaskState); + } + } + + return taskStatusPlusList; + } + private List getTasks( TaskRunner taskRunner, TaskStateLookup state, @@ -825,6 +877,58 @@ private List getTasks( return statuses; } + private Stream getTaskStatusPlusStreamFromStorage( + TaskStateLookup state, + @Nullable String dataSource, + Duration createdTimeDuration, + @Nullable Integer maxCompletedTasks, + @Nullable String type, + Map runnerWorkItems + ) + { + final Map taskLookups; + switch (state) { + case ALL: + taskLookups = ImmutableMap.of( + TaskLookupType.ACTIVE, + ActiveTaskLookup.getInstance(), + TaskLookupType.COMPLETE, + CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) + ); + break; + case COMPLETE: + taskLookups = ImmutableMap.of( + TaskLookupType.COMPLETE, + CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) + ); + break; + case WAITING: + case PENDING: + case RUNNING: + taskLookups = ImmutableMap.of( + TaskLookupType.ACTIVE, + ActiveTaskLookup.getInstance() + ); + break; + default: + throw new IAE("Unknown state: [%s]", state); + } + + final Stream taskStatusPlusStream = + taskStorageQueryAdapter.getTaskStatusPlusList( + taskLookups, + dataSource, + runnerWorkItems + ).stream(); + if (type != null) { + return taskStatusPlusStream.filter( + taskStatusPlus -> type.equals(taskStatusPlus == null ? null : taskStatusPlus.getType()) + ); + } else { + return taskStatusPlusStream; + } + } + private Stream> getTaskInfoStreamFromTaskStorage( TaskStateLookup state, @Nullable String dataSource, diff --git a/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java index fa39d506b4cc..3fa24979e9c6 100644 --- a/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java @@ -23,8 +23,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.java.util.common.StringUtils; -public class DerbyMetadataStorageActionHandler - extends SQLMetadataStorageActionHandler +public class DerbyMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler { @VisibleForTesting DerbyMetadataStorageActionHandler( diff --git a/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java index cf6dc44ed93b..3a102d53744e 100644 --- a/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java +++ b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java @@ -35,7 +35,7 @@ public DerbyMetadataStorageActionHandlerFactory( } @Override - public MetadataStorageActionHandler create( + public MetadataStorageActionHandler create( final String entryType, MetadataStorageActionHandlerTypes payloadTypes ) diff --git a/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandler.java index 79c028202f41..70f8e956f515 100644 --- a/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandler.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; -public class MySQLMetadataStorageActionHandler - extends SQLMetadataStorageActionHandler +public class MySQLMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler { MySQLMetadataStorageActionHandler( SQLMetadataConnector connector, diff --git a/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java b/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java index 0af63ba1d631..a5291710a5fb 100644 --- a/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java +++ b/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java @@ -35,8 +35,8 @@ public MySQLMetadataStorageActionHandlerFactory( } @Override - public - MetadataStorageActionHandler create( + public + MetadataStorageActionHandler create( String entryType, MetadataStorageActionHandlerTypes payloadTypes ) diff --git a/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java index b91427bcd662..e9c06ca11420 100644 --- a/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.java.util.common.StringUtils; -public class PostgreSQLMetadataStorageActionHandler - extends SQLMetadataStorageActionHandler +public class PostgreSQLMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler { public PostgreSQLMetadataStorageActionHandler( SQLMetadataConnector connector, diff --git a/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java b/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java index 329b7a52474b..d55c86ce4cfc 100644 --- a/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java +++ b/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java @@ -35,8 +35,8 @@ public PostgreSQLMetadataStorageActionHandlerFactory( } @Override - public - MetadataStorageActionHandler create( + public + MetadataStorageActionHandler create( String entryType, MetadataStorageActionHandlerTypes payloadTypes ) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index 0b8837c640dc..ed79cb74250e 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -22,10 +22,16 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Maps; +import net.minidev.json.parser.JSONParser; +import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskInfo; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; @@ -54,13 +60,14 @@ import java.util.Map; import java.util.Map.Entry; -public abstract class SQLMetadataStorageActionHandler - implements MetadataStorageActionHandler +public abstract class SQLMetadataStorageActionHandler + implements MetadataStorageActionHandler { private static final EmittingLogger log = new EmittingLogger(SQLMetadataStorageActionHandler.class); private final SQLMetadataConnector connector; private final ObjectMapper jsonMapper; + private final JSONParser jsonParser; private final TypeReference entryType; private final TypeReference statusType; private final TypeReference logType; @@ -89,6 +96,7 @@ public SQLMetadataStorageActionHandler( //noinspection UnnecessaryFullyQualifiedName this.jsonMapper = jsonMapper.copy().addMixIn(org.apache.druid.metadata.PasswordProvider.class, org.apache.druid.metadata.PasswordProviderRedactionMixIn.class); + this.jsonParser = new JSONParser(); this.entryType = types.getEntryType(); this.statusType = types.getStatusType(); this.logType = types.getLogType(); @@ -271,6 +279,48 @@ public TaskInfo getTaskInfo(String entryId) }); } + @Override + public List getTaskStatusPlusList( + Map taskLookups, + @Nullable String dataSource, + Map runnerWorkItems + ) + { + TaskStatusPlusMapper taskStatusPlusMapper = new TaskStatusPlusMapper<>(jsonMapper, runnerWorkItems); + return getConnector().retryTransaction( + (handle, status) -> { + final List tasks = new ArrayList<>(); + for (Entry entry : taskLookups.entrySet()) { + final Query> query; + switch (entry.getKey()) { + case ACTIVE: + query = createActiveTaskInfoStreamingQuery( + handle, + dataSource + ); + tasks.addAll(query.map(taskStatusPlusMapper).list()); + break; + case COMPLETE: + CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); + query = createCompletedTaskInfoStreamingQuery( + handle, + completeTaskLookup.getTasksCreatedPriorTo(), + completeTaskLookup.getMaxTaskStatuses(), + dataSource + ); + tasks.addAll(query.map(taskStatusPlusMapper).list()); + break; + default: + throw new IAE("Unknown TaskLookupType: [%s]", entry.getKey()); + } + } + return tasks; + }, + 3, + SQLMetadataConnector.DEFAULT_MAX_TRIES + ); + } + @Override public List> getTaskInfos( Map taskLookups, @@ -284,7 +334,7 @@ public List> getTaskInfos( final Query> query; switch (entry.getKey()) { case ACTIVE: - query = createActiveTaskInfoQuery( + query = createActiveTaskInfoStreamingQuery( handle, dataSource ); @@ -292,7 +342,7 @@ public List> getTaskInfos( break; case COMPLETE: CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); - query = createCompletedTaskInfoQuery( + query = createCompletedTaskInfoStreamingQuery( handle, completeTaskLookup.getTasksCreatedPriorTo(), completeTaskLookup.getMaxTaskStatuses(), @@ -311,7 +361,7 @@ public List> getTaskInfos( ); } - protected Query> createCompletedTaskInfoQuery( + protected Query> createCompletedTaskInfoStreamingQuery( Handle handle, DateTime timestamp, @Nullable Integer maxNumStatuses, @@ -336,7 +386,9 @@ protected Query> createCompletedTaskInfoQuery( if (maxNumStatuses != null) { sql = decorateSqlWithLimit(sql); } - Query> query = handle.createQuery(sql).bind("start", timestamp.toString()); + Query> query = handle.createQuery(sql) + .bind("start", timestamp.toString()) + .setFetchSize(connector.getStreamingFetchSize()); if (maxNumStatuses != null) { query = query.bind("n", maxNumStatuses); @@ -358,7 +410,7 @@ private String getWhereClauseForInactiveStatusesSinceQuery(@Nullable String data return sql; } - private Query> createActiveTaskInfoQuery(Handle handle, @Nullable String dataSource) + private Query> createActiveTaskInfoStreamingQuery(Handle handle, @Nullable String dataSource) { String sql = StringUtils.format( "SELECT " @@ -375,7 +427,8 @@ private Query> createActiveTaskInfoQuery(Handle handle, @Nul entryTable ); - Query> query = handle.createQuery(sql); + Query> query = handle.createQuery(sql) + .setFetchSize(connector.getStreamingFetchSize()); if (dataSource != null) { query = query.bind("ds", dataSource); } @@ -391,6 +444,108 @@ private String getWhereClauseForActiveStatusesQuery(String dataSource) return sql; } + static class TaskStatusPlusMapper implements ResultSetMapper + { + private final ObjectMapper objectMapper; + private final Map runnerWorkItems; + + TaskStatusPlusMapper(ObjectMapper objectMapper, Map runnerWorkItems) + { + this.objectMapper = objectMapper; + this.runnerWorkItems = runnerWorkItems; + } + + @Override + public TaskStatusPlus map(int index, ResultSet resultSet, StatementContext context) + throws SQLException + { + ObjectNode task; + try { + task = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); + } + catch (IOException e) { + log.warn("Encountered exception[%s] while deserializing task payload, setting payload to null", + e.getMessage()); + task = null; + } + + ObjectNode status; + try { + status = objectMapper.readValue(resultSet.getBytes("status_payload"), ObjectNode.class); + } + catch (IOException e) { + log.error(e, "Encountered exception while deserializing task status_payload"); + throw new SQLException(e); + } + final String id = resultSet.getString("id"); + final String datasource = resultSet.getString("datasource"); + final DateTime createdDate = DateTimes.of(resultSet.getString("created_date")); + return getTaskStatusPlus(id, datasource, createdDate, task, status); + } + + private TaskStatusPlus getTaskStatusPlus(String id, String datasource, DateTime createdDate, + ObjectNode task, ObjectNode status) + { + final TaskState statusCode = TaskState.valueOf(status.get("status").asText()); + final TaskRunnerWorkItem runnerWorkItem = runnerWorkItems.get(id); + TaskLocation taskLocation; + if (!statusCode.isRunnable() || runnerWorkItem == null) { + // COMPLETED OR WAITING task + try { + taskLocation = objectMapper.treeToValue(status.get("location"), TaskLocation.class); + } + catch (JsonProcessingException e) { + throw new IllegalArgumentException(e); + } + TaskStatusPlus taskStatusPlus = new TaskStatusPlus( + id, + task == null ? null : task.get("groupId").asText(), + task == null ? null : task.get("type").asText(), + createdDate, + DateTimes.EPOCH, + statusCode, + null, // set below to NONE(completed) or WAITING + status.get("duration").asLong(), + taskLocation, + datasource, + status.get("errorMsg").asText() + ); + if (!statusCode.isRunnable()) { + taskStatusPlus.setRunnerTaskState(RunnerTaskState.NONE); + } else { + taskStatusPlus.setRunnerTaskState(RunnerTaskState.WAITING); + } + return taskStatusPlus; + } else { + DateTime createdTime; + DateTime queueInsertionTime; + try { + ObjectNode runnerWorkItemJson = objectMapper.convertValue(runnerWorkItem, ObjectNode.class); + createdTime = objectMapper.treeToValue(runnerWorkItemJson.get("createdTime"), DateTime.class); + queueInsertionTime = objectMapper.treeToValue(runnerWorkItemJson.get("queueInsertionTime"), DateTime.class); + // location in taskInfo is only updated after the task is done. + taskLocation = objectMapper.treeToValue(runnerWorkItemJson.get("location"), TaskLocation.class); + } + catch (JsonProcessingException e) { + throw new IllegalArgumentException(e); + } + return new TaskStatusPlus( + id, + task == null ? null : task.get("groupId").asText(), + task == null ? null : task.get("type").asText(), + createdTime, + queueInsertionTime, + statusCode, + null, // Set later in OverlordResource to taskRunner.getRunnerTaskState(id) + status.get("duration").asLong(), + taskLocation, + datasource, + status.get("errorMsg").asText() + ); + } + } + } + static class TaskInfoMapper implements ResultSetMapper> { private final ObjectMapper objectMapper; diff --git a/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandler.java index 2c4e93820d25..c6cd8c671e1e 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandler.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; -public class SQLServerMetadataStorageActionHandler - extends SQLMetadataStorageActionHandler +public class SQLServerMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler { public SQLServerMetadataStorageActionHandler( SQLMetadataConnector connector, diff --git a/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java b/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java index af6725175340..430fd5e932e9 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java @@ -35,8 +35,8 @@ public SQLServerMetadataStorageActionHandlerFactory( } @Override - public - MetadataStorageActionHandler create( + public + MetadataStorageActionHandler create( String entryType, MetadataStorageActionHandlerTypes payloadTypes ) diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 2e85e5a5d49e..75f1ae54a0c2 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -53,7 +53,7 @@ public class SQLMetadataStorageActionHandlerTest public final ExpectedException thrown = ExpectedException.none(); private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); - private SQLMetadataStorageActionHandler, Map, Map, Map> handler; + private SQLMetadataStorageActionHandler, Map, Map, Map, Map> handler; @Before public void setUp() From 98fca6686f504257b3cb2e6cdbd22f7a83d27367 Mon Sep 17 00:00:00 2001 From: Amatya Date: Sat, 9 Apr 2022 00:54:08 +0530 Subject: [PATCH 02/16] Use extra columns for better perf --- .../MetadataStorageActionHandler.java | 27 +++ .../metadata/MetadataStorageConnector.java | 2 + .../overlord/HeapMemoryTaskStorage.java | 9 + .../overlord/MetadataTaskStorage.java | 35 +++- .../druid/indexing/overlord/TaskStorage.java | 15 ++ .../overlord/TaskStorageQueryAdapter.java | 8 + .../overlord/http/OverlordResource.java | 170 ++++++++++++++- .../druid/metadata/SQLMetadataConnector.java | 114 ++++++++++ .../SQLMetadataStorageActionHandler.java | 198 ++++++++++++++++++ 9 files changed, 575 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index 292225a73ff9..83aa2dedf5f0 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -53,6 +53,17 @@ void insert( @Nullable StatusType status ) throws EntryExistsException; + void insertTask( + @NotNull String id, + @NotNull DateTime timestamp, + @NotNull String dataSource, + @NotNull EntryType entry, + @NotNull String type, + @NotNull String groupId, + boolean active, + @Nullable StatusType status + ) throws EntryExistsException; + /** * Sets or updates the status for any active entry with the given id. @@ -118,6 +129,22 @@ List> getTaskInfos( @Nullable String datasource ); + /** + * Returns a list of {@link TaskInfo} from metadata store that matches to the given filters. + * + * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. + * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata + * store. For complete tasks, additional filters in {@code CompleteTaskLookup} can be applied. + * All lookups should be processed atomically if there are more than one lookup is given. + * + * @param taskLookups task lookup type and filters. + * @param datasource datasource filter + */ + List, StatusType>> getTaskSummaryList( + Map taskLookups, + @Nullable String datasource + ); + default List> getTaskInfos( TaskLookup taskLookup, @Nullable String datasource diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java index 45fb6639082c..1990b02c2d0b 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -83,6 +83,8 @@ default void exportTable( void createTaskTables(); + void createNewTaskTables(); + void createAuditTable(); void createSupervisorsTable(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index 5365982bd3d8..4bbab708179d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -234,6 +234,15 @@ public List> getTaskInfos( return tasks; } + @Override + public List, TaskStatus>> getTaskSummaryList( + Map taskLookups, + @Nullable String datasource + ) + { + return Collections.emptyList(); + } + @Override public List getTaskStatusPlusList( Map taskLookups, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 590244926bf9..be20a3e91f63 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -115,7 +115,7 @@ public MetadataTaskStorage( @LifecycleStart public void start() { - metadataStorageConnector.createTaskTables(); + metadataStorageConnector.createNewTaskTables(); } @LifecycleStop @@ -138,12 +138,20 @@ public void insert(final Task task, final TaskStatus status) throws EntryExistsE log.info("Inserting task %s with status: %s", task.getId(), status); + String dummy = "dummy"; + while (dummy.length() < (int) 1e5) { + dummy += dummy; + } + task.getContext().put("dummy", dummy); + try { - handler.insert( + handler.insertTask( task.getId(), DateTimes.nowUtc(), task.getDataSource(), task, + task.getType(), + task.getGroupId(), status.isRunnable(), status ); @@ -268,6 +276,29 @@ public List> getTaskInfos( return Collections.unmodifiableList(handler.getTaskInfos(theTaskLookups, datasource)); } + @Override + public List, TaskStatus>> getTaskSummaryList( + Map taskLookups, + @Nullable String datasource + ) + { + Map theTaskLookups = Maps.newHashMapWithExpectedSize(taskLookups.size()); + for (Entry entry : taskLookups.entrySet()) { + if (entry.getKey() == TaskLookupType.COMPLETE) { + CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); + theTaskLookups.put( + entry.getKey(), + completeTaskLookup.hasTaskCreatedTimeFilter() + ? completeTaskLookup + : completeTaskLookup.withDurationBeforeNow(config.getRecentlyFinishedThreshold()) + ); + } else { + theTaskLookups.put(entry.getKey(), entry.getValue()); + } + } + return Collections.unmodifiableList(handler.getTaskSummaryList(theTaskLookups, datasource)); + } + @Override public void addLock(final String taskid, final TaskLock taskLock) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java index 7a71711ea103..cedfe02b53d2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java @@ -168,6 +168,21 @@ List getTaskStatusPlusList( Map runnerWorkItems ); + /** + * Returns a list of tasks summaries stored in the storage facility as {@link TaskInfo}. No + * particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. + * + * The returned list can contain active tasks and complete tasks depending on the {@code taskLookups} parameter. + * See {@link TaskLookup} for more details of active and complete tasks. + * + * @param taskLookups lookup types and filters + * @param datasource datasource filter + */ + List, TaskStatus>> getTaskSummaryList( + Map taskLookups, + @Nullable String datasource + ); + /** * Returns a list of tasks stored in the storage facility as {@link TaskInfo}. No * particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java index fa65d735208d..702f04f18783 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -95,6 +95,14 @@ public List> getCompletedTaskInfoByCreatedTimeDuratio return storage.getTaskInfos(CompleteTaskLookup.of(maxTaskStatuses, duration), dataSource); } + public List, TaskStatus>> getTaskSummaryList( + Map taskLookups, + @Nullable String dataSource + ) + { + return storage.getTaskSummaryList(taskLookups, dataSource); + } + public List> getTaskInfos( Map taskLookups, @Nullable String dataSource diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index 83d6f2ab458a..01437e387d8e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -690,7 +690,7 @@ public Response getTasks( taskMaster.getTaskRunner(), taskRunner -> { final List authorizedList = securedTaskStatusPlus( - getTaskStatusPlusList( + getTaskSummaryList( taskRunner, TaskStateLookup.fromString(state), dataSource, @@ -877,6 +877,125 @@ private List getTasks( return statuses; } + private List getTaskSummaryList( + TaskRunner taskRunner, + TaskStateLookup state, + @Nullable String dataSource, + @Nullable String createdTimeInterval, + @Nullable Integer maxCompletedTasks, + @Nullable String type + ) + { + final Duration createdTimeDuration; + if (createdTimeInterval != null) { + final Interval theInterval = Intervals.of(StringUtils.replace(createdTimeInterval, "_", "/")); + createdTimeDuration = theInterval.toDuration(); + } else { + createdTimeDuration = null; + } + + // Ideally, snapshotting in taskStorage and taskRunner should be done atomically, + // but there is no way to do it today. + // Instead, we first gets a snapshot from taskStorage and then one from taskRunner. + // This way, we can use the snapshot from taskStorage as the source of truth for the set of tasks to process + // and use the snapshot from taskRunner as a reference for potential task state updates happened + // after the first snapshotting. + Stream, TaskStatus>> taskInfoStreamFromTaskStorage = getTaskSummaryStreamFromTaskStorage( + state, + dataSource, + createdTimeDuration, + maxCompletedTasks, + type + ); + final Map runnerWorkItems = getTaskRunnerWorkItems( + taskRunner, + state, + dataSource, + type + ); + + if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING) { + // We are interested in only those tasks which are in taskRunner. + taskInfoStreamFromTaskStorage = taskInfoStreamFromTaskStorage + .filter(info -> runnerWorkItems.containsKey(info.getId())); + } + final List, TaskStatus>> taskInfoFromTaskStorage = taskInfoStreamFromTaskStorage + .collect(Collectors.toList()); + + // Separate complete and active tasks from taskStorage. + // Note that taskStorage can return only either complete tasks or active tasks per TaskLookupType. + final List, TaskStatus>> completeTaskInfoFromTaskStorage = new ArrayList<>(); + final List, TaskStatus>> activeTaskInfoFromTaskStorage = new ArrayList<>(); + for (TaskInfo, TaskStatus> info : taskInfoFromTaskStorage) { + if (info.getStatus().isComplete()) { + completeTaskInfoFromTaskStorage.add(info); + } else { + activeTaskInfoFromTaskStorage.add(info); + } + } + + final List statuses = new ArrayList<>(); + completeTaskInfoFromTaskStorage.forEach(taskInfo -> statuses.add( + new TaskStatusPlus( + taskInfo.getId(), + taskInfo.getTask() == null ? null : taskInfo.getTask().get("groupId"), + taskInfo.getTask() == null ? null : taskInfo.getTask().get("type"), + taskInfo.getCreatedTime(), + DateTimes.EPOCH, + taskInfo.getStatus().getStatusCode(), + RunnerTaskState.NONE, + taskInfo.getStatus().getDuration(), + taskInfo.getStatus().getLocation(), + taskInfo.getDataSource(), + taskInfo.getStatus().getErrorMsg() + ) + )); + + activeTaskInfoFromTaskStorage.forEach(taskInfo -> { + final TaskRunnerWorkItem runnerWorkItem = runnerWorkItems.get(taskInfo.getId()); + if (runnerWorkItem == null) { + // a task is assumed to be a waiting task if it exists in taskStorage but not in taskRunner. + if (state == TaskStateLookup.WAITING || state == TaskStateLookup.ALL) { + statuses.add( + new TaskStatusPlus( + taskInfo.getId(), + taskInfo.getTask() == null ? null : taskInfo.getTask().get("groupId"), + taskInfo.getTask() == null ? null : taskInfo.getTask().get("type"), + taskInfo.getCreatedTime(), + DateTimes.EPOCH, + taskInfo.getStatus().getStatusCode(), + RunnerTaskState.WAITING, + taskInfo.getStatus().getDuration(), + taskInfo.getStatus().getLocation(), + taskInfo.getDataSource(), + taskInfo.getStatus().getErrorMsg() + ) + ); + } + } else { + if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING || state == TaskStateLookup.ALL) { + statuses.add( + new TaskStatusPlus( + taskInfo.getId(), + taskInfo.getTask() == null ? null : taskInfo.getTask().get("groupId"), + taskInfo.getTask() == null ? null : taskInfo.getTask().get("type"), + runnerWorkItem.getCreatedTime(), + runnerWorkItem.getQueueInsertionTime(), + taskInfo.getStatus().getStatusCode(), + taskRunner.getRunnerTaskState(taskInfo.getId()), // this is racy for remoteTaskRunner + taskInfo.getStatus().getDuration(), + runnerWorkItem.getLocation(), // location in taskInfo is only updated after the task is done. + taskInfo.getDataSource(), + taskInfo.getStatus().getErrorMsg() + ) + ); + } + } + }); + + return statuses; + } + private Stream getTaskStatusPlusStreamFromStorage( TaskStateLookup state, @Nullable String dataSource, @@ -929,6 +1048,55 @@ private Stream getTaskStatusPlusStreamFromStorage( } } + private Stream, TaskStatus>> getTaskSummaryStreamFromTaskStorage( + TaskStateLookup state, + @Nullable String dataSource, + Duration createdTimeDuration, + @Nullable Integer maxCompletedTasks, + @Nullable String type + ) + { + final Map taskLookups; + switch (state) { + case ALL: + taskLookups = ImmutableMap.of( + TaskLookupType.ACTIVE, + ActiveTaskLookup.getInstance(), + TaskLookupType.COMPLETE, + CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) + ); + break; + case COMPLETE: + taskLookups = ImmutableMap.of( + TaskLookupType.COMPLETE, + CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) + ); + break; + case WAITING: + case PENDING: + case RUNNING: + taskLookups = ImmutableMap.of( + TaskLookupType.ACTIVE, + ActiveTaskLookup.getInstance() + ); + break; + default: + throw new IAE("Unknown state: [%s]", state); + } + + final Stream, TaskStatus>> taskInfoStreamFromTaskStorage = taskStorageQueryAdapter.getTaskSummaryList( + taskLookups, + dataSource + ).stream(); + if (type != null) { + return taskInfoStreamFromTaskStorage.filter( + info -> type.equals(info.getTask() == null ? null : info.getTask().get("type")) + ); + } else { + return taskInfoStreamFromTaskStorage; + } + } + private Stream> getTaskInfoStreamFromTaskStorage( TaskStateLookup state, @Nullable String dataSource, diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 9fe81dcdffbf..4888e64c966c 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -19,6 +19,8 @@ package org.apache.druid.metadata; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.base.Throwables; @@ -44,9 +46,12 @@ import javax.annotation.Nullable; import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; import java.sql.SQLException; import java.sql.SQLRecoverableException; import java.sql.SQLTransientException; +import java.sql.Statement; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -328,6 +333,103 @@ tableName, getPayloadType() ); } + public void createNewEntryTable(final String tableName) + { + try { + retryWithHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws SQLException + { + if (!tableExists(handle, tableName)) { + log.info("Creating table[%s]", tableName); + List sql = ImmutableList.of( + StringUtils.format( + "CREATE TABLE %1$s (\n" + + " id VARCHAR(255) NOT NULL,\n" + + " created_date VARCHAR(255) NOT NULL,\n" + + " datasource VARCHAR(255) %3$s NOT NULL,\n" + + " payload %2$s NOT NULL,\n" + + " status_payload %2$s NOT NULL,\n" + + " active BOOLEAN NOT NULL DEFAULT FALSE,\n" + + " PRIMARY KEY (id)\n" + + ")", + tableName, getPayloadType(), getCollation() + ), + StringUtils.format("CREATE INDEX idx_%1$s_active_created_date ON %1$s(active, created_date)", + tableName) + ); + final Batch batch = handle.createBatch(); + for (String s : sql) { + batch.add(s); + } + batch.execute(); + } else { + log.info("Table[%s] already exists", tableName); + } + + if (!tableContainsColumn(handle, tableName, "type")) { + Connection connection = handle.getConnection(); + + log.info("Table schema of [%s] is being altered", tableName); + + Statement statement = connection.createStatement(); + statement.execute(StringUtils.format("ALTER TABLE %1$s ADD COLUMN type VARCHAR(255)", tableName)); + statement.execute(StringUtils.format("ALTER TABLE %1$s ADD COLUMN group_id VARCHAR(255)", tableName)); + + connection.setAutoCommit(false); + try { + log.info("Populate fields task and group_id of [%s] from payload", tableName); + ObjectMapper objectMapper = new ObjectMapper(); + statement = connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_UPDATABLE); + String prevId = "''"; + while (prevId != null) { + String sql = StringUtils.format( + "SELECT * FROM %1$s WHERE id > %2$s ORDER BY id LIMIT 100", + tableName, + prevId + ); + ResultSet resultSet = statement.executeQuery(sql); + prevId = null; + while (resultSet.next()) { + ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); + resultSet.updateString("type", payload.get("type").asText()); + resultSet.updateString("group_id", payload.get("groupId").asText()); + prevId = "'" + resultSet.getString("id") + "'"; + resultSet.updateRow(); + } + } + connection.commit(); + } + catch (Exception e) { + connection.rollback(); + statement = connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_UPDATABLE); + statement.execute(StringUtils.format("ALTER TABLE %1$s DROP COLUMN type", tableName)); + statement.execute(StringUtils.format("ALTER TABLE %1$s DROP COLUMN group_id", tableName)); + throw new SQLException(e); + } + finally { + connection.close(); + } + } + return null; + } + } + ); + } + catch (Exception e) { + log.error(e, "Exception preparing table"); + } + } + + public boolean tableContainsColumn(Handle handle, String table, String column) throws SQLException + { + DatabaseMetaData databaseMetaData = handle.getConnection().getMetaData(); + ResultSet columns = databaseMetaData.getColumns(null, null, table.toUpperCase(), column.toUpperCase()); + return columns.next(); + } + public void createEntryTable(final String tableName) { createTable( @@ -584,6 +686,18 @@ public void createTaskTables() } } + @Override + public void createNewTaskTables() + { + if (config.get().isCreateTables()) { + final MetadataStorageTablesConfig tablesConfig = tablesConfigSupplier.get(); + final String entryType = tablesConfig.getTaskEntryType(); + createNewEntryTable(tablesConfig.getEntryTable(entryType)); + createLogTable(tablesConfig.getLogTable(entryType), entryType); + createLockTable(tablesConfig.getLockTable(entryType), entryType); + } + } + @Override public void createSupervisorsTable() { diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index ed79cb74250e..74c4e320a057 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.base.Optional; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import net.minidev.json.parser.JSONParser; import org.apache.druid.indexer.RunnerTaskState; @@ -79,6 +80,7 @@ public abstract class SQLMetadataStorageActionHandler taskInfoMapper; + private final TaskStatusMapper taskStatusMapper; @SuppressWarnings("PMD.UnnecessaryFullyQualifiedName") public SQLMetadataStorageActionHandler( @@ -106,6 +108,7 @@ public SQLMetadataStorageActionHandler( this.logTable = logTable; this.lockTable = lockTable; this.taskInfoMapper = new TaskInfoMapper<>(jsonMapper, entryType, statusType); + this.taskStatusMapper = new TaskStatusMapper<>(jsonMapper, statusType); } protected SQLMetadataConnector getConnector() @@ -143,6 +146,51 @@ public TypeReference getEntryType() return entryType; } + @Override + public void insertTask( + final String id, + final DateTime timestamp, + final String dataSource, + final EntryType entry, + final String type, + final String groupId, + final boolean active, + final StatusType status + ) throws EntryExistsException + { + try { + getConnector().retryWithHandle( + (HandleCallback) handle -> { + final String sql = StringUtils.format( + "INSERT INTO %s (id, created_date, datasource, payload, type, group_id, active, status_payload) " + + "VALUES (:id, :created_date, :datasource, :payload, :type, :group_id, :active, :status_payload)", + getEntryTable() + ); + handle.createStatement(sql) + .bind("id", id) + .bind("created_date", timestamp.toString()) + .bind("datasource", dataSource) + .bind("payload", jsonMapper.writeValueAsBytes(entry)) + .bind("type", type) + .bind("group_id", groupId) + .bind("active", active) + .bind("status_payload", jsonMapper.writeValueAsBytes(status)) + .execute(); + return null; + }, + e -> getConnector().isTransientException(e) && !(isStatementException(e) && getEntry(id).isPresent()) + ); + } + catch (Exception e) { + if (isStatementException(e) && getEntry(id).isPresent()) { + throw new EntryExistsException(id, e); + } else { + Throwables.propagateIfPossible(e); + throw new RuntimeException(e); + } + } + } + @Override public void insert( final String id, @@ -361,6 +409,86 @@ public List> getTaskInfos( ); } + @Override + public List, StatusType>> getTaskSummaryList( + Map taskLookups, + @Nullable String dataSource + ) + { + return getConnector().retryTransaction( + (handle, status) -> { + final List, StatusType>> tasks = new ArrayList<>(); + for (Entry entry : taskLookups.entrySet()) { + final Query> query; + switch (entry.getKey()) { + case ACTIVE: + query = createActiveTaskSummaryStreamingQuery( + handle, + dataSource + ); + tasks.addAll(query.map(taskStatusMapper).list()); + break; + case COMPLETE: + CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); + query = createCompletedTaskSummaryStreamingQuery( + handle, + completeTaskLookup.getTasksCreatedPriorTo(), + completeTaskLookup.getMaxTaskStatuses(), + dataSource + ); + tasks.addAll(query.map(taskStatusMapper).list()); + break; + default: + throw new IAE("Unknown TaskLookupType: [%s]", entry.getKey()); + } + } + return tasks; + }, + 3, + SQLMetadataConnector.DEFAULT_MAX_TRIES + ); + } + + + protected Query> createCompletedTaskSummaryStreamingQuery( + Handle handle, + DateTime timestamp, + @Nullable Integer maxNumStatuses, + @Nullable String dataSource + ) + { + String sql = StringUtils.format( + "SELECT " + + " id, " + + " created_date, " + + " datasource, " + + " group_id, " + + " type, " + + " status_payload " + + "FROM " + + " %s " + + "WHERE " + + getWhereClauseForInactiveStatusesSinceQuery(dataSource) + + "ORDER BY created_date DESC", + getEntryTable() + ); + + if (maxNumStatuses != null) { + sql = decorateSqlWithLimit(sql); + } + Query> query = handle.createQuery(sql) + .bind("start", timestamp.toString()) + .setFetchSize(connector.getStreamingFetchSize()); + + if (maxNumStatuses != null) { + query = query.bind("n", maxNumStatuses); + } + if (dataSource != null) { + query = query.bind("ds", dataSource); + } + return query; + } + protected Query> createCompletedTaskInfoStreamingQuery( Handle handle, DateTime timestamp, @@ -410,6 +538,32 @@ private String getWhereClauseForInactiveStatusesSinceQuery(@Nullable String data return sql; } + private Query> createActiveTaskSummaryStreamingQuery(Handle handle, @Nullable String dataSource) + { + String sql = StringUtils.format( + "SELECT " + + " id, " + + " status_payload, " + + " group_id, " + + " type, " + + " datasource, " + + " created_date " + + "FROM " + + " %s " + + "WHERE " + + getWhereClauseForActiveStatusesQuery(dataSource) + + "ORDER BY created_date", + entryTable + ); + + Query> query = handle.createQuery(sql) + .setFetchSize(connector.getStreamingFetchSize()); + if (dataSource != null) { + query = query.bind("ds", dataSource); + } + return query; + } + private Query> createActiveTaskInfoStreamingQuery(Handle handle, @Nullable String dataSource) { String sql = StringUtils.format( @@ -591,6 +745,50 @@ public TaskInfo map(int index, ResultSet resultSet, State } } + static class TaskStatusMapper implements ResultSetMapper, StatusType>> + { + private final ObjectMapper objectMapper; + private final TypeReference statusType; + + TaskStatusMapper(ObjectMapper objectMapper, TypeReference statusType) + { + this.objectMapper = objectMapper; + this.statusType = statusType; + } + + @Override + public TaskInfo, StatusType> map(int index, ResultSet resultSet, StatementContext context) + throws SQLException + { + final TaskInfo, StatusType> taskInfo; + StatusType status; + try { + status = objectMapper.readValue(resultSet.getBytes("status_payload"), statusType); + } + catch (IOException e) { + log.error(e, "Encountered exception while deserializing task status_payload"); + throw new SQLException(e); + } + ImmutableMap.Builder task = ImmutableMap.builder(); + String groupId = resultSet.getString("group_id"); + if (groupId != null) { + task.put("groupId", groupId); + } + String type = resultSet.getString("type"); + if (type != null) { + task.put("type", type); + } + taskInfo = new TaskInfo<>( + resultSet.getString("id"), + DateTimes.of(resultSet.getString("created_date")), + status, + resultSet.getString("datasource"), + task.build() + ); + return taskInfo; + } + } + @Override public boolean addLock(final String entryId, final LockType lock) { From bf8e8c56be8f3dac9d3a2732f6ee0deb4ee54b49 Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 13 Apr 2022 20:43:35 +0530 Subject: [PATCH 03/16] Migration in thread, determining strategy for sql fetch --- .../MetadataStorageActionHandler.java | 47 +-- .../MetadataStorageActionHandlerFactory.java | 2 +- .../metadata/MetadataStorageConnector.java | 4 +- .../storage/sqlserver/SQLServerConnector.java | 6 + .../storage/mysql/MySQLConnector.java | 6 + .../postgresql/PostgreSQLConnector.java | 6 + .../overlord/HeapMemoryTaskStorage.java | 11 - .../overlord/MetadataTaskStorage.java | 77 +++-- .../druid/indexing/overlord/TaskStorage.java | 18 -- .../overlord/TaskStorageQueryAdapter.java | 10 - .../overlord/http/OverlordResource.java | 272 ----------------- .../DerbyMetadataStorageActionHandler.java | 4 +- ...byMetadataStorageActionHandlerFactory.java | 2 +- .../MySQLMetadataStorageActionHandler.java | 4 +- ...QLMetadataStorageActionHandlerFactory.java | 4 +- ...ostgreSQLMetadataStorageActionHandler.java | 4 +- ...QLMetadataStorageActionHandlerFactory.java | 4 +- .../druid/metadata/SQLMetadataConnector.java | 193 ++++++------ .../SQLMetadataStorageActionHandler.java | 276 ++++++------------ ...SQLServerMetadataStorageActionHandler.java | 4 +- ...erMetadataStorageActionHandlerFactory.java | 4 +- .../storage/derby/DerbyConnector.java | 6 + .../metadata/SQLMetadataConnectorTest.java | 6 + .../SQLMetadataStorageActionHandlerTest.java | 28 +- 24 files changed, 291 insertions(+), 707 deletions(-) diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index 83aa2dedf5f0..e1dd9e9f457d 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -21,7 +21,6 @@ import com.google.common.base.Optional; import org.apache.druid.indexer.TaskInfo; -import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.metadata.TaskLookup.TaskLookupType; import org.joda.time.DateTime; @@ -31,7 +30,7 @@ import java.util.List; import java.util.Map; -public interface MetadataStorageActionHandler +public interface MetadataStorageActionHandler { /** * Creates a new entry. @@ -42,6 +41,8 @@ public interface MetadataStorageActionHandler getTaskInfo(String entryId); - /** - * Returns a list of {@link TaskStatusPlus} from metadata store matching the given filters. - * - * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. - * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata - * store. For complete tasks, additional filters in {@code CompleteTaskLookup} can be applied. - * All lookups should be processed atomically if there are more than one lookup is given. - * - * @param taskLookups task lookup type and filters. - * @param datasource datasource filter - * @param runnerWorkItems map of id to work items for running tasks - */ - List getTaskStatusPlusList( - Map taskLookups, - @Nullable String datasource, - Map runnerWorkItems - ); - /** * Returns a list of {@link TaskInfo} from metadata store that matches to the given filters. * @@ -130,19 +103,25 @@ List> getTaskInfos( ); /** - * Returns a list of {@link TaskInfo} from metadata store that matches to the given filters. + * Returns a list of {@link TaskInfo} from metadata store that matches the given filters. * * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata * store. For complete tasks, additional filters in {@code CompleteTaskLookup} can be applied. * All lookups should be processed atomically if there are more than one lookup is given. * + * taskMigrationComplete indicates if migration was completed before the time of calling. + * If yes, use the newly created type and group_id columns in the query for task summaries + * Else, fetch the payload and deserialize it to obtain the above fields + * * @param taskLookups task lookup type and filters. * @param datasource datasource filter + * @param taskMigrationComplete indicates which query to use based on migration status */ List, StatusType>> getTaskSummaryList( Map taskLookups, - @Nullable String datasource + @Nullable String datasource, + boolean taskMigrationComplete ); default List> getTaskInfos( diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerFactory.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerFactory.java index 4770c0dc885c..a81ecfdbf1c4 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerFactory.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerFactory.java @@ -21,7 +21,7 @@ public interface MetadataStorageActionHandlerFactory { - MetadataStorageActionHandler create( + MetadataStorageActionHandler create( String entryType, MetadataStorageActionHandlerTypes payloadTypes ); diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java index 1990b02c2d0b..263533595470 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -83,11 +83,11 @@ default void exportTable( void createTaskTables(); - void createNewTaskTables(); - void createAuditTable(); void createSupervisorsTable(); + boolean migrateTaskTable() throws Exception; + void deleteAllRecords(String tableName); } diff --git a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java index d44e3489ae82..7d72dfd34efc 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java @@ -212,6 +212,12 @@ public boolean tableExists(final Handle handle, final String tableName) .isEmpty(); } + @Override + public String withFetchLimit(int limit) + { + return String.format("FETCH NEXT %d ROWS ONLY", limit); + } + /** * * {@inheritDoc} diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java index d4eb57bb8189..5e80cbb09ac3 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java @@ -177,6 +177,12 @@ public int getStreamingFetchSize() return Integer.MIN_VALUE; } + @Override + public String withFetchLimit(int limit) + { + return String.format("LIMIT %d", limit); + } + @Override public boolean tableExists(Handle handle, String tableName) { diff --git a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java index 95542cf34026..ffb4c21d156f 100644 --- a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java +++ b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java @@ -144,6 +144,12 @@ public int getStreamingFetchSize() return DEFAULT_STREAMING_RESULT_SIZE; } + @Override + public String withFetchLimit(int limit) + { + return String.format("LIMIT %d", limit); + } + protected boolean canUpsert(Handle handle) throws SQLException { if (canUpsert == null) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index 4bbab708179d..c738a18d1807 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -30,7 +30,6 @@ import com.google.inject.Inject; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.config.TaskStorageConfig; @@ -243,16 +242,6 @@ public List, TaskStatus>> getTaskSummaryList( return Collections.emptyList(); } - @Override - public List getTaskStatusPlusList( - Map taskLookups, - @Nullable String datasource, - Map runnerWorkItems - ) - { - return Collections.emptyList(); - } - private List> getRecentlyCreatedAlreadyFinishedTaskInfoSince( DateTime start, @Nullable Integer n, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index be20a3e91f63..196123a7434d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -29,7 +29,6 @@ import com.google.inject.Inject; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.config.TaskStorageConfig; @@ -55,10 +54,16 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.stream.Collectors; public class MetadataTaskStorage implements TaskStorage { + + private Future taskMigrationCompleteFuture; + private static final MetadataStorageActionHandlerTypes TASK_TYPES = new MetadataStorageActionHandlerTypes() { @Override @@ -96,7 +101,7 @@ public TypeReference getLockType() private final MetadataStorageConnector metadataStorageConnector; private final TaskStorageConfig config; - private final MetadataStorageActionHandler handler; + private final MetadataStorageActionHandler handler; private static final EmittingLogger log = new EmittingLogger(MetadataTaskStorage.class); @@ -115,7 +120,11 @@ public MetadataTaskStorage( @LifecycleStart public void start() { - metadataStorageConnector.createNewTaskTables(); + metadataStorageConnector.createTaskTables(); + ExecutorService executorService = Executors.newSingleThreadExecutor(); + taskMigrationCompleteFuture = executorService.submit(() -> { + return metadataStorageConnector.migrateTaskTable(); + }); } @LifecycleStop @@ -145,15 +154,15 @@ public void insert(final Task task, final TaskStatus status) throws EntryExistsE task.getContext().put("dummy", dummy); try { - handler.insertTask( + handler.insert( task.getId(), DateTimes.nowUtc(), task.getDataSource(), task, - task.getType(), - task.getGroupId(), status.isRunnable(), - status + status, + task.getType(), + task.getGroupId() ); } catch (Exception e) { @@ -230,56 +239,36 @@ public List getActiveTasksByDatasource(String datasource) } @Override - public List getTaskStatusPlusList( + public List> getTaskInfos( Map taskLookups, - @Nullable String datasource, - Map runnerWorkItems + @Nullable String datasource ) { - Map theTaskLookups = Maps.newHashMapWithExpectedSize(taskLookups.size()); - for (Entry entry : taskLookups.entrySet()) { - if (entry.getKey() == TaskLookupType.COMPLETE) { - CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); - theTaskLookups.put( - entry.getKey(), - completeTaskLookup.hasTaskCreatedTimeFilter() - ? completeTaskLookup - : completeTaskLookup.withDurationBeforeNow(config.getRecentlyFinishedThreshold()) - ); - } else { - theTaskLookups.put(entry.getKey(), entry.getValue()); - } - } - return Collections.unmodifiableList(handler.getTaskStatusPlusList(theTaskLookups, datasource, runnerWorkItems)); + Map theTaskLookups = processTaskLookups(taskLookups); + return Collections.unmodifiableList(handler.getTaskInfos(theTaskLookups, datasource)); } @Override - public List> getTaskInfos( + public List, TaskStatus>> getTaskSummaryList( Map taskLookups, @Nullable String datasource ) { - Map theTaskLookups = Maps.newHashMapWithExpectedSize(taskLookups.size()); - for (Entry entry : taskLookups.entrySet()) { - if (entry.getKey() == TaskLookupType.COMPLETE) { - CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); - theTaskLookups.put( - entry.getKey(), - completeTaskLookup.hasTaskCreatedTimeFilter() - ? completeTaskLookup - : completeTaskLookup.withDurationBeforeNow(config.getRecentlyFinishedThreshold()) - ); - } else { - theTaskLookups.put(entry.getKey(), entry.getValue()); + Map theTaskLookups = processTaskLookups(taskLookups); + boolean taskMigrationComplete = false; + if (taskMigrationCompleteFuture.isDone()) { + try { + taskMigrationComplete = taskMigrationCompleteFuture.get(); + } + catch (Exception e) { + log.info(e, "Exception getting task migration future"); } } - return Collections.unmodifiableList(handler.getTaskInfos(theTaskLookups, datasource)); + return Collections.unmodifiableList(handler.getTaskSummaryList(theTaskLookups, datasource, taskMigrationComplete)); } - @Override - public List, TaskStatus>> getTaskSummaryList( - Map taskLookups, - @Nullable String datasource + private Map processTaskLookups( + Map taskLookups ) { Map theTaskLookups = Maps.newHashMapWithExpectedSize(taskLookups.size()); @@ -296,7 +285,7 @@ public List, TaskStatus>> getTaskSummaryList( theTaskLookups.put(entry.getKey(), entry.getValue()); } } - return Collections.unmodifiableList(handler.getTaskSummaryList(theTaskLookups, datasource)); + return theTaskLookups; } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java index cedfe02b53d2..8b551262f47d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java @@ -22,7 +22,6 @@ import com.google.common.base.Optional; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.task.Task; @@ -151,23 +150,6 @@ public interface TaskStorage */ List getActiveTasksByDatasource(String datasource); - /** - * Returns a list of tasks' relevant status info stored in the storage facility as {@link TaskStatusPlus}. No - * particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. - * - * The returned list can contain active tasks and complete tasks depending on the {@code taskLookups} parameter. - * See {@link TaskLookup} for more details of active and complete tasks. - * - * @param taskLookups lookup types and filters - * @param datasource datasource filter - * @param runnerWorkItems - */ - List getTaskStatusPlusList( - Map taskLookups, - @Nullable String datasource, - Map runnerWorkItems - ); - /** * Returns a list of tasks summaries stored in the storage facility as {@link TaskInfo}. No * particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java index 702f04f18783..7160a7a977ba 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -23,7 +23,6 @@ import com.google.inject.Inject; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.actions.SegmentInsertAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskAction; @@ -111,15 +110,6 @@ public List> getTaskInfos( return storage.getTaskInfos(taskLookups, dataSource); } - public List getTaskStatusPlusList( - Map taskLookups, - @Nullable String datasource, - Map runnerWorkItems - ) - { - return storage.getTaskStatusPlusList(taskLookups, datasource, runnerWorkItems); - } - public Optional getTask(final String taskid) { return storage.getTask(taskid); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index 01437e387d8e..f8225db42c12 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -706,177 +706,6 @@ public Response getTasks( ); } - private List getTaskStatusPlusList( - TaskRunner taskRunner, - TaskStateLookup state, - @Nullable String dataSource, - @Nullable String createdTimeInterval, - @Nullable Integer maxCompletedTasks, - @Nullable String type - ) - { - final Duration createdTimeDuration; - if (createdTimeInterval != null) { - final Interval theInterval = Intervals.of(StringUtils.replace(createdTimeInterval, "_", "/")); - createdTimeDuration = theInterval.toDuration(); - } else { - createdTimeDuration = null; - } - - // This way, we can use the snapshot from taskStorage as the source of truth for the set of tasks to process - // and use the snapshot from taskRunner as a reference for potential task state updates happened - // after the first snapshotting. - final Map runnerWorkItems = getTaskRunnerWorkItems( - taskRunner, - state, - dataSource, - type - ); - Stream taskStatusPlusStream = getTaskStatusPlusStreamFromStorage( - state, - dataSource, - createdTimeDuration, - maxCompletedTasks, - type, - runnerWorkItems - ); - - if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING) { - // We are interested in only those tasks which are in taskRunner. - taskStatusPlusStream = taskStatusPlusStream - .filter(taskStatusPlus -> runnerWorkItems.containsKey(taskStatusPlus.getId())); - } - final List taskStatusPlusList = taskStatusPlusStream.collect(Collectors.toList()); - for (TaskStatusPlus taskStatusPlus : taskStatusPlusList) { - if (taskStatusPlus.getRunnerStatusCode() == null) { - // this is racy for remoteTaskRunner - RunnerTaskState runnerTaskState = taskRunner.getRunnerTaskState(taskStatusPlus.getId()); - taskStatusPlus.setRunnerTaskState(runnerTaskState); - } - } - - return taskStatusPlusList; - } - - private List getTasks( - TaskRunner taskRunner, - TaskStateLookup state, - @Nullable String dataSource, - @Nullable String createdTimeInterval, - @Nullable Integer maxCompletedTasks, - @Nullable String type - ) - { - final Duration createdTimeDuration; - if (createdTimeInterval != null) { - final Interval theInterval = Intervals.of(StringUtils.replace(createdTimeInterval, "_", "/")); - createdTimeDuration = theInterval.toDuration(); - } else { - createdTimeDuration = null; - } - - // Ideally, snapshotting in taskStorage and taskRunner should be done atomically, - // but there is no way to do it today. - // Instead, we first gets a snapshot from taskStorage and then one from taskRunner. - // This way, we can use the snapshot from taskStorage as the source of truth for the set of tasks to process - // and use the snapshot from taskRunner as a reference for potential task state updates happened - // after the first snapshotting. - Stream> taskInfoStreamFromTaskStorage = getTaskInfoStreamFromTaskStorage( - state, - dataSource, - createdTimeDuration, - maxCompletedTasks, - type - ); - final Map runnerWorkItems = getTaskRunnerWorkItems( - taskRunner, - state, - dataSource, - type - ); - - if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING) { - // We are interested in only those tasks which are in taskRunner. - taskInfoStreamFromTaskStorage = taskInfoStreamFromTaskStorage - .filter(info -> runnerWorkItems.containsKey(info.getId())); - } - final List> taskInfoFromTaskStorage = taskInfoStreamFromTaskStorage - .collect(Collectors.toList()); - - // Separate complete and active tasks from taskStorage. - // Note that taskStorage can return only either complete tasks or active tasks per TaskLookupType. - final List> completeTaskInfoFromTaskStorage = new ArrayList<>(); - final List> activeTaskInfoFromTaskStorage = new ArrayList<>(); - for (TaskInfo info : taskInfoFromTaskStorage) { - if (info.getStatus().isComplete()) { - completeTaskInfoFromTaskStorage.add(info); - } else { - activeTaskInfoFromTaskStorage.add(info); - } - } - - final List statuses = new ArrayList<>(); - completeTaskInfoFromTaskStorage.forEach(taskInfo -> statuses.add( - new TaskStatusPlus( - taskInfo.getId(), - taskInfo.getTask() == null ? null : taskInfo.getTask().getGroupId(), - taskInfo.getTask() == null ? null : taskInfo.getTask().getType(), - taskInfo.getCreatedTime(), - DateTimes.EPOCH, - taskInfo.getStatus().getStatusCode(), - RunnerTaskState.NONE, - taskInfo.getStatus().getDuration(), - taskInfo.getStatus().getLocation(), - taskInfo.getDataSource(), - taskInfo.getStatus().getErrorMsg() - ) - )); - - activeTaskInfoFromTaskStorage.forEach(taskInfo -> { - final TaskRunnerWorkItem runnerWorkItem = runnerWorkItems.get(taskInfo.getId()); - if (runnerWorkItem == null) { - // a task is assumed to be a waiting task if it exists in taskStorage but not in taskRunner. - if (state == TaskStateLookup.WAITING || state == TaskStateLookup.ALL) { - statuses.add( - new TaskStatusPlus( - taskInfo.getId(), - taskInfo.getTask() == null ? null : taskInfo.getTask().getGroupId(), - taskInfo.getTask() == null ? null : taskInfo.getTask().getType(), - taskInfo.getCreatedTime(), - DateTimes.EPOCH, - taskInfo.getStatus().getStatusCode(), - RunnerTaskState.WAITING, - taskInfo.getStatus().getDuration(), - taskInfo.getStatus().getLocation(), - taskInfo.getDataSource(), - taskInfo.getStatus().getErrorMsg() - ) - ); - } - } else { - if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING || state == TaskStateLookup.ALL) { - statuses.add( - new TaskStatusPlus( - taskInfo.getId(), - taskInfo.getTask() == null ? null : taskInfo.getTask().getGroupId(), - taskInfo.getTask() == null ? null : taskInfo.getTask().getType(), - runnerWorkItem.getCreatedTime(), - runnerWorkItem.getQueueInsertionTime(), - taskInfo.getStatus().getStatusCode(), - taskRunner.getRunnerTaskState(taskInfo.getId()), // this is racy for remoteTaskRunner - taskInfo.getStatus().getDuration(), - runnerWorkItem.getLocation(), // location in taskInfo is only updated after the task is done. - taskInfo.getDataSource(), - taskInfo.getStatus().getErrorMsg() - ) - ); - } - } - }); - - return statuses; - } - private List getTaskSummaryList( TaskRunner taskRunner, TaskStateLookup state, @@ -996,58 +825,6 @@ private List getTaskSummaryList( return statuses; } - private Stream getTaskStatusPlusStreamFromStorage( - TaskStateLookup state, - @Nullable String dataSource, - Duration createdTimeDuration, - @Nullable Integer maxCompletedTasks, - @Nullable String type, - Map runnerWorkItems - ) - { - final Map taskLookups; - switch (state) { - case ALL: - taskLookups = ImmutableMap.of( - TaskLookupType.ACTIVE, - ActiveTaskLookup.getInstance(), - TaskLookupType.COMPLETE, - CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) - ); - break; - case COMPLETE: - taskLookups = ImmutableMap.of( - TaskLookupType.COMPLETE, - CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) - ); - break; - case WAITING: - case PENDING: - case RUNNING: - taskLookups = ImmutableMap.of( - TaskLookupType.ACTIVE, - ActiveTaskLookup.getInstance() - ); - break; - default: - throw new IAE("Unknown state: [%s]", state); - } - - final Stream taskStatusPlusStream = - taskStorageQueryAdapter.getTaskStatusPlusList( - taskLookups, - dataSource, - runnerWorkItems - ).stream(); - if (type != null) { - return taskStatusPlusStream.filter( - taskStatusPlus -> type.equals(taskStatusPlus == null ? null : taskStatusPlus.getType()) - ); - } else { - return taskStatusPlusStream; - } - } - private Stream, TaskStatus>> getTaskSummaryStreamFromTaskStorage( TaskStateLookup state, @Nullable String dataSource, @@ -1097,55 +874,6 @@ private Stream, TaskStatus>> getTaskSummaryStreamFr } } - private Stream> getTaskInfoStreamFromTaskStorage( - TaskStateLookup state, - @Nullable String dataSource, - Duration createdTimeDuration, - @Nullable Integer maxCompletedTasks, - @Nullable String type - ) - { - final Map taskLookups; - switch (state) { - case ALL: - taskLookups = ImmutableMap.of( - TaskLookupType.ACTIVE, - ActiveTaskLookup.getInstance(), - TaskLookupType.COMPLETE, - CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) - ); - break; - case COMPLETE: - taskLookups = ImmutableMap.of( - TaskLookupType.COMPLETE, - CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) - ); - break; - case WAITING: - case PENDING: - case RUNNING: - taskLookups = ImmutableMap.of( - TaskLookupType.ACTIVE, - ActiveTaskLookup.getInstance() - ); - break; - default: - throw new IAE("Unknown state: [%s]", state); - } - - final Stream> taskInfoStreamFromTaskStorage = taskStorageQueryAdapter.getTaskInfos( - taskLookups, - dataSource - ).stream(); - if (type != null) { - return taskInfoStreamFromTaskStorage.filter( - info -> type.equals(info.getTask() == null ? null : info.getTask().getType()) - ); - } else { - return taskInfoStreamFromTaskStorage; - } - } - private Map getTaskRunnerWorkItems( TaskRunner taskRunner, TaskStateLookup state, diff --git a/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java index 3fa24979e9c6..fa39d506b4cc 100644 --- a/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java @@ -23,8 +23,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.java.util.common.StringUtils; -public class DerbyMetadataStorageActionHandler - extends SQLMetadataStorageActionHandler +public class DerbyMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler { @VisibleForTesting DerbyMetadataStorageActionHandler( diff --git a/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java index 3a102d53744e..cf6dc44ed93b 100644 --- a/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java +++ b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandlerFactory.java @@ -35,7 +35,7 @@ public DerbyMetadataStorageActionHandlerFactory( } @Override - public MetadataStorageActionHandler create( + public MetadataStorageActionHandler create( final String entryType, MetadataStorageActionHandlerTypes payloadTypes ) diff --git a/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandler.java index 70f8e956f515..79c028202f41 100644 --- a/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandler.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; -public class MySQLMetadataStorageActionHandler - extends SQLMetadataStorageActionHandler +public class MySQLMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler { MySQLMetadataStorageActionHandler( SQLMetadataConnector connector, diff --git a/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java b/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java index a5291710a5fb..0af63ba1d631 100644 --- a/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java +++ b/server/src/main/java/org/apache/druid/metadata/MySQLMetadataStorageActionHandlerFactory.java @@ -35,8 +35,8 @@ public MySQLMetadataStorageActionHandlerFactory( } @Override - public - MetadataStorageActionHandler create( + public + MetadataStorageActionHandler create( String entryType, MetadataStorageActionHandlerTypes payloadTypes ) diff --git a/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java index e9c06ca11420..b91427bcd662 100644 --- a/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.java.util.common.StringUtils; -public class PostgreSQLMetadataStorageActionHandler - extends SQLMetadataStorageActionHandler +public class PostgreSQLMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler { public PostgreSQLMetadataStorageActionHandler( SQLMetadataConnector connector, diff --git a/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java b/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java index d55c86ce4cfc..329b7a52474b 100644 --- a/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java +++ b/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandlerFactory.java @@ -35,8 +35,8 @@ public PostgreSQLMetadataStorageActionHandlerFactory( } @Override - public - MetadataStorageActionHandler create( + public + MetadataStorageActionHandler create( String entryType, MetadataStorageActionHandlerTypes payloadTypes ) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 4888e64c966c..f51ea04b90aa 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -45,6 +45,7 @@ import org.skife.jdbi.v2.util.IntegerMapper; import javax.annotation.Nullable; +import java.io.IOException; import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.ResultSet; @@ -55,6 +56,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.Properties; public abstract class SQLMetadataConnector implements MetadataStorageConnector @@ -134,6 +136,8 @@ public String getValidationQuery() public abstract boolean tableExists(Handle handle, String tableName); + public abstract String withFetchLimit(int limit); + public T retryWithHandle( final HandleCallback callback, final Predicate myShouldRetry @@ -332,103 +336,23 @@ tableName, getPayloadType() ) ); } - - public void createNewEntryTable(final String tableName) - { - try { - retryWithHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) throws SQLException - { - if (!tableExists(handle, tableName)) { - log.info("Creating table[%s]", tableName); - List sql = ImmutableList.of( - StringUtils.format( - "CREATE TABLE %1$s (\n" - + " id VARCHAR(255) NOT NULL,\n" - + " created_date VARCHAR(255) NOT NULL,\n" - + " datasource VARCHAR(255) %3$s NOT NULL,\n" - + " payload %2$s NOT NULL,\n" - + " status_payload %2$s NOT NULL,\n" - + " active BOOLEAN NOT NULL DEFAULT FALSE,\n" - + " PRIMARY KEY (id)\n" - + ")", - tableName, getPayloadType(), getCollation() - ), - StringUtils.format("CREATE INDEX idx_%1$s_active_created_date ON %1$s(active, created_date)", - tableName) - ); - final Batch batch = handle.createBatch(); - for (String s : sql) { - batch.add(s); - } - batch.execute(); - } else { - log.info("Table[%s] already exists", tableName); - } - - if (!tableContainsColumn(handle, tableName, "type")) { - Connection connection = handle.getConnection(); - - log.info("Table schema of [%s] is being altered", tableName); - - Statement statement = connection.createStatement(); - statement.execute(StringUtils.format("ALTER TABLE %1$s ADD COLUMN type VARCHAR(255)", tableName)); - statement.execute(StringUtils.format("ALTER TABLE %1$s ADD COLUMN group_id VARCHAR(255)", tableName)); - - connection.setAutoCommit(false); - try { - log.info("Populate fields task and group_id of [%s] from payload", tableName); - ObjectMapper objectMapper = new ObjectMapper(); - statement = connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_UPDATABLE); - String prevId = "''"; - while (prevId != null) { - String sql = StringUtils.format( - "SELECT * FROM %1$s WHERE id > %2$s ORDER BY id LIMIT 100", - tableName, - prevId - ); - ResultSet resultSet = statement.executeQuery(sql); - prevId = null; - while (resultSet.next()) { - ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); - resultSet.updateString("type", payload.get("type").asText()); - resultSet.updateString("group_id", payload.get("groupId").asText()); - prevId = "'" + resultSet.getString("id") + "'"; - resultSet.updateRow(); - } - } - connection.commit(); - } - catch (Exception e) { - connection.rollback(); - statement = connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_UPDATABLE); - statement.execute(StringUtils.format("ALTER TABLE %1$s DROP COLUMN type", tableName)); - statement.execute(StringUtils.format("ALTER TABLE %1$s DROP COLUMN group_id", tableName)); - throw new SQLException(e); - } - finally { - connection.close(); - } - } - return null; - } - } - ); - } - catch (Exception e) { - log.error(e, "Exception preparing table"); - } - } - + public boolean tableContainsColumn(Handle handle, String table, String column) throws SQLException { DatabaseMetaData databaseMetaData = handle.getConnection().getMetaData(); - ResultSet columns = databaseMetaData.getColumns(null, null, table.toUpperCase(), column.toUpperCase()); + ResultSet columns = databaseMetaData.getColumns(null, + null, + table.toUpperCase(Locale.ENGLISH), + column.toUpperCase(Locale.ENGLISH) + ); return columns.next(); } + + public void prepareEntryTable(final String tableName) + { + createEntryTable(tableName); + alterEntryTable(tableName); + } public void createEntryTable(final String tableName) { @@ -452,6 +376,77 @@ tableName, getPayloadType(), getCollation() ); } + public void alterEntryTable(final String tableName) + { + try { + retryWithHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) + { + log.info("Altering table[%s]", tableName); + final Batch batch = handle.createBatch(); + batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN type VARCHAR(255)", tableName)); + batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN group_id VARCHAR(255)", tableName)); + batch.execute(); + return null; + } + } + ); + } + catch (Exception e) { + log.warn(e, "Exception altering table"); + } + } + + @Override + public boolean migrateTaskTable() + { + final MetadataStorageTablesConfig tablesConfig = tablesConfigSupplier.get(); + final String entryType = tablesConfig.getTaskEntryType(); + final String tableName = tablesConfig.getEntryTable(entryType); + log.info("Populate fields task and group_id of task entry table [%s] from payload", tableName); + try { + retryWithHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws SQLException, IOException + { + ObjectMapper objectMapper = new ObjectMapper(); + Connection connection = handle.getConnection(); + Statement statement = connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_UPDATABLE); + String prevId = "''"; + while (prevId != null) { + String sql = StringUtils.format( + "SELECT * FROM %1$s WHERE id > %2$s AND active = false AND type IS null ORDER BY id ASC %3$s", + tableName, + prevId, + withFetchLimit(100) + ); + ResultSet resultSet = statement.executeQuery(sql); + prevId = null; + while (resultSet.next()) { + ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); + resultSet.updateString("type", payload.get("type").asText()); + resultSet.updateString("group_id", payload.get("groupId").asText()); + prevId = "'" + resultSet.getString("id") + "'"; + resultSet.updateRow(); + } + } + return null; + } + } + ); + return true; + } + catch (Exception e) { + log.warn(e, "Exception migrating task table [%s]", tableName); + return false; + } + } + public void createLogTable(final String tableName, final String entryTypeName) { createTable( @@ -680,19 +675,7 @@ public void createTaskTables() if (config.get().isCreateTables()) { final MetadataStorageTablesConfig tablesConfig = tablesConfigSupplier.get(); final String entryType = tablesConfig.getTaskEntryType(); - createEntryTable(tablesConfig.getEntryTable(entryType)); - createLogTable(tablesConfig.getLogTable(entryType), entryType); - createLockTable(tablesConfig.getLockTable(entryType), entryType); - } - } - - @Override - public void createNewTaskTables() - { - if (config.get().isCreateTables()) { - final MetadataStorageTablesConfig tablesConfig = tablesConfigSupplier.get(); - final String entryType = tablesConfig.getTaskEntryType(); - createNewEntryTable(tablesConfig.getEntryTable(entryType)); + prepareEntryTable(tablesConfig.getEntryTable(entryType)); createLogTable(tablesConfig.getLogTable(entryType), entryType); createLockTable(tablesConfig.getLockTable(entryType), entryType); } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index 74c4e320a057..366a0fd3398b 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -28,11 +28,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import net.minidev.json.parser.JSONParser; -import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskInfo; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; @@ -61,8 +57,8 @@ import java.util.Map; import java.util.Map.Entry; -public abstract class SQLMetadataStorageActionHandler - implements MetadataStorageActionHandler +public abstract class SQLMetadataStorageActionHandler + implements MetadataStorageActionHandler { private static final EmittingLogger log = new EmittingLogger(SQLMetadataStorageActionHandler.class); @@ -80,7 +76,8 @@ public abstract class SQLMetadataStorageActionHandler taskInfoMapper; - private final TaskStatusMapper taskStatusMapper; + private final TaskSummaryMapper taskStatusMapper; + private final TaskPayloadMapper taskPayloadMapper; @SuppressWarnings("PMD.UnnecessaryFullyQualifiedName") public SQLMetadataStorageActionHandler( @@ -108,7 +105,8 @@ public SQLMetadataStorageActionHandler( this.logTable = logTable; this.lockTable = lockTable; this.taskInfoMapper = new TaskInfoMapper<>(jsonMapper, entryType, statusType); - this.taskStatusMapper = new TaskStatusMapper<>(jsonMapper, statusType); + this.taskStatusMapper = new TaskSummaryMapper<>(jsonMapper, statusType); + this.taskPayloadMapper = new TaskPayloadMapper<>(jsonMapper, statusType); } protected SQLMetadataConnector getConnector() @@ -147,15 +145,15 @@ public TypeReference getEntryType() } @Override - public void insertTask( + public void insert( final String id, final DateTime timestamp, final String dataSource, final EntryType entry, - final String type, - final String groupId, final boolean active, - final StatusType status + final StatusType status, + final String type, + final String groupId ) throws EntryExistsException { try { @@ -191,47 +189,6 @@ public void insertTask( } } - @Override - public void insert( - final String id, - final DateTime timestamp, - final String dataSource, - final EntryType entry, - final boolean active, - final StatusType status - ) throws EntryExistsException - { - try { - getConnector().retryWithHandle( - (HandleCallback) handle -> { - final String sql = StringUtils.format( - "INSERT INTO %s (id, created_date, datasource, payload, active, status_payload) " - + "VALUES (:id, :created_date, :datasource, :payload, :active, :status_payload)", - getEntryTable() - ); - handle.createStatement(sql) - .bind("id", id) - .bind("created_date", timestamp.toString()) - .bind("datasource", dataSource) - .bind("payload", jsonMapper.writeValueAsBytes(entry)) - .bind("active", active) - .bind("status_payload", jsonMapper.writeValueAsBytes(status)) - .execute(); - return null; - }, - e -> getConnector().isTransientException(e) && !(isStatementException(e) && getEntry(id).isPresent()) - ); - } - catch (Exception e) { - if (isStatementException(e) && getEntry(id).isPresent()) { - throw new EntryExistsException(id, e); - } else { - Throwables.propagateIfPossible(e); - throw new RuntimeException(e); - } - } - } - public static boolean isStatementException(Throwable e) { return e instanceof StatementException || @@ -328,16 +285,14 @@ public TaskInfo getTaskInfo(String entryId) } @Override - public List getTaskStatusPlusList( + public List> getTaskInfos( Map taskLookups, - @Nullable String dataSource, - Map runnerWorkItems + @Nullable String dataSource ) { - TaskStatusPlusMapper taskStatusPlusMapper = new TaskStatusPlusMapper<>(jsonMapper, runnerWorkItems); return getConnector().retryTransaction( (handle, status) -> { - final List tasks = new ArrayList<>(); + final List> tasks = new ArrayList<>(); for (Entry entry : taskLookups.entrySet()) { final Query> query; switch (entry.getKey()) { @@ -346,7 +301,7 @@ public List getTaskStatusPlusList( handle, dataSource ); - tasks.addAll(query.map(taskStatusPlusMapper).list()); + tasks.addAll(query.map(taskInfoMapper).list()); break; case COMPLETE: CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); @@ -356,7 +311,7 @@ public List getTaskStatusPlusList( completeTaskLookup.getMaxTaskStatuses(), dataSource ); - tasks.addAll(query.map(taskStatusPlusMapper).list()); + tasks.addAll(query.map(taskInfoMapper).list()); break; default: throw new IAE("Unknown TaskLookupType: [%s]", entry.getKey()); @@ -370,33 +325,45 @@ public List getTaskStatusPlusList( } @Override - public List> getTaskInfos( + public List, StatusType>> getTaskSummaryList( + Map taskLookups, + @Nullable String dataSource, + boolean taskMigrationComplete + ) + { + if (taskMigrationComplete) { + getTaskSummaryList(taskLookups, dataSource); + } + return getTaskSummaryListFromPayload(taskLookups, dataSource); + } + + public List, StatusType>> getTaskSummaryList( Map taskLookups, @Nullable String dataSource ) { return getConnector().retryTransaction( (handle, status) -> { - final List> tasks = new ArrayList<>(); + final List, StatusType>> tasks = new ArrayList<>(); for (Entry entry : taskLookups.entrySet()) { final Query> query; switch (entry.getKey()) { case ACTIVE: - query = createActiveTaskInfoStreamingQuery( + query = createActiveTaskSummaryStreamingQuery( handle, dataSource ); - tasks.addAll(query.map(taskInfoMapper).list()); + tasks.addAll(query.map(taskStatusMapper).list()); break; case COMPLETE: CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); - query = createCompletedTaskInfoStreamingQuery( + query = createCompletedTaskSummaryStreamingQuery( handle, completeTaskLookup.getTasksCreatedPriorTo(), completeTaskLookup.getMaxTaskStatuses(), dataSource ); - tasks.addAll(query.map(taskInfoMapper).list()); + tasks.addAll(query.map(taskStatusMapper).list()); break; default: throw new IAE("Unknown TaskLookupType: [%s]", entry.getKey()); @@ -409,8 +376,7 @@ public List> getTaskInfos( ); } - @Override - public List, StatusType>> getTaskSummaryList( + public List, StatusType>> getTaskSummaryListFromPayload( Map taskLookups, @Nullable String dataSource ) @@ -422,21 +388,21 @@ public List, StatusType>> getTaskSummaryList( final Query> query; switch (entry.getKey()) { case ACTIVE: - query = createActiveTaskSummaryStreamingQuery( + query = createActiveTaskInfoStreamingQuery( handle, dataSource ); - tasks.addAll(query.map(taskStatusMapper).list()); + tasks.addAll(query.map(taskPayloadMapper).list()); break; case COMPLETE: CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); - query = createCompletedTaskSummaryStreamingQuery( + query = createCompletedTaskInfoStreamingQuery( handle, completeTaskLookup.getTasksCreatedPriorTo(), completeTaskLookup.getMaxTaskStatuses(), dataSource ); - tasks.addAll(query.map(taskStatusMapper).list()); + tasks.addAll(query.map(taskPayloadMapper).list()); break; default: throw new IAE("Unknown TaskLookupType: [%s]", entry.getKey()); @@ -449,7 +415,6 @@ public List, StatusType>> getTaskSummaryList( ); } - protected Query> createCompletedTaskSummaryStreamingQuery( Handle handle, DateTime timestamp, @@ -598,135 +563,83 @@ private String getWhereClauseForActiveStatusesQuery(String dataSource) return sql; } - static class TaskStatusPlusMapper implements ResultSetMapper + static class TaskSummaryMapper implements ResultSetMapper, StatusType>> { private final ObjectMapper objectMapper; - private final Map runnerWorkItems; + private final TypeReference statusType; - TaskStatusPlusMapper(ObjectMapper objectMapper, Map runnerWorkItems) + TaskSummaryMapper(ObjectMapper objectMapper, TypeReference statusType) { this.objectMapper = objectMapper; - this.runnerWorkItems = runnerWorkItems; + this.statusType = statusType; } @Override - public TaskStatusPlus map(int index, ResultSet resultSet, StatementContext context) + public TaskInfo, StatusType> map(int index, ResultSet resultSet, StatementContext context) throws SQLException { - ObjectNode task; - try { - task = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); - } - catch (IOException e) { - log.warn("Encountered exception[%s] while deserializing task payload, setting payload to null", - e.getMessage()); - task = null; - } - - ObjectNode status; + final TaskInfo, StatusType> taskInfo; + StatusType status; try { - status = objectMapper.readValue(resultSet.getBytes("status_payload"), ObjectNode.class); + status = objectMapper.readValue(resultSet.getBytes("status_payload"), statusType); } catch (IOException e) { log.error(e, "Encountered exception while deserializing task status_payload"); throw new SQLException(e); } - final String id = resultSet.getString("id"); - final String datasource = resultSet.getString("datasource"); - final DateTime createdDate = DateTimes.of(resultSet.getString("created_date")); - return getTaskStatusPlus(id, datasource, createdDate, task, status); - } - - private TaskStatusPlus getTaskStatusPlus(String id, String datasource, DateTime createdDate, - ObjectNode task, ObjectNode status) - { - final TaskState statusCode = TaskState.valueOf(status.get("status").asText()); - final TaskRunnerWorkItem runnerWorkItem = runnerWorkItems.get(id); - TaskLocation taskLocation; - if (!statusCode.isRunnable() || runnerWorkItem == null) { - // COMPLETED OR WAITING task - try { - taskLocation = objectMapper.treeToValue(status.get("location"), TaskLocation.class); - } - catch (JsonProcessingException e) { - throw new IllegalArgumentException(e); - } - TaskStatusPlus taskStatusPlus = new TaskStatusPlus( - id, - task == null ? null : task.get("groupId").asText(), - task == null ? null : task.get("type").asText(), - createdDate, - DateTimes.EPOCH, - statusCode, - null, // set below to NONE(completed) or WAITING - status.get("duration").asLong(), - taskLocation, - datasource, - status.get("errorMsg").asText() - ); - if (!statusCode.isRunnable()) { - taskStatusPlus.setRunnerTaskState(RunnerTaskState.NONE); - } else { - taskStatusPlus.setRunnerTaskState(RunnerTaskState.WAITING); - } - return taskStatusPlus; - } else { - DateTime createdTime; - DateTime queueInsertionTime; - try { - ObjectNode runnerWorkItemJson = objectMapper.convertValue(runnerWorkItem, ObjectNode.class); - createdTime = objectMapper.treeToValue(runnerWorkItemJson.get("createdTime"), DateTime.class); - queueInsertionTime = objectMapper.treeToValue(runnerWorkItemJson.get("queueInsertionTime"), DateTime.class); - // location in taskInfo is only updated after the task is done. - taskLocation = objectMapper.treeToValue(runnerWorkItemJson.get("location"), TaskLocation.class); - } - catch (JsonProcessingException e) { - throw new IllegalArgumentException(e); - } - return new TaskStatusPlus( - id, - task == null ? null : task.get("groupId").asText(), - task == null ? null : task.get("type").asText(), - createdTime, - queueInsertionTime, - statusCode, - null, // Set later in OverlordResource to taskRunner.getRunnerTaskState(id) - status.get("duration").asLong(), - taskLocation, - datasource, - status.get("errorMsg").asText() - ); + ImmutableMap.Builder task = ImmutableMap.builder(); + String groupId = resultSet.getString("group_id"); + if (groupId != null) { + task.put("groupId", groupId); + } + String type = resultSet.getString("type"); + if (type != null) { + task.put("type", type); } + taskInfo = new TaskInfo<>( + resultSet.getString("id"), + DateTimes.of(resultSet.getString("created_date")), + status, + resultSet.getString("datasource"), + task.build() + ); + return taskInfo; } } - static class TaskInfoMapper implements ResultSetMapper> + static class TaskPayloadMapper implements ResultSetMapper, StatusType>> { private final ObjectMapper objectMapper; - private final TypeReference entryType; private final TypeReference statusType; - TaskInfoMapper(ObjectMapper objectMapper, TypeReference entryType, TypeReference statusType) + TaskPayloadMapper(ObjectMapper objectMapper, TypeReference statusType) { this.objectMapper = objectMapper; - this.entryType = entryType; this.statusType = statusType; } + @Override - public TaskInfo map(int index, ResultSet resultSet, StatementContext context) + public TaskInfo, StatusType> map(int index, ResultSet resultSet, StatementContext context) throws SQLException { - final TaskInfo taskInfo; - EntryType task; - StatusType status; + final TaskInfo, StatusType> taskInfo; + final ImmutableMap.Builder task = ImmutableMap.builder(); try { - task = objectMapper.readValue(resultSet.getBytes("payload"), entryType); + ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); + String groupId = payload.get("groupId").asText(); + if (groupId != null) { + task.put("groupId", groupId); + } + String type = payload.get("type").asText(); + if (type != null) { + task.put("type", type); + } } catch (IOException e) { - log.warn("Encountered exception[%s] while deserializing task payload, setting payload to null", e.getMessage()); - task = null; + log.warn("Encountered exception[%s] while deserializing task payload", e.getMessage()); } + StatusType status; try { status = objectMapper.readValue(resultSet.getBytes("status_payload"), statusType); } @@ -739,29 +652,39 @@ public TaskInfo map(int index, ResultSet resultSet, State DateTimes.of(resultSet.getString("created_date")), status, resultSet.getString("datasource"), - task + task.build() ); return taskInfo; } } - static class TaskStatusMapper implements ResultSetMapper, StatusType>> + static class TaskInfoMapper implements ResultSetMapper> { private final ObjectMapper objectMapper; + private final TypeReference entryType; private final TypeReference statusType; - TaskStatusMapper(ObjectMapper objectMapper, TypeReference statusType) + TaskInfoMapper(ObjectMapper objectMapper, TypeReference entryType, TypeReference statusType) { this.objectMapper = objectMapper; + this.entryType = entryType; this.statusType = statusType; } @Override - public TaskInfo, StatusType> map(int index, ResultSet resultSet, StatementContext context) + public TaskInfo map(int index, ResultSet resultSet, StatementContext context) throws SQLException { - final TaskInfo, StatusType> taskInfo; + final TaskInfo taskInfo; + EntryType task; StatusType status; + try { + task = objectMapper.readValue(resultSet.getBytes("payload"), entryType); + } + catch (IOException e) { + log.warn("Encountered exception[%s] while deserializing task payload, setting payload to null", e.getMessage()); + task = null; + } try { status = objectMapper.readValue(resultSet.getBytes("status_payload"), statusType); } @@ -769,21 +692,12 @@ public TaskInfo, StatusType> map(int index, ResultSet result log.error(e, "Encountered exception while deserializing task status_payload"); throw new SQLException(e); } - ImmutableMap.Builder task = ImmutableMap.builder(); - String groupId = resultSet.getString("group_id"); - if (groupId != null) { - task.put("groupId", groupId); - } - String type = resultSet.getString("type"); - if (type != null) { - task.put("type", type); - } taskInfo = new TaskInfo<>( resultSet.getString("id"), DateTimes.of(resultSet.getString("created_date")), status, resultSet.getString("datasource"), - task.build() + task ); return taskInfo; } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandler.java index c6cd8c671e1e..2c4e93820d25 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandler.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; -public class SQLServerMetadataStorageActionHandler - extends SQLMetadataStorageActionHandler +public class SQLServerMetadataStorageActionHandler + extends SQLMetadataStorageActionHandler { public SQLServerMetadataStorageActionHandler( SQLMetadataConnector connector, diff --git a/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java b/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java index 430fd5e932e9..af6725175340 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLServerMetadataStorageActionHandlerFactory.java @@ -35,8 +35,8 @@ public SQLServerMetadataStorageActionHandlerFactory( } @Override - public - MetadataStorageActionHandler create( + public + MetadataStorageActionHandler create( String entryType, MetadataStorageActionHandlerTypes payloadTypes ) diff --git a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java index 989d0decf3db..f75cd0a4b719 100644 --- a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java @@ -114,6 +114,12 @@ public String getValidationQuery() return "VALUES 1"; } + @Override + public String withFetchLimit(int limit) + { + return String.format("FETCH NEXT %d ROWS ONLY", limit); + } + @Override public void exportTable( String tableName, diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java index 9c55a252b92d..a4d45b515b46 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -170,6 +170,12 @@ public int getStreamingFetchSize() return 0; } + @Override + public String withFetchLimit(int limit) + { + return ""; + } + @Override public String getQuoteString() { diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 75f1ae54a0c2..30e7cc105c60 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -53,7 +53,7 @@ public class SQLMetadataStorageActionHandlerTest public final ExpectedException thrown = ExpectedException.none(); private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); - private SQLMetadataStorageActionHandler, Map, Map, Map, Map> handler; + private SQLMetadataStorageActionHandler, Map, Map, Map> handler; @Before public void setUp() @@ -65,7 +65,7 @@ public void setUp() final String logTable = "logs"; final String lockTable = "locks"; - connector.createEntryTable(entryTable); + connector.prepareEntryTable(entryTable); connector.createLockTable(lockTable, entryType); connector.createLogTable(logTable, entryType); @@ -120,7 +120,7 @@ public void testEntryAndStatus() throws Exception final String entryId = "1234"; - handler.insert(entryId, DateTimes.of("2014-01-02T00:00:00.123"), "testDataSource", entry, true, null); + handler.insert(entryId, DateTimes.of("2014-01-02T00:00:00.123"), "testDataSource", entry, true, null, "type", "group"); Assert.assertEquals( Optional.of(entry), @@ -198,7 +198,7 @@ public void testGetRecentStatuses() throws EntryExistsException 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); + handler.insert(entryId, DateTimes.of(StringUtils.format("2014-01-%02d", i)), "test", entry, false, status, "type", "group"); } final List, Map>> statuses = handler.getTaskInfos( @@ -223,7 +223,7 @@ public void testGetRecentStatuses2() throws EntryExistsException 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); + handler.insert(entryId, DateTimes.of(StringUtils.format("2014-01-%02d", i)), "test", entry, false, status, "type", "group"); } final List, Map>> statuses = handler.getTaskInfos( @@ -247,10 +247,10 @@ public void testRepeatInsert() throws Exception Map entry = ImmutableMap.of("a", 1); Map status = ImmutableMap.of("count", 42); - handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status); + handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); thrown.expect(EntryExistsException.class); - handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status); + handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); } @Test @@ -260,7 +260,7 @@ public void testLogs() throws Exception Map entry = ImmutableMap.of("a", 1); Map status = ImmutableMap.of("count", 42); - handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status); + handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); Assert.assertEquals( ImmutableList.of(), @@ -292,7 +292,7 @@ public void testLocks() throws Exception Map entry = ImmutableMap.of("a", 1); Map status = ImmutableMap.of("count", 42); - handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status); + handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); Assert.assertEquals( ImmutableMap.>of(), @@ -337,7 +337,7 @@ public void testReplaceLock() throws EntryExistsException Map entry = ImmutableMap.of("a", 1); Map status = ImmutableMap.of("count", 42); - handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status); + handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); Assert.assertEquals( ImmutableMap.>of(), @@ -367,7 +367,7 @@ public void testGetLockId() throws EntryExistsException Map entry = ImmutableMap.of("a", 1); Map status = ImmutableMap.of("count", 42); - handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status); + handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); Assert.assertEquals( ImmutableMap.>of(), @@ -394,19 +394,19 @@ public void testRemoveTasksOlderThan() throws Exception final String entryId1 = "1234"; Map entry1 = ImmutableMap.of("numericId", 1234); Map status1 = ImmutableMap.of("count", 42, "temp", 1); - handler.insert(entryId1, DateTimes.of("2014-01-01T00:00:00.123"), "testDataSource", entry1, false, status1); + handler.insert(entryId1, DateTimes.of("2014-01-01T00:00:00.123"), "testDataSource", entry1, false, status1, "type", "group"); Assert.assertTrue(handler.addLog(entryId1, ImmutableMap.of("logentry", "created"))); final String entryId2 = "ABC123"; Map entry2 = ImmutableMap.of("a", 1); Map status2 = ImmutableMap.of("count", 42); - handler.insert(entryId2, DateTimes.of("2014-01-01T00:00:00.123"), "test", entry2, true, status2); + handler.insert(entryId2, DateTimes.of("2014-01-01T00:00:00.123"), "test", entry2, true, status2, "type", "group"); Assert.assertTrue(handler.addLog(entryId2, ImmutableMap.of("logentry", "created"))); final String entryId3 = "DEF5678"; Map entry3 = ImmutableMap.of("numericId", 5678); Map status3 = ImmutableMap.of("count", 21, "temp", 2); - handler.insert(entryId3, DateTimes.of("2014-01-02T12:00:00.123"), "testDataSource", entry3, false, status3); + handler.insert(entryId3, DateTimes.of("2014-01-02T12:00:00.123"), "testDataSource", entry3, false, status3, "type", "group"); Assert.assertTrue(handler.addLog(entryId3, ImmutableMap.of("logentry", "created"))); Assert.assertEquals(Optional.of(entry1), handler.getEntry(entryId1)); From c784c3ab75aa9405fffb8ad72b6ee8ef6cdb78f9 Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 14 Apr 2022 12:16:08 +0530 Subject: [PATCH 04/16] Fix existing unit tests --- .../apache/druid/indexer/TaskStatusPlus.java | 10 +- .../metadata/MetadataStorageConnector.java | 2 +- .../storage/sqlserver/SQLServerConnector.java | 3 +- .../storage/mysql/MySQLConnector.java | 3 +- .../postgresql/PostgreSQLConnector.java | 3 +- .../overlord/HeapMemoryTaskStorage.java | 57 ++++++++- .../overlord/MetadataTaskStorage.java | 6 - .../overlord/TaskStorageQueryAdapter.java | 19 --- .../overlord/http/OverlordResourceTest.java | 116 +++++++++--------- .../SQLMetadataStorageActionHandler.java | 3 - .../storage/derby/DerbyConnector.java | 4 +- 11 files changed, 125 insertions(+), 101 deletions(-) diff --git a/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java b/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java index 0aba225ab36c..75bbefa0eaf3 100644 --- a/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java +++ b/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java @@ -35,7 +35,7 @@ public class TaskStatusPlus private final DateTime createdTime; private final DateTime queueInsertionTime; private final TaskState statusCode; - private RunnerTaskState runnerTaskState; + private final RunnerTaskState runnerTaskState; private final Long duration; private final TaskLocation location; private final String dataSource; @@ -197,14 +197,6 @@ public String getErrorMsg() return errorMsg; } - public void setRunnerTaskState(RunnerTaskState runnerTaskState) - { - if (this.runnerTaskState != null) { - throw new IllegalAccessError("Cannot set runnerTaskState since it has already been set"); - } - this.runnerTaskState = runnerTaskState; - } - @Override public boolean equals(Object o) { diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java index 263533595470..d50d69d4937c 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -87,7 +87,7 @@ default void exportTable( void createSupervisorsTable(); - boolean migrateTaskTable() throws Exception; + boolean migrateTaskTable(); void deleteAllRecords(String tableName); } diff --git a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java index 7d72dfd34efc..94612fd91f92 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java @@ -40,6 +40,7 @@ import java.sql.SQLException; import java.util.Arrays; import java.util.HashSet; +import java.util.Locale; import java.util.Set; import java.util.regex.Pattern; @@ -215,7 +216,7 @@ public boolean tableExists(final Handle handle, final String tableName) @Override public String withFetchLimit(int limit) { - return String.format("FETCH NEXT %d ROWS ONLY", limit); + return String.format(Locale.ENGLISH, "FETCH NEXT %d ROWS ONLY", limit); } /** diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java index 5e80cbb09ac3..04f163049809 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java @@ -37,6 +37,7 @@ import javax.annotation.Nullable; import java.io.File; import java.sql.SQLException; +import java.util.Locale; public class MySQLConnector extends SQLMetadataConnector { @@ -180,7 +181,7 @@ public int getStreamingFetchSize() @Override public String withFetchLimit(int limit) { - return String.format("LIMIT %d", limit); + return String.format(Locale.ENGLISH, "LIMIT %d", limit); } @Override diff --git a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java index ffb4c21d156f..c873a89f4522 100644 --- a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java +++ b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java @@ -40,6 +40,7 @@ import java.sql.DatabaseMetaData; import java.sql.SQLException; import java.util.List; +import java.util.Locale; public class PostgreSQLConnector extends SQLMetadataConnector { @@ -147,7 +148,7 @@ public int getStreamingFetchSize() @Override public String withFetchLimit(int limit) { - return String.format("LIMIT %d", limit); + return String.format(Locale.ENGLISH, "LIMIT %d", limit); } protected boolean canUpsert(Handle handle) throws SQLException diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index c738a18d1807..4756f1aff408 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -45,6 +45,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -170,6 +171,40 @@ public List getActiveTasksByDatasource(String datasource) return listBuilder.build(); } + public List, TaskStatus>> getActiveTaskSummary(@Nullable String dataSource) + { + final ImmutableList.Builder, TaskStatus>> listBuilder = ImmutableList.builder(); + for (final TaskStuff taskStuff : tasks.values()) { + if (taskStuff.getStatus().isRunnable()) { + if (dataSource == null || dataSource.equals(taskStuff.getDataSource())) { + TaskInfo taskInfo = TaskStuff.toTaskInfo(taskStuff); + } + } + } + return listBuilder.build(); + } + + private TaskInfo, TaskStatus> taskSummary(TaskInfo taskInfo) + { + Task task = taskInfo.getTask(); + Map taskMap = new HashMap<>(); + String type = task.getType(); + if (type != null) { + taskMap.put("type", type); + } + String groupId = task.getGroupId(); + if (groupId != null) { + taskMap.put("groupId", groupId); + } + return new TaskInfo<>( + taskInfo.getId(), + taskInfo.getCreatedTime(), + taskInfo.getStatus(), + taskInfo.getDataSource(), + taskMap + ); + } + public List> getActiveTaskInfo(@Nullable String dataSource) { final ImmutableList.Builder> listBuilder = ImmutableList.builder(); @@ -239,7 +274,27 @@ public List, TaskStatus>> getTaskSummaryList( @Nullable String datasource ) { - return Collections.emptyList(); + final List, TaskStatus>> tasks = new ArrayList<>(); + taskLookups.forEach((type, lookup) -> { + if (type == TaskLookupType.COMPLETE) { + CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) lookup; + tasks.addAll( + getRecentlyCreatedAlreadyFinishedTaskInfo( + completeTaskLookup.hasTaskCreatedTimeFilter() + ? completeTaskLookup + : completeTaskLookup.withDurationBeforeNow(config.getRecentlyFinishedThreshold()), + datasource + ).stream() + .map(taskTaskStatusTaskInfo -> taskSummary(taskTaskStatusTaskInfo)) + .collect(Collectors.toList()) + ); + } else { + tasks.addAll(getActiveTaskInfo(datasource).stream() + .map(taskInfo -> taskSummary(taskInfo)) + .collect(Collectors.toList())); + } + }); + return tasks; } private List> getRecentlyCreatedAlreadyFinishedTaskInfoSince( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 196123a7434d..4e1e0e6f88e4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -147,12 +147,6 @@ public void insert(final Task task, final TaskStatus status) throws EntryExistsE log.info("Inserting task %s with status: %s", task.getId(), status); - String dummy = "dummy"; - while (dummy.length() < (int) 1e5) { - dummy += dummy; - } - task.getContext().put("dummy", dummy); - try { handler.insert( task.getId(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java index 7160a7a977ba..b6c69017d5ff 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -29,10 +29,8 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; -import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup; import org.apache.druid.metadata.TaskLookup.TaskLookupType; import org.apache.druid.timeline.DataSegment; -import org.joda.time.Duration; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -85,15 +83,6 @@ public List> getActiveTaskInfo(@Nullable String dataS ); } - public List> getCompletedTaskInfoByCreatedTimeDuration( - @Nullable Integer maxTaskStatuses, - @Nullable Duration duration, - @Nullable String dataSource - ) - { - return storage.getTaskInfos(CompleteTaskLookup.of(maxTaskStatuses, duration), dataSource); - } - public List, TaskStatus>> getTaskSummaryList( Map taskLookups, @Nullable String dataSource @@ -102,14 +91,6 @@ public List, TaskStatus>> getTaskSummaryList( return storage.getTaskSummaryList(taskLookups, dataSource); } - public List> getTaskInfos( - Map taskLookups, - @Nullable String dataSource - ) - { - return storage.getTaskInfos(taskLookups, dataSource); - } - public Optional getTask(final String taskid) { return storage.getTask(taskid); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java index 15a8619eb28b..1b6b72d62977 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java @@ -231,7 +231,7 @@ public void testSecuredGetWaitingTask() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -242,28 +242,28 @@ public void testSecuredGetWaitingTask() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "allow", - getTaskWithIdAndDatasource("id_1", "allow") + null ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_2"), "allow", - getTaskWithIdAndDatasource("id_2", "allow") + null ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_3"), "deny", - getTaskWithIdAndDatasource("id_3", "deny") + null ), new TaskInfo<>( "id_4", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_4"), "deny", - getTaskWithIdAndDatasource("id_4", "deny") + null ) ) ); @@ -297,7 +297,7 @@ public void testSecuredGetCompleteTasks() List tasksIds = ImmutableList.of("id_1", "id_2", "id_3"); EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of(TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, (Duration) null)), null) ).andStubReturn( ImmutableList.of( @@ -306,21 +306,21 @@ public void testSecuredGetCompleteTasks() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_1"), "deny", - getTaskWithIdAndDatasource("id_1", "deny") + null ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_2"), "allow", - getTaskWithIdAndDatasource("id_2", "allow") + null ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_3"), "allow", - getTaskWithIdAndDatasource("id_3", "allow") + null ) ) ); @@ -352,7 +352,7 @@ public void testSecuredGetRunningTasks() ) ); EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -363,14 +363,14 @@ public void testSecuredGetRunningTasks() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "deny", - getTaskWithIdAndDatasource("id_1", "deny") + null ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_2"), "allow", - getTaskWithIdAndDatasource("id_2", "allow") + null ) ) ); @@ -398,7 +398,7 @@ public void testGetTasks() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance(), @@ -414,42 +414,42 @@ public void testGetTasks() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_5"), "deny", - getTaskWithIdAndDatasource("id_5", "deny") + null ), new TaskInfo<>( "id_6", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_6"), "allow", - getTaskWithIdAndDatasource("id_6", "allow") + null ), new TaskInfo<>( "id_7", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_7"), "allow", - getTaskWithIdAndDatasource("id_7", "allow") + null ), new TaskInfo<>( "id_5", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_5"), "deny", - getTaskWithIdAndDatasource("id_5", "deny") + null ), new TaskInfo<>( "id_6", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_6"), "allow", - getTaskWithIdAndDatasource("id_6", "allow") + null ), new TaskInfo<>( "id_7", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_7"), "allow", - getTaskWithIdAndDatasource("id_7", "allow") + null ) ) ); @@ -481,7 +481,7 @@ public void testGetTasksFilterDataSource() expectAuthorizationTokenCheck(); //completed tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -497,49 +497,49 @@ public void testGetTasksFilterDataSource() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_5"), "allow", - getTaskWithIdAndDatasource("id_5", "allow") + null ), new TaskInfo<>( "id_6", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_6"), "allow", - getTaskWithIdAndDatasource("id_6", "allow") + null ), new TaskInfo<>( "id_7", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_7"), "allow", - getTaskWithIdAndDatasource("id_7", "allow") + null ), new TaskInfo<>( "id_1", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "allow", - getTaskWithIdAndDatasource("id_1", "allow") + null ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_2"), "allow", - getTaskWithIdAndDatasource("id_2", "allow") + null ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_3"), "allow", - getTaskWithIdAndDatasource("id_3", "allow") + null ), new TaskInfo<>( "id_4", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_4"), "allow", - getTaskWithIdAndDatasource("id_4", "allow") + null ) ) ); @@ -572,7 +572,7 @@ public void testGetTasksFilterWaitingState() expectAuthorizationTokenCheck(); //active tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance() @@ -586,28 +586,28 @@ public void testGetTasksFilterWaitingState() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "allow", - getTaskWithIdAndDatasource("id_1", "allow") + null ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_2"), "allow", - getTaskWithIdAndDatasource("id_2", "allow") + null ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_3"), "deny", - getTaskWithIdAndDatasource("id_3", "deny") + null ), new TaskInfo<>( "id_4", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_4"), "deny", - getTaskWithIdAndDatasource("id_4", "deny") + null ) ) ); @@ -645,7 +645,7 @@ public void testGetTasksFilterRunningState() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance() @@ -659,28 +659,28 @@ public void testGetTasksFilterRunningState() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "allow", - getTaskWithIdAndDatasource("id_1", "allow") + null ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_2"), "allow", - getTaskWithIdAndDatasource("id_2", "allow") + null ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_3"), "allow", - getTaskWithIdAndDatasource("id_3", "allow") + null ), new TaskInfo<>( "id_4", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_4"), "deny", - getTaskWithIdAndDatasource("id_4", "deny") + null ) ) ); @@ -726,7 +726,7 @@ public void testGetTasksFilterPendingState() ) ); EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -737,28 +737,28 @@ public void testGetTasksFilterPendingState() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "deny", - getTaskWithIdAndDatasource("id_1", "deny") + null ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_2"), "allow", - getTaskWithIdAndDatasource("id_2", "allow") + null ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_3"), "allow", - getTaskWithIdAndDatasource("id_3", "allow") + null ), new TaskInfo<>( "id_4", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_4"), "deny", - getTaskWithIdAndDatasource("id_4", "deny") + null ) ) ); @@ -791,7 +791,7 @@ public void testGetTasksFilterCompleteState() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of(TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, (Duration) null)), null ) @@ -802,21 +802,21 @@ public void testGetTasksFilterCompleteState() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_1"), "allow", - getTaskWithIdAndDatasource("id_1", "allow") + null ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_2"), "deny", - getTaskWithIdAndDatasource("id_2", "deny") + null ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_3"), "allow", - getTaskWithIdAndDatasource("id_3", "allow") + null ) ) ); @@ -842,7 +842,7 @@ public void testGetTasksFilterCompleteStateWithInterval() expectAuthorizationTokenCheck(); Duration duration = new Period("PT86400S").toStandardDuration(); EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( EasyMock.anyObject(), EasyMock.anyObject() ) @@ -853,21 +853,21 @@ public void testGetTasksFilterCompleteStateWithInterval() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_1"), "deny", - getTaskWithIdAndDatasource("id_1", "deny") + null ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_2"), "allow", - getTaskWithIdAndDatasource("id_2", "allow") + null ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_3"), "allow", - getTaskWithIdAndDatasource("id_3", "allow") + null ) ) ); @@ -898,7 +898,7 @@ public void testGetTasksRequiresDatasourceRead() // Setup mocks to return completed, active, known, pending and running tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -955,7 +955,7 @@ public void testGetTasksFilterByTaskTypeRequiresDatasourceRead() // Setup mocks to return completed, active, known, pending and running tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -1029,7 +1029,7 @@ public void testGetNullCompleteTask() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskInfos( + taskStorageQueryAdapter.getTaskSummaryList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null) @@ -1050,14 +1050,14 @@ public void testGetNullCompleteTask() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_2"), "deny", - getTaskWithIdAndDatasource("id_2", "deny") + null ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_3"), "allow", - getTaskWithIdAndDatasource("id_3", "allow") + null ) ) ); @@ -1758,7 +1758,7 @@ public TaskStatus run(TaskToolbox toolbox) }; } - private TaskInfo createTaskInfo( + private TaskInfo, TaskStatus> createTaskInfo( String taskId, String datasource ) @@ -1766,7 +1766,7 @@ private TaskInfo createTaskInfo( return createTaskInfo(taskId, datasource, TaskState.SUCCESS, "test"); } - private TaskInfo createTaskInfo( + private TaskInfo, TaskStatus> createTaskInfo( String taskId, String datasource, TaskState state, @@ -1778,7 +1778,7 @@ private TaskInfo createTaskInfo( DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.fromCode(taskId, state), datasource, - getTaskWithIdAndDatasource(taskId, datasource, taskType) + ImmutableMap.of("type", taskType) ); } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index 366a0fd3398b..e9ae07fc258a 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -27,7 +27,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; -import net.minidev.json.parser.JSONParser; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; @@ -64,7 +63,6 @@ public abstract class SQLMetadataStorageActionHandler entryType; private final TypeReference statusType; private final TypeReference logType; @@ -95,7 +93,6 @@ public SQLMetadataStorageActionHandler( //noinspection UnnecessaryFullyQualifiedName this.jsonMapper = jsonMapper.copy().addMixIn(org.apache.druid.metadata.PasswordProvider.class, org.apache.druid.metadata.PasswordProviderRedactionMixIn.class); - this.jsonParser = new JSONParser(); this.entryType = types.getEntryType(); this.statusType = types.getStatusType(); this.logType = types.getLogType(); diff --git a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java index f75cd0a4b719..f5b9abf0fcc5 100644 --- a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java @@ -35,6 +35,8 @@ import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.tweak.HandleCallback; +import java.util.Locale; + @ManageLifecycle public class DerbyConnector extends SQLMetadataConnector { @@ -117,7 +119,7 @@ public String getValidationQuery() @Override public String withFetchLimit(int limit) { - return String.format("FETCH NEXT %d ROWS ONLY", limit); + return String.format(Locale.ENGLISH, "FETCH NEXT %d ROWS ONLY", limit); } @Override From 74fac09fa30ba872f5e08facf094a70be42bf89f Mon Sep 17 00:00:00 2001 From: Amatya Date: Tue, 19 Apr 2022 06:39:03 +0530 Subject: [PATCH 05/16] SQLMetadataConnector migration fix and tests --- .../druid/metadata/SQLMetadataConnector.java | 55 +++++---- .../metadata/SQLMetadataConnectorTest.java | 116 ++++++++++++++++++ .../SQLMetadataStorageActionHandlerTest.java | 2 +- 3 files changed, 151 insertions(+), 22 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index f51ea04b90aa..cb8b26bb492f 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -337,18 +337,24 @@ tableName, getPayloadType() ); } - public boolean tableContainsColumn(Handle handle, String table, String column) throws SQLException + public boolean tableContainsColumn(Handle handle, String table, String column) { - DatabaseMetaData databaseMetaData = handle.getConnection().getMetaData(); - ResultSet columns = databaseMetaData.getColumns(null, - null, - table.toUpperCase(Locale.ENGLISH), - column.toUpperCase(Locale.ENGLISH) - ); - return columns.next(); + try { + DatabaseMetaData databaseMetaData = handle.getConnection().getMetaData(); + ResultSet columns = databaseMetaData.getColumns( + null, + null, + table.toUpperCase(Locale.ENGLISH), + column.toUpperCase(Locale.ENGLISH) + ); + return columns.next(); + } + catch (SQLException e) { + return false; + } } - public void prepareEntryTable(final String tableName) + public void prepareTaskEntryTable(final String tableName) { createEntryTable(tableName); alterEntryTable(tableName); @@ -385,10 +391,15 @@ public void alterEntryTable(final String tableName) @Override public Void withHandle(Handle handle) { - log.info("Altering table[%s]", tableName); final Batch batch = handle.createBatch(); - batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN type VARCHAR(255)", tableName)); - batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN group_id VARCHAR(255)", tableName)); + if (!tableContainsColumn(handle, tableName, "type")) { + log.info("Adding column: type to table[%s]", tableName); + batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN type VARCHAR(255)", tableName)); + } + if (!tableContainsColumn(handle, tableName, "group_id")) { + log.info("Adding column: group_id to table[%s]", tableName); + batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN group_id VARCHAR(255)", tableName)); + } batch.execute(); return null; } @@ -416,25 +427,27 @@ public Void withHandle(Handle handle) throws SQLException, IOException { ObjectMapper objectMapper = new ObjectMapper(); Connection connection = handle.getConnection(); - Statement statement = connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_UPDATABLE); - String prevId = "''"; - while (prevId != null) { + Statement statement = connection.createStatement(ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_UPDATABLE); + boolean flag = true; + String limit = withFetchLimit(100); + while (flag) { + // Should ideally use a cursor and sort by id for efficiency, but updates with ordering aren't allowed String sql = StringUtils.format( - "SELECT * FROM %1$s WHERE id > %2$s AND active = false AND type IS null ORDER BY id ASC %3$s", + "SELECT * FROM %1$s WHERE active = false AND type IS null %2$s", tableName, - prevId, - withFetchLimit(100) + limit ); ResultSet resultSet = statement.executeQuery(sql); - prevId = null; + flag = false; while (resultSet.next()) { ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); resultSet.updateString("type", payload.get("type").asText()); resultSet.updateString("group_id", payload.get("groupId").asText()); - prevId = "'" + resultSet.getString("id") + "'"; resultSet.updateRow(); + flag = true; } } + statement.close(); return null; } } @@ -675,7 +688,7 @@ public void createTaskTables() if (config.get().isCreateTables()) { final MetadataStorageTablesConfig tablesConfig = tablesConfigSupplier.get(); final String entryType = tablesConfig.getTaskEntryType(); - prepareEntryTable(tablesConfig.getEntryTable(entryType)); + prepareTaskEntryTable(tablesConfig.getEntryTable(entryType)); createLogTable(tablesConfig.getLogTable(entryType), entryType); createLockTable(tablesConfig.getLockTable(entryType), entryType); } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java index a4d45b515b46..0f14aca7fdb2 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -23,6 +23,7 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import org.apache.commons.dbcp2.BasicDataSource; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.junit.Assert; import org.junit.Before; @@ -32,8 +33,15 @@ import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.tweak.HandleCallback; +import java.sql.Blob; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; +import java.util.UUID; public class SQLMetadataConnectorTest @@ -86,6 +94,14 @@ public Void withHandle(Handle handle) ); } + String taskTable = tablesConfig.getTasksTable(); + for (String column : Arrays.asList("type", "group_id")) { + Assert.assertTrue( + StringUtils.format("Tasks table column %s was not created!", column), + connector.tableContainsColumn(handle, taskTable, column) + ); + } + return null; } } @@ -96,6 +112,37 @@ public Void withHandle(Handle handle) } } + @Test + public void testMigrateTaskTable() + { + String table = tablesConfig.getTasksTable(); + connector.getDBI().withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) + { + connector.prepareTaskEntryTable(table); + int numRows = 1000; + List ids = new ArrayList<>(); + for (int i = 0; i < numRows; i++) { + String id = insertIntoTasksTable(table, "type_" + i, "groupId_" + i); + Assert.assertNotEquals("", id); + ids.add(id); + } + + connector.migrateTaskTable(); + + for (int i = 0; i < numRows; i++) { + Assert.assertTrue(verifyTaskTypeAndGroupId(table, ids.get(i), "type_" + i, "groupId_" + i)); + } + return null; + } + } + ); + dropTable(table); + } + @Test public void testInsertOrUpdate() { @@ -248,4 +295,73 @@ public void testBasicDataSourceCreation() throws Exception Assert.assertEquals(dataSource.getMaxConnLifetimeMillis(), 1200000); Assert.assertEquals((long) dataSource.getDefaultQueryTimeout(), 30000); } + + private String insertIntoTasksTable(String table, String type, String groupId) + { + String id = UUID.randomUUID().toString(); + String dummy = "dummyData"; + String createdDate = DateTimes.nowUtc().toString(); + try { + connector.retryWithHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws SQLException + { + String json = StringUtils.format("{\"type\":\"%1$s\",\"groupId\":\"%2$s\"}", type, groupId); + Blob payload = handle.getConnection().createBlob(); + payload.setBytes(1, json.getBytes()); + + String sql = StringUtils.format( + "INSERT INTO %1$s (id, created_date, datasource, active, payload, status_payload) VALUES (?,?,?,?,?,?)", + table + ); + PreparedStatement preparedStatement = handle.getConnection().prepareStatement(sql); + preparedStatement.setString(1, id); + preparedStatement.setString(2, createdDate); + preparedStatement.setString(3, dummy); + preparedStatement.setBoolean(4, false); + preparedStatement.setBlob(5, payload); + preparedStatement.setBlob(6, payload); + preparedStatement.execute(); + preparedStatement.close(); + + return null; + } + } + ); + return id; + } + catch (Exception e) { + return ""; + } + } + + private boolean verifyTaskTypeAndGroupId(String table, String id, String type, String groupId) + { + try { + return connector.retryWithHandle( + new HandleCallback() + { + @Override + public Boolean withHandle(Handle handle) throws SQLException + { + Statement statement = handle.getConnection().createStatement(); + ResultSet resultSet = statement.executeQuery( + StringUtils.format("SELECT * FROM %1$s WHERE id = '%2$s'", table, id) + ); + resultSet.next(); + boolean flag = type.equals(resultSet.getString("type")) + && groupId.equals(resultSet.getString("group_id")); + statement.close(); + return flag; + } + } + ); + } + catch (Exception e) { + e.printStackTrace(); + return false; + } + } } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 30e7cc105c60..fe4b03fa8a00 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -65,7 +65,7 @@ public void setUp() final String logTable = "logs"; final String lockTable = "locks"; - connector.prepareEntryTable(entryTable); + connector.prepareTaskEntryTable(entryTable); connector.createLockTable(lockTable, entryType); connector.createLogTable(logTable, entryType); From c436ba5f3fef35f8daa119762d301f84f5fa1ef1 Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 20 Apr 2022 16:00:44 +0530 Subject: [PATCH 06/16] Refactoring --- .../MetadataStorageActionHandler.java | 5 +- .../overlord/HeapMemoryTaskStorage.java | 48 ++-- .../overlord/MetadataTaskStorage.java | 7 +- .../druid/indexing/overlord/TaskStorage.java | 5 +- .../overlord/TaskStorageQueryAdapter.java | 5 +- .../overlord/http/OverlordResource.java | 98 +++---- .../overlord/http/OverlordResourceTest.java | 164 ++++++------ .../SQLMetadataStorageActionHandler.java | 240 ++++++++---------- .../metadata/SQLMetadataConnectorTest.java | 3 +- 9 files changed, 266 insertions(+), 309 deletions(-) diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index e1dd9e9f457d..0d8164a95622 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -21,6 +21,7 @@ import com.google.common.base.Optional; import org.apache.druid.indexer.TaskInfo; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.metadata.TaskLookup.TaskLookupType; import org.joda.time.DateTime; @@ -103,7 +104,7 @@ List> getTaskInfos( ); /** - * Returns a list of {@link TaskInfo} from metadata store that matches the given filters. + * Returns a list of TaskStatusPlus for the tasks corresponding to the given filters * * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata @@ -118,7 +119,7 @@ List> getTaskInfos( * @param datasource datasource filter * @param taskMigrationComplete indicates which query to use based on migration status */ - List, StatusType>> getTaskSummaryList( + List getTaskStatusPlusList( Map taskLookups, @Nullable String datasource, boolean taskMigrationComplete diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index 4756f1aff408..f59d25a391b0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -28,8 +28,10 @@ import com.google.common.collect.Ordering; import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; +import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.config.TaskStorageConfig; @@ -45,7 +47,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -171,37 +172,22 @@ public List getActiveTasksByDatasource(String datasource) return listBuilder.build(); } - public List, TaskStatus>> getActiveTaskSummary(@Nullable String dataSource) - { - final ImmutableList.Builder, TaskStatus>> listBuilder = ImmutableList.builder(); - for (final TaskStuff taskStuff : tasks.values()) { - if (taskStuff.getStatus().isRunnable()) { - if (dataSource == null || dataSource.equals(taskStuff.getDataSource())) { - TaskInfo taskInfo = TaskStuff.toTaskInfo(taskStuff); - } - } - } - return listBuilder.build(); - } - - private TaskInfo, TaskStatus> taskSummary(TaskInfo taskInfo) + private TaskStatusPlus toTaskStatusPlus(TaskInfo taskInfo) { Task task = taskInfo.getTask(); - Map taskMap = new HashMap<>(); - String type = task.getType(); - if (type != null) { - taskMap.put("type", type); - } - String groupId = task.getGroupId(); - if (groupId != null) { - taskMap.put("groupId", groupId); - } - return new TaskInfo<>( + TaskStatus status = taskInfo.getStatus(); + return new TaskStatusPlus( taskInfo.getId(), + task == null ? null : task.getGroupId(), + task == null ? null : task.getType(), taskInfo.getCreatedTime(), - taskInfo.getStatus(), + DateTimes.EPOCH, + status.getStatusCode(), + status.getStatusCode().isComplete() ? RunnerTaskState.NONE : RunnerTaskState.WAITING, + status.getDuration(), + status.getLocation(), taskInfo.getDataSource(), - taskMap + status.getErrorMsg() ); } @@ -269,12 +255,12 @@ public List> getTaskInfos( } @Override - public List, TaskStatus>> getTaskSummaryList( + public List getTaskStatusPlusList( Map taskLookups, @Nullable String datasource ) { - final List, TaskStatus>> tasks = new ArrayList<>(); + final List tasks = new ArrayList<>(); taskLookups.forEach((type, lookup) -> { if (type == TaskLookupType.COMPLETE) { CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) lookup; @@ -285,12 +271,12 @@ public List, TaskStatus>> getTaskSummaryList( : completeTaskLookup.withDurationBeforeNow(config.getRecentlyFinishedThreshold()), datasource ).stream() - .map(taskTaskStatusTaskInfo -> taskSummary(taskTaskStatusTaskInfo)) + .map(taskInfo -> toTaskStatusPlus(taskInfo)) .collect(Collectors.toList()) ); } else { tasks.addAll(getActiveTaskInfo(datasource).stream() - .map(taskInfo -> taskSummary(taskInfo)) + .map(taskInfo -> toTaskStatusPlus(taskInfo)) .collect(Collectors.toList())); } }); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 4e1e0e6f88e4..2fde517db0b2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -29,6 +29,7 @@ import com.google.inject.Inject; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.config.TaskStorageConfig; @@ -243,7 +244,7 @@ public List> getTaskInfos( } @Override - public List, TaskStatus>> getTaskSummaryList( + public List getTaskStatusPlusList( Map taskLookups, @Nullable String datasource ) @@ -258,7 +259,9 @@ public List, TaskStatus>> getTaskSummaryList( log.info(e, "Exception getting task migration future"); } } - return Collections.unmodifiableList(handler.getTaskSummaryList(theTaskLookups, datasource, taskMigrationComplete)); + return Collections.unmodifiableList( + handler.getTaskStatusPlusList(theTaskLookups, datasource, taskMigrationComplete) + ); } private Map processTaskLookups( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java index 8b551262f47d..2d5053b6e682 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java @@ -22,6 +22,7 @@ import com.google.common.base.Optional; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.task.Task; @@ -151,7 +152,7 @@ public interface TaskStorage List getActiveTasksByDatasource(String datasource); /** - * Returns a list of tasks summaries stored in the storage facility as {@link TaskInfo}. No + * Returns tasks stored in the storage facility as a List of TaskStatusPlus * particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. * * The returned list can contain active tasks and complete tasks depending on the {@code taskLookups} parameter. @@ -160,7 +161,7 @@ public interface TaskStorage * @param taskLookups lookup types and filters * @param datasource datasource filter */ - List, TaskStatus>> getTaskSummaryList( + List getTaskStatusPlusList( Map taskLookups, @Nullable String datasource ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java index b6c69017d5ff..3fa570ccb32d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -23,6 +23,7 @@ import com.google.inject.Inject; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.actions.SegmentInsertAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskAction; @@ -83,12 +84,12 @@ public List> getActiveTaskInfo(@Nullable String dataS ); } - public List, TaskStatus>> getTaskSummaryList( + public List getTaskStatusPlusList( Map taskLookups, @Nullable String dataSource ) { - return storage.getTaskSummaryList(taskLookups, dataSource); + return storage.getTaskStatusPlusList(taskLookups, dataSource); } public Optional getTask(final String taskid) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index f8225db42c12..aa26d3cd8f50 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -690,7 +690,7 @@ public Response getTasks( taskMaster.getTaskRunner(), taskRunner -> { final List authorizedList = securedTaskStatusPlus( - getTaskSummaryList( + getTaskStatusPlusList( taskRunner, TaskStateLookup.fromString(state), dataSource, @@ -706,7 +706,7 @@ public Response getTasks( ); } - private List getTaskSummaryList( + private List getTaskStatusPlusList( TaskRunner taskRunner, TaskStateLookup state, @Nullable String dataSource, @@ -729,7 +729,7 @@ private List getTaskSummaryList( // This way, we can use the snapshot from taskStorage as the source of truth for the set of tasks to process // and use the snapshot from taskRunner as a reference for potential task state updates happened // after the first snapshotting. - Stream, TaskStatus>> taskInfoStreamFromTaskStorage = getTaskSummaryStreamFromTaskStorage( + Stream taskStatusPlusStream = getTaskStatusPlusList( state, dataSource, createdTimeDuration, @@ -745,87 +745,71 @@ private List getTaskSummaryList( if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING) { // We are interested in only those tasks which are in taskRunner. - taskInfoStreamFromTaskStorage = taskInfoStreamFromTaskStorage - .filter(info -> runnerWorkItems.containsKey(info.getId())); + taskStatusPlusStream = taskStatusPlusStream + .filter(statusPlus -> runnerWorkItems.containsKey(statusPlus.getId())); } - final List, TaskStatus>> taskInfoFromTaskStorage = taskInfoStreamFromTaskStorage - .collect(Collectors.toList()); + final List taskStatusPlusList = taskStatusPlusStream.collect(Collectors.toList()); // Separate complete and active tasks from taskStorage. // Note that taskStorage can return only either complete tasks or active tasks per TaskLookupType. - final List, TaskStatus>> completeTaskInfoFromTaskStorage = new ArrayList<>(); - final List, TaskStatus>> activeTaskInfoFromTaskStorage = new ArrayList<>(); - for (TaskInfo, TaskStatus> info : taskInfoFromTaskStorage) { - if (info.getStatus().isComplete()) { - completeTaskInfoFromTaskStorage.add(info); + final List completeTaskStatusPlusList = new ArrayList<>(); + final List activeTaskStatusPlusList = new ArrayList<>(); + for (TaskStatusPlus statusPlus : taskStatusPlusList) { + if (statusPlus.getStatusCode().isComplete()) { + completeTaskStatusPlusList.add(statusPlus); } else { - activeTaskInfoFromTaskStorage.add(info); + activeTaskStatusPlusList.add(statusPlus); } } - final List statuses = new ArrayList<>(); - completeTaskInfoFromTaskStorage.forEach(taskInfo -> statuses.add( - new TaskStatusPlus( - taskInfo.getId(), - taskInfo.getTask() == null ? null : taskInfo.getTask().get("groupId"), - taskInfo.getTask() == null ? null : taskInfo.getTask().get("type"), - taskInfo.getCreatedTime(), - DateTimes.EPOCH, - taskInfo.getStatus().getStatusCode(), - RunnerTaskState.NONE, - taskInfo.getStatus().getDuration(), - taskInfo.getStatus().getLocation(), - taskInfo.getDataSource(), - taskInfo.getStatus().getErrorMsg() - ) - )); + final List taskStatuses = new ArrayList<>(completeTaskStatusPlusList); - activeTaskInfoFromTaskStorage.forEach(taskInfo -> { - final TaskRunnerWorkItem runnerWorkItem = runnerWorkItems.get(taskInfo.getId()); + activeTaskStatusPlusList.forEach(statusPlus -> { + final TaskRunnerWorkItem runnerWorkItem = runnerWorkItems.get(statusPlus.getId()); if (runnerWorkItem == null) { // a task is assumed to be a waiting task if it exists in taskStorage but not in taskRunner. if (state == TaskStateLookup.WAITING || state == TaskStateLookup.ALL) { - statuses.add( + taskStatuses.add( new TaskStatusPlus( - taskInfo.getId(), - taskInfo.getTask() == null ? null : taskInfo.getTask().get("groupId"), - taskInfo.getTask() == null ? null : taskInfo.getTask().get("type"), - taskInfo.getCreatedTime(), - DateTimes.EPOCH, - taskInfo.getStatus().getStatusCode(), + statusPlus.getId(), + statusPlus.getGroupId(), + statusPlus.getType(), + statusPlus.getCreatedTime(), + statusPlus.getQueueInsertionTime(), + statusPlus.getStatusCode(), RunnerTaskState.WAITING, - taskInfo.getStatus().getDuration(), - taskInfo.getStatus().getLocation(), - taskInfo.getDataSource(), - taskInfo.getStatus().getErrorMsg() + statusPlus.getDuration(), + statusPlus.getLocation(), + statusPlus.getDataSource(), + statusPlus.getErrorMsg() ) ); } } else { if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING || state == TaskStateLookup.ALL) { - statuses.add( + taskStatuses.add( new TaskStatusPlus( - taskInfo.getId(), - taskInfo.getTask() == null ? null : taskInfo.getTask().get("groupId"), - taskInfo.getTask() == null ? null : taskInfo.getTask().get("type"), + statusPlus.getId(), + statusPlus.getGroupId(), + statusPlus.getType(), runnerWorkItem.getCreatedTime(), runnerWorkItem.getQueueInsertionTime(), - taskInfo.getStatus().getStatusCode(), - taskRunner.getRunnerTaskState(taskInfo.getId()), // this is racy for remoteTaskRunner - taskInfo.getStatus().getDuration(), + statusPlus.getStatusCode(), + taskRunner.getRunnerTaskState(statusPlus.getId()), // this is racy for remoteTaskRunner + statusPlus.getDuration(), runnerWorkItem.getLocation(), // location in taskInfo is only updated after the task is done. - taskInfo.getDataSource(), - taskInfo.getStatus().getErrorMsg() + statusPlus.getDataSource(), + statusPlus.getErrorMsg() ) ); } } }); - return statuses; + return taskStatuses; } - private Stream, TaskStatus>> getTaskSummaryStreamFromTaskStorage( + private Stream getTaskStatusPlusList( TaskStateLookup state, @Nullable String dataSource, Duration createdTimeDuration, @@ -861,16 +845,16 @@ private Stream, TaskStatus>> getTaskSummaryStreamFr throw new IAE("Unknown state: [%s]", state); } - final Stream, TaskStatus>> taskInfoStreamFromTaskStorage = taskStorageQueryAdapter.getTaskSummaryList( + final Stream taskStatusPlusStream = taskStorageQueryAdapter.getTaskStatusPlusList( taskLookups, dataSource ).stream(); if (type != null) { - return taskInfoStreamFromTaskStorage.filter( - info -> type.equals(info.getTask() == null ? null : info.getTask().get("type")) + return taskStatusPlusStream.filter( + statusPlus -> type.equals(statusPlus == null ? null : statusPlus.getType()) ); } else { - return taskInfoStreamFromTaskStorage; + return taskStatusPlusStream; } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java index 1b6b72d62977..947dd85f8663 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java @@ -92,6 +92,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; public class OverlordResourceTest { @@ -231,7 +232,7 @@ public void testSecuredGetWaitingTask() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -242,30 +243,30 @@ public void testSecuredGetWaitingTask() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "allow", - null + getTaskWithIdAndDatasource("id_1", "allow") ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_2"), "allow", - null + getTaskWithIdAndDatasource("id_2", "allow") ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_3"), "deny", - null + getTaskWithIdAndDatasource("id_3", "deny") ), new TaskInfo<>( "id_4", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_4"), "deny", - null + getTaskWithIdAndDatasource("id_4", "deny") ) - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( @@ -297,7 +298,7 @@ public void testSecuredGetCompleteTasks() List tasksIds = ImmutableList.of("id_1", "id_2", "id_3"); EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, (Duration) null)), null) ).andStubReturn( ImmutableList.of( @@ -306,23 +307,23 @@ public void testSecuredGetCompleteTasks() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_1"), "deny", - null + getTaskWithIdAndDatasource("id_1", "deny") ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_2"), "allow", - null + getTaskWithIdAndDatasource("id_2", "allow") ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_3"), "allow", - null + getTaskWithIdAndDatasource("id_3", "allow") ) - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.replay( taskRunner, @@ -352,7 +353,7 @@ public void testSecuredGetRunningTasks() ) ); EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -363,16 +364,16 @@ public void testSecuredGetRunningTasks() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "deny", - null + getTaskWithIdAndDatasource("id_1", "deny") ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_2"), "allow", - null + getTaskWithIdAndDatasource("id_2", "allow") ) - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andStubReturn(RunnerTaskState.RUNNING); EasyMock.expect(taskRunner.getRunnerTaskState("id_2")).andStubReturn(RunnerTaskState.RUNNING); @@ -398,7 +399,7 @@ public void testGetTasks() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance(), @@ -414,44 +415,44 @@ public void testGetTasks() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_5"), "deny", - null + getTaskWithIdAndDatasource("id_5", "deny") ), new TaskInfo<>( "id_6", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_6"), "allow", - null + getTaskWithIdAndDatasource("id_6", "allow") ), new TaskInfo<>( "id_7", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_7"), "allow", - null + getTaskWithIdAndDatasource("id_7", "allow") ), new TaskInfo<>( "id_5", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_5"), "deny", - null + getTaskWithIdAndDatasource("id_5", "deny") ), new TaskInfo<>( "id_6", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_6"), "allow", - null + getTaskWithIdAndDatasource("id_6", "allow") ), new TaskInfo<>( "id_7", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_7"), "allow", - null + getTaskWithIdAndDatasource("id_7", "allow") ) - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( @@ -481,7 +482,7 @@ public void testGetTasksFilterDataSource() expectAuthorizationTokenCheck(); //completed tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -497,51 +498,51 @@ public void testGetTasksFilterDataSource() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_5"), "allow", - null + getTaskWithIdAndDatasource("id_5", "allow") ), new TaskInfo<>( "id_6", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_6"), "allow", - null + getTaskWithIdAndDatasource("id_6", "allow") ), new TaskInfo<>( "id_7", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_7"), "allow", - null + getTaskWithIdAndDatasource("id_7", "allow") ), new TaskInfo<>( "id_1", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "allow", - null + getTaskWithIdAndDatasource("id_1", "allow") ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_2"), "allow", - null + getTaskWithIdAndDatasource("id_2", "allow") ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_3"), "allow", - null + getTaskWithIdAndDatasource("id_3", "allow") ), new TaskInfo<>( "id_4", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_4"), "allow", - null + getTaskWithIdAndDatasource("id_4", "allow") ) - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( ImmutableList.of( @@ -572,7 +573,7 @@ public void testGetTasksFilterWaitingState() expectAuthorizationTokenCheck(); //active tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance() @@ -586,30 +587,30 @@ public void testGetTasksFilterWaitingState() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "allow", - null + getTaskWithIdAndDatasource("id_1", "allow") ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_2"), "allow", - null + getTaskWithIdAndDatasource("id_2", "allow") ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_3"), "deny", - null + getTaskWithIdAndDatasource("id_3", "deny") ), new TaskInfo<>( "id_4", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_4"), "deny", - null + getTaskWithIdAndDatasource("id_4", "deny") ) - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( @@ -645,7 +646,7 @@ public void testGetTasksFilterRunningState() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance() @@ -659,30 +660,30 @@ public void testGetTasksFilterRunningState() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "allow", - null + getTaskWithIdAndDatasource("id_1", "allow") ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_2"), "allow", - null + getTaskWithIdAndDatasource("id_2", "allow") ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_3"), "allow", - null + getTaskWithIdAndDatasource("id_3", "allow") ), new TaskInfo<>( "id_4", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_4"), "deny", - null + getTaskWithIdAndDatasource("id_4", "deny") ) - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); List tasksIds = ImmutableList.of("id_1", "id_2"); @@ -726,7 +727,7 @@ public void testGetTasksFilterPendingState() ) ); EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -737,30 +738,30 @@ public void testGetTasksFilterPendingState() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_1"), "deny", - null + getTaskWithIdAndDatasource("id_1", "deny") ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_2"), "allow", - null + getTaskWithIdAndDatasource("id_2", "allow") ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_3"), "allow", - null + getTaskWithIdAndDatasource("id_3", "allow") ), new TaskInfo<>( "id_4", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.running("id_4"), "deny", - null + getTaskWithIdAndDatasource("id_4", "deny") ) - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andStubReturn(RunnerTaskState.PENDING); @@ -791,7 +792,7 @@ public void testGetTasksFilterCompleteState() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, (Duration) null)), null ) @@ -802,23 +803,23 @@ public void testGetTasksFilterCompleteState() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_1"), "allow", - null + getTaskWithIdAndDatasource("id_1", "allow") ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_2"), "deny", - null + getTaskWithIdAndDatasource("id_2", "deny") ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_3"), "allow", - null + getTaskWithIdAndDatasource("id_3", "allow") ) - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.replay( taskRunner, @@ -842,7 +843,7 @@ public void testGetTasksFilterCompleteStateWithInterval() expectAuthorizationTokenCheck(); Duration duration = new Period("PT86400S").toStandardDuration(); EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( EasyMock.anyObject(), EasyMock.anyObject() ) @@ -853,23 +854,23 @@ public void testGetTasksFilterCompleteStateWithInterval() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_1"), "deny", - null + getTaskWithIdAndDatasource("id_1", "deny") ), new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_2"), "allow", - null + getTaskWithIdAndDatasource("id_2", "allow") ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_3"), "allow", - null + getTaskWithIdAndDatasource("id_3", "allow") ) - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.replay( @@ -898,7 +899,7 @@ public void testGetTasksRequiresDatasourceRead() // Setup mocks to return completed, active, known, pending and running tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -913,7 +914,7 @@ public void testGetTasksRequiresDatasourceRead() createTaskInfo("id_6", Datasources.BUZZFEED), createTaskInfo("id_1", Datasources.WIKIPEDIA, TaskState.RUNNING, "test"), createTaskInfo("id_4", Datasources.BUZZFEED, TaskState.RUNNING, "test") - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( @@ -955,7 +956,7 @@ public void testGetTasksFilterByTaskTypeRequiresDatasourceRead() // Setup mocks to return completed, active, known, pending and running tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -970,7 +971,7 @@ public void testGetTasksFilterByTaskTypeRequiresDatasourceRead() createTaskInfo("id_6", Datasources.BUZZFEED), createTaskInfo("id_1", Datasources.WIKIPEDIA, TaskState.RUNNING, "to-return"), createTaskInfo("id_4", Datasources.WIKIPEDIA, TaskState.RUNNING, "test") - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( @@ -1029,7 +1030,7 @@ public void testGetNullCompleteTask() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskSummaryList( + taskStorageQueryAdapter.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null) @@ -1038,7 +1039,7 @@ public void testGetNullCompleteTask() ) ).andStubReturn( ImmutableList.of( - new TaskInfo<>( + new TaskInfo( "id_1", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_1"), @@ -1050,16 +1051,16 @@ public void testGetNullCompleteTask() DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_2"), "deny", - null + getTaskWithIdAndDatasource("id_2", "deny") ), new TaskInfo<>( "id_3", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_3"), "allow", - null + getTaskWithIdAndDatasource("id_3", "allow") ) - ) + ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) ); EasyMock.replay( taskRunner, @@ -1758,7 +1759,7 @@ public TaskStatus run(TaskToolbox toolbox) }; } - private TaskInfo, TaskStatus> createTaskInfo( + private TaskInfo createTaskInfo( String taskId, String datasource ) @@ -1766,7 +1767,7 @@ private TaskInfo, TaskStatus> createTaskInfo( return createTaskInfo(taskId, datasource, TaskState.SUCCESS, "test"); } - private TaskInfo, TaskStatus> createTaskInfo( + private TaskInfo createTaskInfo( String taskId, String datasource, TaskState state, @@ -1778,7 +1779,26 @@ private TaskInfo, TaskStatus> createTaskInfo( DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.fromCode(taskId, state), datasource, - ImmutableMap.of("type", taskType) + getTaskWithIdAndDatasource(taskId, datasource, taskType) + ); + } + + private TaskStatusPlus toTaskStatusPlus(TaskInfo taskInfo) + { + Task task = taskInfo.getTask(); + TaskStatus status = taskInfo.getStatus(); + return new TaskStatusPlus( + taskInfo.getId(), + task == null ? null : task.getGroupId(), + task == null ? null : task.getType(), + taskInfo.getCreatedTime(), + DateTimes.EPOCH, + status.getStatusCode(), + status.getStatusCode().isComplete() ? RunnerTaskState.NONE : RunnerTaskState.WAITING, + status.getDuration(), + status.getLocation(), + taskInfo.getDataSource(), + status.getErrorMsg() ); } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index e9ae07fc258a..56e4fa7e714f 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -25,9 +25,12 @@ import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.base.Optional; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskInfo; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; @@ -74,8 +77,7 @@ public abstract class SQLMetadataStorageActionHandler taskInfoMapper; - private final TaskSummaryMapper taskStatusMapper; - private final TaskPayloadMapper taskPayloadMapper; + private final TaskStatusPlusMapper taskStatusPlusMapper; @SuppressWarnings("PMD.UnnecessaryFullyQualifiedName") public SQLMetadataStorageActionHandler( @@ -102,8 +104,7 @@ public SQLMetadataStorageActionHandler( this.logTable = logTable; this.lockTable = lockTable; this.taskInfoMapper = new TaskInfoMapper<>(jsonMapper, entryType, statusType); - this.taskStatusMapper = new TaskSummaryMapper<>(jsonMapper, statusType); - this.taskPayloadMapper = new TaskPayloadMapper<>(jsonMapper, statusType); + this.taskStatusPlusMapper = new TaskStatusPlusMapper(jsonMapper); } protected SQLMetadataConnector getConnector() @@ -294,7 +295,7 @@ public List> getTaskInfos( final Query> query; switch (entry.getKey()) { case ACTIVE: - query = createActiveTaskInfoStreamingQuery( + query = createActiveTaskStreamingQuery( handle, dataSource ); @@ -302,7 +303,7 @@ public List> getTaskInfos( break; case COMPLETE: CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); - query = createCompletedTaskInfoStreamingQuery( + query = createCompletedTaskStreamingQuery( handle, completeTaskLookup.getTasksCreatedPriorTo(), completeTaskLookup.getMaxTaskStatuses(), @@ -322,96 +323,54 @@ public List> getTaskInfos( } @Override - public List, StatusType>> getTaskSummaryList( + public List getTaskStatusPlusList( Map taskLookups, @Nullable String dataSource, boolean taskMigrationComplete ) { - if (taskMigrationComplete) { - getTaskSummaryList(taskLookups, dataSource); - } - return getTaskSummaryListFromPayload(taskLookups, dataSource); - } - - public List, StatusType>> getTaskSummaryList( - Map taskLookups, - @Nullable String dataSource - ) - { - return getConnector().retryTransaction( - (handle, status) -> { - final List, StatusType>> tasks = new ArrayList<>(); - for (Entry entry : taskLookups.entrySet()) { - final Query> query; - switch (entry.getKey()) { - case ACTIVE: - query = createActiveTaskSummaryStreamingQuery( - handle, - dataSource - ); - tasks.addAll(query.map(taskStatusMapper).list()); - break; - case COMPLETE: - CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); - query = createCompletedTaskSummaryStreamingQuery( - handle, - completeTaskLookup.getTasksCreatedPriorTo(), - completeTaskLookup.getMaxTaskStatuses(), - dataSource - ); - tasks.addAll(query.map(taskStatusMapper).list()); - break; - default: - throw new IAE("Unknown TaskLookupType: [%s]", entry.getKey()); - } - } - return tasks; - }, - 3, - SQLMetadataConnector.DEFAULT_MAX_TRIES - ); - } - - public List, StatusType>> getTaskSummaryListFromPayload( - Map taskLookups, - @Nullable String dataSource - ) - { + taskStatusPlusMapper.setUsePayload(!taskMigrationComplete); return getConnector().retryTransaction( (handle, status) -> { - final List, StatusType>> tasks = new ArrayList<>(); + final List taskStatusPlusList = new ArrayList<>(); for (Entry entry : taskLookups.entrySet()) { final Query> query; switch (entry.getKey()) { case ACTIVE: - query = createActiveTaskInfoStreamingQuery( - handle, - dataSource - ); - tasks.addAll(query.map(taskPayloadMapper).list()); + query = taskMigrationComplete + ? createActiveTaskSummaryStreamingQuery(handle, dataSource) + : createActiveTaskStreamingQuery(handle, dataSource); + taskStatusPlusList.addAll(query.map(taskStatusPlusMapper).list()); break; case COMPLETE: CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); - query = createCompletedTaskInfoStreamingQuery( - handle, - completeTaskLookup.getTasksCreatedPriorTo(), - completeTaskLookup.getMaxTaskStatuses(), - dataSource - ); - tasks.addAll(query.map(taskPayloadMapper).list()); + DateTime priorTo = completeTaskLookup.getTasksCreatedPriorTo(); + Integer limit = completeTaskLookup.getMaxTaskStatuses(); + query = taskMigrationComplete + ? createCompletedTaskSummaryStreamingQuery(handle, priorTo, limit, dataSource) + : createCompletedTaskStreamingQuery(handle, priorTo, limit, dataSource); + taskStatusPlusList.addAll(query.map(taskStatusPlusMapper).list()); break; default: throw new IAE("Unknown TaskLookupType: [%s]", entry.getKey()); } } - return tasks; + return taskStatusPlusList; }, 3, SQLMetadataConnector.DEFAULT_MAX_TRIES ); } + /** + * Fetches the columns needed to build TaskStatusPlus for completed tasks + * Please note that this requires completion of data migration to avoid empty values for task type and groupId + * Recommended for GET /tasks API + * Uses streaming SQL query to avoid fetching too many rows at once into memory + * @param handle db handle + * @param dataSource datasource to which the tasks belong. null if we don't want to filter + * @return Query object for TaskStatusPlus for completed tasks of interest + */ protected Query> createCompletedTaskSummaryStreamingQuery( Handle handle, DateTime timestamp, @@ -451,7 +410,16 @@ protected Query> createCompletedTaskSummaryStreamingQuery( return query; } - protected Query> createCompletedTaskInfoStreamingQuery( + /** + * Fetches the columns needed to build a Task object with payload for completed tasks + * This requires the task payload which can be large. Please use only when necessary. + * For example for ingestion tasks view before migration of the new columns + * Uses streaming SQL query to avoid fetching too many rows at once into memory + * @param handle db handle + * @param dataSource datasource to which the tasks belong. null if we don't want to filter + * @return Query object for completed TaskInfos of interest + */ + protected Query> createCompletedTaskStreamingQuery( Handle handle, DateTime timestamp, @Nullable Integer maxNumStatuses, @@ -500,6 +468,15 @@ private String getWhereClauseForInactiveStatusesSinceQuery(@Nullable String data return sql; } + /** + * Fetches the columns needed to build TaskStatusPlus for active tasks + * Please note that this requires completion of data migration to avoid empty values for task type and groupId + * Recommended for GET /tasks API + * Uses streaming SQL query to avoid fetching too many rows at once into memory + * @param handle db handle + * @param dataSource datasource to which the tasks belong. null if we don't want to filter + * @return Query object for TaskStatusPlus for active tasks of interest + */ private Query> createActiveTaskSummaryStreamingQuery(Handle handle, @Nullable String dataSource) { String sql = StringUtils.format( @@ -526,7 +503,16 @@ private Query> createActiveTaskSummaryStreamingQuery(Handle return query; } - private Query> createActiveTaskInfoStreamingQuery(Handle handle, @Nullable String dataSource) + /** + * Fetches the columns needed to build Task objects with payload for active tasks + * This requires the task payload which can be large. Please use only when necessary. + * For example for ingestion tasks view before migration of the new columns + * Uses streaming SQL query to avoid fetching too many rows at once into memory + * @param handle db handle + * @param dataSource datasource to which the tasks belong. null if we don't want to filter + * @return Query object for active TaskInfos of interest + */ + private Query> createActiveTaskStreamingQuery(Handle handle, @Nullable String dataSource) { String sql = StringUtils.format( "SELECT " @@ -560,98 +546,72 @@ private String getWhereClauseForActiveStatusesQuery(String dataSource) return sql; } - static class TaskSummaryMapper implements ResultSetMapper, StatusType>> + static class TaskStatusPlusMapper implements ResultSetMapper { private final ObjectMapper objectMapper; - private final TypeReference statusType; + private boolean usePayload; - TaskSummaryMapper(ObjectMapper objectMapper, TypeReference statusType) + TaskStatusPlusMapper(ObjectMapper objectMapper) { this.objectMapper = objectMapper; - this.statusType = statusType; - } - - @Override - public TaskInfo, StatusType> map(int index, ResultSet resultSet, StatementContext context) - throws SQLException - { - final TaskInfo, StatusType> taskInfo; - StatusType status; - try { - status = objectMapper.readValue(resultSet.getBytes("status_payload"), statusType); - } - catch (IOException e) { - log.error(e, "Encountered exception while deserializing task status_payload"); - throw new SQLException(e); - } - ImmutableMap.Builder task = ImmutableMap.builder(); - String groupId = resultSet.getString("group_id"); - if (groupId != null) { - task.put("groupId", groupId); - } - String type = resultSet.getString("type"); - if (type != null) { - task.put("type", type); - } - taskInfo = new TaskInfo<>( - resultSet.getString("id"), - DateTimes.of(resultSet.getString("created_date")), - status, - resultSet.getString("datasource"), - task.build() - ); - return taskInfo; + this.usePayload = true; } - } - - static class TaskPayloadMapper implements ResultSetMapper, StatusType>> - { - private final ObjectMapper objectMapper; - private final TypeReference statusType; - TaskPayloadMapper(ObjectMapper objectMapper, TypeReference statusType) + void setUsePayload(boolean usePayload) { - this.objectMapper = objectMapper; - this.statusType = statusType; + this.usePayload = usePayload; } - @Override - public TaskInfo, StatusType> map(int index, ResultSet resultSet, StatementContext context) + public TaskStatusPlus map(int index, ResultSet resultSet, StatementContext context) throws SQLException { - final TaskInfo, StatusType> taskInfo; - final ImmutableMap.Builder task = ImmutableMap.builder(); - try { - ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); - String groupId = payload.get("groupId").asText(); - if (groupId != null) { - task.put("groupId", groupId); + String type; + String groupId; + if (usePayload) { + try { + ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); + type = payload.get("type").asText(); + groupId = payload.get("groupId").asText(); } - String type = payload.get("type").asText(); - if (type != null) { - task.put("type", type); + catch (IOException e) { + log.error(e, "Encountered exception while deserializing task payload"); + throw new SQLException(e); } + } else { + type = resultSet.getString("type"); + groupId = resultSet.getString("group_id"); } - catch (IOException e) { - log.warn("Encountered exception[%s] while deserializing task payload", e.getMessage()); - } - StatusType status; + + TaskState statusCode; + Long duration; + TaskLocation location; + String errorMsg; try { - status = objectMapper.readValue(resultSet.getBytes("status_payload"), statusType); + ObjectNode status = objectMapper.readValue(resultSet.getBytes("status_payload"), ObjectNode.class); + statusCode = objectMapper.convertValue(status.get("status"), TaskState.class); + duration = status.get("duration").asLong(); + location = objectMapper.convertValue(status.get("location"), TaskLocation.class); + errorMsg = status.get("errorMsg").asText(); } catch (IOException e) { log.error(e, "Encountered exception while deserializing task status_payload"); throw new SQLException(e); } - taskInfo = new TaskInfo<>( + + return new TaskStatusPlus( resultSet.getString("id"), + groupId, + type, DateTimes.of(resultSet.getString("created_date")), - status, + DateTimes.EPOCH, + statusCode, + statusCode.isComplete() ? RunnerTaskState.NONE : RunnerTaskState.WAITING, + duration, + location, resultSet.getString("datasource"), - task.build() + errorMsg ); - return taskInfo; } } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java index 0f14aca7fdb2..7ab6cac4f7ef 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -33,6 +33,7 @@ import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.tweak.HandleCallback; +import java.nio.charset.StandardCharsets; import java.sql.Blob; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -310,7 +311,7 @@ public Void withHandle(Handle handle) throws SQLException { String json = StringUtils.format("{\"type\":\"%1$s\",\"groupId\":\"%2$s\"}", type, groupId); Blob payload = handle.getConnection().createBlob(); - payload.setBytes(1, json.getBytes()); + payload.setBytes(1, json.getBytes(StandardCharsets.UTF_8)); String sql = StringUtils.format( "INSERT INTO %1$s (id, created_date, datasource, active, payload, status_payload) VALUES (?,?,?,?,?,?)", From 71b2a4a88232cae83993f961b010a81edf83ca8e Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 20 Apr 2022 21:43:09 +0530 Subject: [PATCH 07/16] Add test --- .../overlord/http/OverlordResource.java | 16 +- .../druid/metadata/SQLMetadataConnector.java | 5 + .../SQLMetadataStorageActionHandler.java | 8 +- .../SQLMetadataStorageActionHandlerTest.java | 258 ++++++++++++++---- 4 files changed, 217 insertions(+), 70 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index aa26d3cd8f50..a111ebe787ad 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -769,21 +769,7 @@ private List getTaskStatusPlusList( if (runnerWorkItem == null) { // a task is assumed to be a waiting task if it exists in taskStorage but not in taskRunner. if (state == TaskStateLookup.WAITING || state == TaskStateLookup.ALL) { - taskStatuses.add( - new TaskStatusPlus( - statusPlus.getId(), - statusPlus.getGroupId(), - statusPlus.getType(), - statusPlus.getCreatedTime(), - statusPlus.getQueueInsertionTime(), - statusPlus.getStatusCode(), - RunnerTaskState.WAITING, - statusPlus.getDuration(), - statusPlus.getLocation(), - statusPlus.getDataSource(), - statusPlus.getErrorMsg() - ) - ); + taskStatuses.add(statusPlus); } } else { if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING || state == TaskStateLookup.ALL) { diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index cb8b26bb492f..a435fd36207e 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -417,6 +417,11 @@ public boolean migrateTaskTable() final MetadataStorageTablesConfig tablesConfig = tablesConfigSupplier.get(); final String entryType = tablesConfig.getTaskEntryType(); final String tableName = tablesConfig.getEntryTable(entryType); + return migrateTaskTable(tableName); + } + + public boolean migrateTaskTable(String tableName) + { log.info("Populate fields task and group_id of task entry table [%s] from payload", tableName); try { retryWithHandle( diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index 56e4fa7e714f..8fab6518890a 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -326,10 +326,10 @@ public List> getTaskInfos( public List getTaskStatusPlusList( Map taskLookups, @Nullable String dataSource, - boolean taskMigrationComplete + boolean fetchPayload ) { - taskStatusPlusMapper.setUsePayload(!taskMigrationComplete); + taskStatusPlusMapper.setUsePayload(fetchPayload); return getConnector().retryTransaction( (handle, status) -> { final List taskStatusPlusList = new ArrayList<>(); @@ -337,7 +337,7 @@ public List getTaskStatusPlusList( final Query> query; switch (entry.getKey()) { case ACTIVE: - query = taskMigrationComplete + query = !fetchPayload ? createActiveTaskSummaryStreamingQuery(handle, dataSource) : createActiveTaskStreamingQuery(handle, dataSource); taskStatusPlusList.addAll(query.map(taskStatusPlusMapper).list()); @@ -346,7 +346,7 @@ public List getTaskStatusPlusList( CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); DateTime priorTo = completeTaskLookup.getTasksCreatedPriorTo(); Integer limit = completeTaskLookup.getMaxTaskStatuses(); - query = taskMigrationComplete + query = !fetchPayload ? createCompletedTaskSummaryStreamingQuery(handle, priorTo, limit, dataSource) : createCompletedTaskStreamingQuery(handle, priorTo, limit, dataSource); taskStatusPlusList.addAll(query.map(taskStatusPlusMapper).list()); diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index fe4b03fa8a00..473068ce327c 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -25,7 +25,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskInfo; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; @@ -33,15 +37,21 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.Duration; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Random; +import java.util.UUID; import java.util.stream.Collectors; public class SQLMetadataStorageActionHandlerTest @@ -53,7 +63,9 @@ public class SQLMetadataStorageActionHandlerTest public final ExpectedException thrown = ExpectedException.none(); private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); - private SQLMetadataStorageActionHandler, Map, Map, Map> handler; + private SQLMetadataStorageActionHandler, Map, Map, Map> handler; + + final String entryTable = "entries"; @Before public void setUp() @@ -61,7 +73,6 @@ public void setUp() TestDerbyConnector connector = derbyConnectorRule.getConnector(); final String entryType = "entry"; - final String entryTable = "entries"; final String logTable = "logs"; final String lockTable = "locks"; @@ -72,20 +83,20 @@ public void setUp() handler = new DerbyMetadataStorageActionHandler<>( connector, JSON_MAPPER, - new MetadataStorageActionHandlerTypes, Map, Map, Map>() + new MetadataStorageActionHandlerTypes, Map, Map, Map>() { @Override - public TypeReference> getEntryType() + public TypeReference> getEntryType() { - return new TypeReference>() + return new TypeReference>() { }; } @Override - public TypeReference> getStatusType() + public TypeReference> getStatusType() { - return new TypeReference>() + return new TypeReference>() { }; } @@ -97,9 +108,9 @@ public TypeReference> getLogType() } @Override - public TypeReference> getLockType() + public TypeReference> getLockType() { - return new TypeReference>() + return new TypeReference>() { }; } @@ -114,9 +125,9 @@ public TypeReference> getLockType() @Test public void testEntryAndStatus() throws Exception { - Map entry = ImmutableMap.of("numericId", 1234); - Map status1 = ImmutableMap.of("count", 42); - Map status2 = ImmutableMap.of("count", 42, "temp", 1); + Map entry = ImmutableMap.of("numericId", 1234); + Map status1 = ImmutableMap.of("count", 42); + Map status2 = ImmutableMap.of("count", 42, "temp", 1); final String entryId = "1234"; @@ -195,13 +206,13 @@ 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); + 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, "type", "group"); } - final List, Map>> statuses = handler.getTaskInfos( + final List, Map>> statuses = handler.getTaskInfos( CompleteTaskLookup.withTasksCreatedPriorTo( 7, DateTimes.of("2014-01-01") @@ -210,7 +221,7 @@ public void testGetRecentStatuses() throws EntryExistsException ); Assert.assertEquals(7, statuses.size()); int i = 10; - for (TaskInfo, Map> status : statuses) { + for (TaskInfo, Map> status : statuses) { Assert.assertEquals(ImmutableMap.of("count", i-- * 10), status.getStatus()); } } @@ -220,13 +231,13 @@ 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); + 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, "type", "group"); } - final List, Map>> statuses = handler.getTaskInfos( + final List, Map>> statuses = handler.getTaskInfos( CompleteTaskLookup.withTasksCreatedPriorTo( 10, DateTimes.of("2014-01-01") @@ -235,7 +246,7 @@ public void testGetRecentStatuses2() throws EntryExistsException ); Assert.assertEquals(5, statuses.size()); int i = 5; - for (TaskInfo, Map> status : statuses) { + for (TaskInfo, Map> status : statuses) { Assert.assertEquals(ImmutableMap.of("count", i-- * 10), status.getStatus()); } } @@ -244,8 +255,8 @@ public void testGetRecentStatuses2() throws EntryExistsException public void testRepeatInsert() throws Exception { final String entryId = "abcd"; - Map entry = ImmutableMap.of("a", 1); - Map status = ImmutableMap.of("count", 42); + Map entry = ImmutableMap.of("a", 1); + Map status = ImmutableMap.of("count", 42); handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); @@ -257,8 +268,8 @@ public void testRepeatInsert() throws Exception public void testLogs() throws Exception { final String entryId = "abcd"; - Map entry = ImmutableMap.of("a", 1); - Map status = ImmutableMap.of("count", 42); + Map entry = ImmutableMap.of("a", 1); + Map status = ImmutableMap.of("count", 42); handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); @@ -289,32 +300,32 @@ public void testLogs() throws Exception public void testLocks() throws Exception { final String entryId = "ABC123"; - Map entry = ImmutableMap.of("a", 1); - Map status = ImmutableMap.of("count", 42); + Map entry = ImmutableMap.of("a", 1); + Map status = ImmutableMap.of("count", 42); handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); Assert.assertEquals( - ImmutableMap.>of(), + ImmutableMap.>of(), handler.getLocks("non_exist_entry") ); Assert.assertEquals( - ImmutableMap.>of(), + ImmutableMap.>of(), handler.getLocks(entryId) ); - final ImmutableMap lock1 = ImmutableMap.of("lock", 1); - final ImmutableMap lock2 = ImmutableMap.of("lock", 2); + final ImmutableMap lock1 = ImmutableMap.of("lock", 1); + final ImmutableMap lock2 = ImmutableMap.of("lock", 2); Assert.assertTrue(handler.addLock(entryId, lock1)); Assert.assertTrue(handler.addLock(entryId, lock2)); - final Map> locks = handler.getLocks(entryId); + final Map> locks = handler.getLocks(entryId); Assert.assertEquals(2, locks.size()); Assert.assertEquals( - ImmutableSet.>of(lock1, lock2), + ImmutableSet.>of(lock1, lock2), new HashSet<>(locks.values()) ); @@ -322,7 +333,7 @@ public void testLocks() throws Exception handler.removeLock(lockId); locks.remove(lockId); - final Map> updated = handler.getLocks(entryId); + final Map> updated = handler.getLocks(entryId); Assert.assertEquals( new HashSet<>(locks.values()), new HashSet<>(updated.values()) @@ -334,23 +345,23 @@ public void testLocks() throws Exception public void testReplaceLock() throws EntryExistsException { final String entryId = "ABC123"; - Map entry = ImmutableMap.of("a", 1); - Map status = ImmutableMap.of("count", 42); + Map entry = ImmutableMap.of("a", 1); + Map status = ImmutableMap.of("count", 42); handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); Assert.assertEquals( - ImmutableMap.>of(), + ImmutableMap.>of(), handler.getLocks("non_exist_entry") ); Assert.assertEquals( - ImmutableMap.>of(), + ImmutableMap.>of(), handler.getLocks(entryId) ); - final ImmutableMap lock1 = ImmutableMap.of("lock", 1); - final ImmutableMap lock2 = ImmutableMap.of("lock", 2); + final ImmutableMap lock1 = ImmutableMap.of("lock", 1); + final ImmutableMap lock2 = ImmutableMap.of("lock", 2); Assert.assertTrue(handler.addLock(entryId, lock1)); @@ -364,23 +375,23 @@ public void testReplaceLock() throws EntryExistsException public void testGetLockId() throws EntryExistsException { final String entryId = "ABC123"; - Map entry = ImmutableMap.of("a", 1); - Map status = ImmutableMap.of("count", 42); + Map entry = ImmutableMap.of("a", 1); + Map status = ImmutableMap.of("count", 42); handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); Assert.assertEquals( - ImmutableMap.>of(), + ImmutableMap.>of(), handler.getLocks("non_exist_entry") ); Assert.assertEquals( - ImmutableMap.>of(), + ImmutableMap.>of(), handler.getLocks(entryId) ); - final ImmutableMap lock1 = ImmutableMap.of("lock", 1); - final ImmutableMap lock2 = ImmutableMap.of("lock", 2); + final ImmutableMap lock1 = ImmutableMap.of("lock", 1); + final ImmutableMap lock2 = ImmutableMap.of("lock", 2); Assert.assertTrue(handler.addLock(entryId, lock1)); @@ -392,20 +403,20 @@ public void testGetLockId() throws EntryExistsException public void testRemoveTasksOlderThan() throws Exception { final String entryId1 = "1234"; - Map entry1 = ImmutableMap.of("numericId", 1234); - Map status1 = ImmutableMap.of("count", 42, "temp", 1); + Map entry1 = ImmutableMap.of("numericId", 1234); + Map status1 = ImmutableMap.of("count", 42, "temp", 1); handler.insert(entryId1, DateTimes.of("2014-01-01T00:00:00.123"), "testDataSource", entry1, false, status1, "type", "group"); Assert.assertTrue(handler.addLog(entryId1, ImmutableMap.of("logentry", "created"))); final String entryId2 = "ABC123"; - Map entry2 = ImmutableMap.of("a", 1); - Map status2 = ImmutableMap.of("count", 42); + Map entry2 = ImmutableMap.of("a", 1); + Map status2 = ImmutableMap.of("count", 42); handler.insert(entryId2, DateTimes.of("2014-01-01T00:00:00.123"), "test", entry2, true, status2, "type", "group"); Assert.assertTrue(handler.addLog(entryId2, ImmutableMap.of("logentry", "created"))); final String entryId3 = "DEF5678"; - Map entry3 = ImmutableMap.of("numericId", 5678); - Map status3 = ImmutableMap.of("count", 21, "temp", 2); + Map entry3 = ImmutableMap.of("numericId", 5678); + Map status3 = ImmutableMap.of("count", 21, "temp", 2); handler.insert(entryId3, DateTimes.of("2014-01-02T12:00:00.123"), "testDataSource", entry3, false, status3, "type", "group"); Assert.assertTrue(handler.addLog(entryId3, ImmutableMap.of("logentry", "created"))); @@ -450,4 +461,149 @@ public void testRemoveTasksOlderThan() throws Exception Assert.assertEquals(1, handler.getLogs(entryId2).size()); Assert.assertEquals(1, handler.getLogs(entryId3).size()); } + + @Test + public void testGetTaskStatusPlusList() + { + // SETUP + TaskInfo, Map> activeUnaltered = getRandomTaskInfo(true); + insertTaskInfo(activeUnaltered, false); + + TaskInfo, Map> completedUnaltered = getRandomTaskInfo(false); + insertTaskInfo(completedUnaltered, false); + + TaskInfo, Map> activeAltered = getRandomTaskInfo(true); + insertTaskInfo(activeAltered, true); + + TaskInfo, Map> completedAltered = getRandomTaskInfo(false); + insertTaskInfo(completedAltered, true); + + Map taskLookups = new HashMap<>(); + taskLookups.put(TaskLookup.TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()); + taskLookups.put(TaskLookup.TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, Duration.millis(86400000))); + + List taskStatusPlusList; + + // BEFORE MIGRATION + + // Payload based fetch. task type and groupid will be populated + taskStatusPlusList = handler.getTaskStatusPlusList(taskLookups, null, true); + Assert.assertEquals(4, taskStatusPlusList.size()); + verify(completedUnaltered, taskStatusPlusList, false, false, true); + verify(completedAltered, taskStatusPlusList, false, true, false); + verify(activeUnaltered, taskStatusPlusList, true, false, false); + verify(activeAltered, taskStatusPlusList, true, true, false); + + // New columns based fetch before migration is complete. type and payload are null when altered = false + taskStatusPlusList = handler.getTaskStatusPlusList(taskLookups, null, false); + Assert.assertEquals(4, taskStatusPlusList.size()); + verify(completedUnaltered, taskStatusPlusList, false, false, true); + verify(completedAltered, taskStatusPlusList, false, true, true); + verify(activeUnaltered, taskStatusPlusList, true, false, true); + verify(activeAltered, taskStatusPlusList, true, true, true); + + // MIGRATION + derbyConnectorRule.getConnector().migrateTaskTable(entryTable); + + // Payload based fetch. task type and groupid will still be populated + taskStatusPlusList = handler.getTaskStatusPlusList(taskLookups, null, true); + Assert.assertEquals(4, taskStatusPlusList.size()); + verify(completedUnaltered, taskStatusPlusList, false, false, false); + verify(completedAltered, taskStatusPlusList, false, true, false); + verify(activeUnaltered, taskStatusPlusList, true, false, false); + verify(activeAltered, taskStatusPlusList, true, true, false); + + // New columns based fetch before migration is complete. + // type and payload are not null for completed task but are still null for active ones since they aren't migrated + // An active task will be eventually updated on its own due to insertion + taskStatusPlusList = handler.getTaskStatusPlusList(taskLookups, null, false); + Assert.assertEquals(4, taskStatusPlusList.size()); + verify(completedUnaltered, taskStatusPlusList, false, false, false); + verify(completedAltered, taskStatusPlusList, false, true, false); + verify(activeUnaltered, taskStatusPlusList, true, false, true); + verify(activeAltered, taskStatusPlusList, true, true, false); + } + + private TaskInfo, Map> getRandomTaskInfo(boolean active) + { + String id = UUID.randomUUID().toString(); + DateTime createdTime = DateTime.now(DateTimeZone.UTC); + String datasource = UUID.randomUUID().toString(); + String type = UUID.randomUUID().toString(); + String groupId = UUID.randomUUID().toString(); + + Map payload = new HashMap<>(); + payload.put("id", id); + payload.put("type", type); + payload.put("groupId", groupId); + + Map status = new HashMap<>(); + status.put("id", id); + status.put("status", active ? TaskState.RUNNING : TaskState.SUCCESS); + status.put("duration", (new Random()).nextLong()); + status.put("location", TaskLocation.create(UUID.randomUUID().toString(), 8080, 995)); + status.put("errorMsg", UUID.randomUUID().toString()); + + return new TaskInfo<>( + id, + createdTime, + status, + datasource, + payload + ); + } + + private void insertTaskInfo(TaskInfo, Map> taskInfo, + boolean altered) + { + try { + handler.insert( + taskInfo.getId(), + taskInfo.getCreatedTime(), + taskInfo.getDataSource(), + taskInfo.getTask(), + TaskState.RUNNING.equals(taskInfo.getStatus().get("status")), + taskInfo.getStatus(), + altered ? taskInfo.getTask().get("type").toString() : null, + altered ? taskInfo.getTask().get("groupId").toString() : null + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private void verify(TaskInfo, Map> taskInfo, List taskStatusPlusList, + boolean active, boolean altered, boolean nullNewColumns) + { + for (TaskStatusPlus taskStatusPlus : taskStatusPlusList) { + if (taskStatusPlus.getId().equals(taskInfo.getId())) { + verify(taskInfo, taskStatusPlus, active, altered, nullNewColumns); + } + return; + } + Assert.fail(); + } + + private void verify(TaskInfo, Map> taskInfo, TaskStatusPlus taskStatusPlus, + boolean active, boolean altered, boolean nullNewColumns) + { + Assert.assertEquals(taskInfo.getId(), taskStatusPlus.getId()); + Assert.assertEquals(taskInfo.getCreatedTime(), taskStatusPlus.getCreatedTime()); + Assert.assertEquals(taskInfo.getCreatedTime(), taskStatusPlus.getCreatedTime()); + Assert.assertEquals(DateTimes.EPOCH, taskStatusPlus.getQueueInsertionTime()); + Assert.assertEquals(active ? TaskState.RUNNING : TaskState.SUCCESS, taskStatusPlus.getStatusCode()); + Assert.assertEquals(!active ? RunnerTaskState.NONE : RunnerTaskState.WAITING, taskStatusPlus.getRunnerStatusCode()); + Assert.assertEquals(taskInfo.getStatus().get("duration"), taskStatusPlus.getDuration()); + Assert.assertEquals(taskInfo.getStatus().get("location"), taskStatusPlus.getLocation()); + Assert.assertEquals(taskInfo.getDataSource(), taskStatusPlus.getDataSource()); + Assert.assertEquals(taskInfo.getStatus().get("errorMsg"), taskStatusPlus.getErrorMsg()); + if (!altered && nullNewColumns) { + Assert.assertNull(taskStatusPlus.getType()); + Assert.assertNull(taskStatusPlus.getGroupId()); + } else { + Assert.assertEquals(taskInfo.getTask().get("type"), taskStatusPlus.getType()); + Assert.assertEquals(taskInfo.getTask().get("groupId"), taskStatusPlus.getGroupId()); + } + } } From 843a32d5d5603a96057f16f41969b0e2d2d92d15 Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 21 Apr 2022 08:18:26 +0530 Subject: [PATCH 08/16] Fix bug, refactor etc --- .../druid/metadata/MetadataStorageActionHandler.java | 12 +++++++----- .../storage/sqlserver/SQLServerConnector.java | 2 +- .../druid/metadata/storage/mysql/MySQLConnector.java | 2 +- .../storage/postgresql/PostgreSQLConnector.java | 2 +- .../druid/indexing/overlord/MetadataTaskStorage.java | 6 +++--- .../apache/druid/metadata/SQLMetadataConnector.java | 5 ++--- .../druid/metadata/storage/derby/DerbyConnector.java | 2 +- .../druid/metadata/SQLMetadataConnectorTest.java | 2 +- .../SQLMetadataStorageActionHandlerTest.java | 6 +++++- 9 files changed, 22 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index 0d8164a95622..f25cedfbccc9 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -111,18 +111,20 @@ List> getTaskInfos( * store. For complete tasks, additional filters in {@code CompleteTaskLookup} can be applied. * All lookups should be processed atomically if there are more than one lookup is given. * - * taskMigrationComplete indicates if migration was completed before the time of calling. - * If yes, use the newly created type and group_id columns in the query for task summaries - * Else, fetch the payload and deserialize it to obtain the above fields + * It is RECOMMENDED to set fetchPayload to false after task table migration of old data to new schmea has completed + * + * fetchPayload determines the query used to fetch from the tasks table + * If true, fetch the payload and deserialize it to obtain the above fields + * Else, use the newly created type and group_id columns in the query for task summaries * * @param taskLookups task lookup type and filters. * @param datasource datasource filter - * @param taskMigrationComplete indicates which query to use based on migration status + * @param fetchPayload indicates which query to use based on migration status */ List getTaskStatusPlusList( Map taskLookups, @Nullable String datasource, - boolean taskMigrationComplete + boolean fetchPayload ); default List> getTaskInfos( diff --git a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java index 94612fd91f92..523214502e33 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java @@ -214,7 +214,7 @@ public boolean tableExists(final Handle handle, final String tableName) } @Override - public String withFetchLimit(int limit) + public String limitClause(int limit) { return String.format(Locale.ENGLISH, "FETCH NEXT %d ROWS ONLY", limit); } diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java index 04f163049809..46e103312451 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java @@ -179,7 +179,7 @@ public int getStreamingFetchSize() } @Override - public String withFetchLimit(int limit) + public String limitClause(int limit) { return String.format(Locale.ENGLISH, "LIMIT %d", limit); } diff --git a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java index c873a89f4522..bdbf71bddc5a 100644 --- a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java +++ b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java @@ -146,7 +146,7 @@ public int getStreamingFetchSize() } @Override - public String withFetchLimit(int limit) + public String limitClause(int limit) { return String.format(Locale.ENGLISH, "LIMIT %d", limit); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 2fde517db0b2..b3551ab615a4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -250,17 +250,17 @@ public List getTaskStatusPlusList( ) { Map theTaskLookups = processTaskLookups(taskLookups); - boolean taskMigrationComplete = false; + boolean fetchPayload = true; if (taskMigrationCompleteFuture.isDone()) { try { - taskMigrationComplete = taskMigrationCompleteFuture.get(); + fetchPayload = !taskMigrationCompleteFuture.get(); } catch (Exception e) { log.info(e, "Exception getting task migration future"); } } return Collections.unmodifiableList( - handler.getTaskStatusPlusList(theTaskLookups, datasource, taskMigrationComplete) + handler.getTaskStatusPlusList(theTaskLookups, datasource, fetchPayload) ); } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index a435fd36207e..c41bee8a8126 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -136,7 +136,7 @@ public String getValidationQuery() public abstract boolean tableExists(Handle handle, String tableName); - public abstract String withFetchLimit(int limit); + public abstract String limitClause(int limit); public T retryWithHandle( final HandleCallback callback, @@ -434,13 +434,12 @@ public Void withHandle(Handle handle) throws SQLException, IOException Connection connection = handle.getConnection(); Statement statement = connection.createStatement(ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_UPDATABLE); boolean flag = true; - String limit = withFetchLimit(100); while (flag) { // Should ideally use a cursor and sort by id for efficiency, but updates with ordering aren't allowed String sql = StringUtils.format( "SELECT * FROM %1$s WHERE active = false AND type IS null %2$s", tableName, - limit + limitClause(100) ); ResultSet resultSet = statement.executeQuery(sql); flag = false; diff --git a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java index f5b9abf0fcc5..27038810cbe0 100644 --- a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java @@ -117,7 +117,7 @@ public String getValidationQuery() } @Override - public String withFetchLimit(int limit) + public String limitClause(int limit) { return String.format(Locale.ENGLISH, "FETCH NEXT %d ROWS ONLY", limit); } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java index 7ab6cac4f7ef..ed9ce91efa29 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -219,7 +219,7 @@ public int getStreamingFetchSize() } @Override - public String withFetchLimit(int limit) + public String limitClause(int limit) { return ""; } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 473068ce327c..2c4f271d12a3 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -63,6 +63,10 @@ public class SQLMetadataStorageActionHandlerTest public final ExpectedException thrown = ExpectedException.none(); private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); + + + private static final Random RANDOM = new Random(1); + private SQLMetadataStorageActionHandler, Map, Map, Map> handler; final String entryTable = "entries"; @@ -540,7 +544,7 @@ private TaskInfo, Map> getRandomTaskInfo(boo Map status = new HashMap<>(); status.put("id", id); status.put("status", active ? TaskState.RUNNING : TaskState.SUCCESS); - status.put("duration", (new Random()).nextLong()); + status.put("duration", RANDOM.nextLong()); status.put("location", TaskLocation.create(UUID.randomUUID().toString(), 8080, 995)); status.put("errorMsg", UUID.randomUUID().toString()); From 4807b13e429aaec407fa1d90639abd53df59a202 Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 21 Apr 2022 08:32:25 +0530 Subject: [PATCH 09/16] trivial tests for coverage --- .../sqlserver/SQLServerConnectorTest.java | 23 ++++++++++++++++ .../storage/mysql/MySQLConnectorTest.java | 12 +++++++++ .../postgresql/PostgreSQLConnectorTest.java | 26 +++++++++++++++++++ 3 files changed, 61 insertions(+) diff --git a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java index d061b99701de..aaaf9b10e1a2 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java @@ -64,4 +64,27 @@ public void testIsTransientException() Assert.assertFalse(connector.isTransientException(new Throwable("Throwable with reason only"))); } + @Test + public void testLimitClause() + { + SQLServerConnector connector = new SQLServerConnector( + Suppliers.ofInstance(new MetadataStorageConnectorConfig()), + Suppliers.ofInstance( + new MetadataStorageTablesConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + ) + ); + Assert.assertEquals("FETCH NEXT 100 ROWS ONLY", connector.limitClause(100)); + } } diff --git a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java index fceefe6e4931..23ce46282232 100644 --- a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java +++ b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java @@ -91,4 +91,16 @@ public void testIsExceptionTransientNoMySqlClazz() connector.connectorIsTransientException(new SQLTransientConnectionException("transient")) ); } + + @Test + public void testLimitClause() + { + MySQLConnector connector = new MySQLConnector( + CONNECTOR_CONFIG_SUPPLIER, + TABLES_CONFIG_SUPPLIER, + new MySQLConnectorSslConfig(), + MYSQL_DRIVER_CONFIG + ); + Assert.assertEquals("LIMIT 100", connector.limitClause(100)); + } } diff --git a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java index bb885d71b7ff..07deb3108f8a 100644 --- a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java +++ b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java @@ -62,4 +62,30 @@ public void testIsTransientException() Assert.assertFalse(connector.isTransientException(new Exception("I'm not happy"))); Assert.assertFalse(connector.isTransientException(new Throwable("I give up"))); } + + @Test + public void testLimitClause() + { + PostgreSQLConnector connector = new PostgreSQLConnector( + Suppliers.ofInstance(new MetadataStorageConnectorConfig()), + Suppliers.ofInstance( + new MetadataStorageTablesConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + ), + new PostgreSQLConnectorConfig(), + new PostgreSQLTablesConfig() + ); + Assert.assertEquals("LIMIT 100", connector.limitClause(100)); + } } From 082d0c31a37e161cd837ef28a8f7bf1332fbb523 Mon Sep 17 00:00:00 2001 From: Amatya Date: Mon, 25 Apr 2022 20:10:02 +0530 Subject: [PATCH 10/16] Possible fix for integration tests --- .../druid/metadata/SQLMetadataConnector.java | 12 +++++------ .../storage/derby/DerbyConnector.java | 21 +++++++++++++++++++ 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index c41bee8a8126..af33f3229a2e 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -56,7 +56,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Locale; import java.util.Properties; public abstract class SQLMetadataConnector implements MetadataStorageConnector @@ -344,8 +343,8 @@ public boolean tableContainsColumn(Handle handle, String table, String column) ResultSet columns = databaseMetaData.getColumns( null, null, - table.toUpperCase(Locale.ENGLISH), - column.toUpperCase(Locale.ENGLISH) + table, + column ); return columns.next(); } @@ -391,16 +390,14 @@ public void alterEntryTable(final String tableName) @Override public Void withHandle(Handle handle) { - final Batch batch = handle.createBatch(); if (!tableContainsColumn(handle, tableName, "type")) { log.info("Adding column: type to table[%s]", tableName); - batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN type VARCHAR(255)", tableName)); + handle.execute(StringUtils.format("ALTER TABLE %1$s ADD COLUMN type VARCHAR(255)", tableName)); } if (!tableContainsColumn(handle, tableName, "group_id")) { log.info("Adding column: group_id to table[%s]", tableName); - batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN group_id VARCHAR(255)", tableName)); + handle.execute(StringUtils.format("ALTER TABLE %1$s ADD COLUMN group_id VARCHAR(255)", tableName)); } - batch.execute(); return null; } } @@ -456,6 +453,7 @@ public Void withHandle(Handle handle) throws SQLException, IOException } } ); + log.info("Migration of tasks complete for table[%s]", tableName); return true; } catch (Exception e) { diff --git a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java index 27038810cbe0..19d0c6b04f16 100644 --- a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java @@ -35,6 +35,9 @@ import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.tweak.HandleCallback; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; import java.util.Locale; @ManageLifecycle @@ -85,6 +88,24 @@ public boolean tableExists(Handle handle, String tableName) .isEmpty(); } + @Override + public boolean tableContainsColumn(Handle handle, String table, String column) + { + try { + DatabaseMetaData databaseMetaData = handle.getConnection().getMetaData(); + ResultSet columns = databaseMetaData.getColumns( + null, + null, + table.toUpperCase(Locale.ENGLISH), + column.toUpperCase(Locale.ENGLISH) + ); + return columns.next(); + } + catch (SQLException e) { + return false; + } + } + @Override public String getSerialType() { From a5bafe75867b5dfa337809679dfa8c2d0d46dce1 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 28 Apr 2022 11:25:22 +0530 Subject: [PATCH 11/16] Cleanup OverlordResourceTest --- .../overlord/http/OverlordResourceTest.java | 488 +++--------------- 1 file changed, 84 insertions(+), 404 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java index 947dd85f8663..f01668fd37a2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java @@ -32,10 +32,6 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.ImmutableWorkerInfo; @@ -92,7 +88,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; public class OverlordResourceTest { @@ -238,35 +233,11 @@ public void testSecuredGetWaitingTask() ) ).andStubReturn( ImmutableList.of( - new TaskInfo<>( - "id_1", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_1"), - "allow", - getTaskWithIdAndDatasource("id_1", "allow") - ), - new TaskInfo<>( - "id_2", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_2"), - "allow", - getTaskWithIdAndDatasource("id_2", "allow") - ), - new TaskInfo<>( - "id_3", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_3"), - "deny", - getTaskWithIdAndDatasource("id_3", "deny") - ), - new TaskInfo<>( - "id_4", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_4"), - "deny", - getTaskWithIdAndDatasource("id_4", "deny") - ) - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_1", TaskState.RUNNING, "allow"), + createTaskStatusPlus("id_2", TaskState.RUNNING, "allow"), + createTaskStatusPlus("id_3", TaskState.RUNNING, "deny"), + createTaskStatusPlus("id_4", TaskState.RUNNING, "deny") + ) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( @@ -302,28 +273,10 @@ public void testSecuredGetCompleteTasks() ImmutableMap.of(TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, (Duration) null)), null) ).andStubReturn( ImmutableList.of( - new TaskInfo<>( - "id_1", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_1"), - "deny", - getTaskWithIdAndDatasource("id_1", "deny") - ), - new TaskInfo<>( - "id_2", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_2"), - "allow", - getTaskWithIdAndDatasource("id_2", "allow") - ), - new TaskInfo<>( - "id_3", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_3"), - "allow", - getTaskWithIdAndDatasource("id_3", "allow") - ) - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_1", TaskState.SUCCESS, "deny"), + createTaskStatusPlus("id_2", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_3", TaskState.SUCCESS, "allow") + ) ); EasyMock.replay( taskRunner, @@ -359,21 +312,9 @@ public void testSecuredGetRunningTasks() ) ).andStubReturn( ImmutableList.of( - new TaskInfo<>( - "id_1", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_1"), - "deny", - getTaskWithIdAndDatasource("id_1", "deny") - ), - new TaskInfo<>( - "id_2", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_2"), - "allow", - getTaskWithIdAndDatasource("id_2", "allow") - ) - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_1", TaskState.RUNNING, "deny"), + createTaskStatusPlus("id_2", TaskState.RUNNING, "allow") + ) ); EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andStubReturn(RunnerTaskState.RUNNING); EasyMock.expect(taskRunner.getRunnerTaskState("id_2")).andStubReturn(RunnerTaskState.RUNNING); @@ -410,49 +351,13 @@ public void testGetTasks() ) ).andStubReturn( ImmutableList.of( - new TaskInfo<>( - "id_5", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_5"), - "deny", - getTaskWithIdAndDatasource("id_5", "deny") - ), - new TaskInfo<>( - "id_6", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_6"), - "allow", - getTaskWithIdAndDatasource("id_6", "allow") - ), - new TaskInfo<>( - "id_7", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_7"), - "allow", - getTaskWithIdAndDatasource("id_7", "allow") - ), - new TaskInfo<>( - "id_5", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_5"), - "deny", - getTaskWithIdAndDatasource("id_5", "deny") - ), - new TaskInfo<>( - "id_6", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_6"), - "allow", - getTaskWithIdAndDatasource("id_6", "allow") - ), - new TaskInfo<>( - "id_7", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_7"), - "allow", - getTaskWithIdAndDatasource("id_7", "allow") - ) - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_5", TaskState.SUCCESS, "deny"), + createTaskStatusPlus("id_6", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_7", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_5", TaskState.SUCCESS, "deny"), + createTaskStatusPlus("id_6", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_7", TaskState.SUCCESS, "allow") + ) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( @@ -493,56 +398,14 @@ public void testGetTasksFilterDataSource() ) ).andStubReturn( ImmutableList.of( - new TaskInfo<>( - "id_5", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_5"), - "allow", - getTaskWithIdAndDatasource("id_5", "allow") - ), - new TaskInfo<>( - "id_6", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_6"), - "allow", - getTaskWithIdAndDatasource("id_6", "allow") - ), - new TaskInfo<>( - "id_7", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_7"), - "allow", - getTaskWithIdAndDatasource("id_7", "allow") - ), - new TaskInfo<>( - "id_1", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_1"), - "allow", - getTaskWithIdAndDatasource("id_1", "allow") - ), - new TaskInfo<>( - "id_2", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_2"), - "allow", - getTaskWithIdAndDatasource("id_2", "allow") - ), - new TaskInfo<>( - "id_3", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_3"), - "allow", - getTaskWithIdAndDatasource("id_3", "allow") - ), - new TaskInfo<>( - "id_4", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_4"), - "allow", - getTaskWithIdAndDatasource("id_4", "allow") - ) - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_5", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_6", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_7", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_1", TaskState.RUNNING, "allow"), + createTaskStatusPlus("id_2", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_3", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_4", TaskState.SUCCESS, "allow") + ) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( ImmutableList.of( @@ -582,35 +445,11 @@ public void testGetTasksFilterWaitingState() ) ).andStubReturn( ImmutableList.of( - new TaskInfo<>( - "id_1", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_1"), - "allow", - getTaskWithIdAndDatasource("id_1", "allow") - ), - new TaskInfo<>( - "id_2", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_2"), - "allow", - getTaskWithIdAndDatasource("id_2", "allow") - ), - new TaskInfo<>( - "id_3", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_3"), - "deny", - getTaskWithIdAndDatasource("id_3", "deny") - ), - new TaskInfo<>( - "id_4", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_4"), - "deny", - getTaskWithIdAndDatasource("id_4", "deny") - ) - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_1", TaskState.RUNNING, "allow"), + createTaskStatusPlus("id_2", TaskState.RUNNING, "allow"), + createTaskStatusPlus("id_3", TaskState.RUNNING, "deny"), + createTaskStatusPlus("id_4", TaskState.RUNNING, "deny") + ) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( @@ -655,35 +494,11 @@ public void testGetTasksFilterRunningState() ) ).andStubReturn( ImmutableList.of( - new TaskInfo<>( - "id_1", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_1"), - "allow", - getTaskWithIdAndDatasource("id_1", "allow") - ), - new TaskInfo<>( - "id_2", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_2"), - "allow", - getTaskWithIdAndDatasource("id_2", "allow") - ), - new TaskInfo<>( - "id_3", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_3"), - "allow", - getTaskWithIdAndDatasource("id_3", "allow") - ), - new TaskInfo<>( - "id_4", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_4"), - "deny", - getTaskWithIdAndDatasource("id_4", "deny") - ) - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_1", TaskState.RUNNING, "allow"), + createTaskStatusPlus("id_2", TaskState.RUNNING, "allow"), + createTaskStatusPlus("id_3", TaskState.RUNNING, "allow"), + createTaskStatusPlus("id_4", TaskState.RUNNING, "deny") + ) ); List tasksIds = ImmutableList.of("id_1", "id_2"); @@ -733,35 +548,11 @@ public void testGetTasksFilterPendingState() ) ).andStubReturn( ImmutableList.of( - new TaskInfo<>( - "id_1", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_1"), - "deny", - getTaskWithIdAndDatasource("id_1", "deny") - ), - new TaskInfo<>( - "id_2", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_2"), - "allow", - getTaskWithIdAndDatasource("id_2", "allow") - ), - new TaskInfo<>( - "id_3", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_3"), - "allow", - getTaskWithIdAndDatasource("id_3", "allow") - ), - new TaskInfo<>( - "id_4", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.running("id_4"), - "deny", - getTaskWithIdAndDatasource("id_4", "deny") - ) - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_1", TaskState.RUNNING, "deny"), + createTaskStatusPlus("id_2", TaskState.RUNNING, "allow"), + createTaskStatusPlus("id_3", TaskState.RUNNING, "allow"), + createTaskStatusPlus("id_4", TaskState.RUNNING, "deny") + ) ); EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andStubReturn(RunnerTaskState.PENDING); @@ -798,28 +589,10 @@ public void testGetTasksFilterCompleteState() ) ).andStubReturn( ImmutableList.of( - new TaskInfo<>( - "id_1", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_1"), - "allow", - getTaskWithIdAndDatasource("id_1", "allow") - ), - new TaskInfo<>( - "id_2", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_2"), - "deny", - getTaskWithIdAndDatasource("id_2", "deny") - ), - new TaskInfo<>( - "id_3", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_3"), - "allow", - getTaskWithIdAndDatasource("id_3", "allow") - ) - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_1", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_2", TaskState.SUCCESS, "deny"), + createTaskStatusPlus("id_3", TaskState.SUCCESS, "allow") + ) ); EasyMock.replay( taskRunner, @@ -849,28 +622,10 @@ public void testGetTasksFilterCompleteStateWithInterval() ) ).andStubReturn( ImmutableList.of( - new TaskInfo<>( - "id_1", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_1"), - "deny", - getTaskWithIdAndDatasource("id_1", "deny") - ), - new TaskInfo<>( - "id_2", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_2"), - "allow", - getTaskWithIdAndDatasource("id_2", "allow") - ), - new TaskInfo<>( - "id_3", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_3"), - "allow", - getTaskWithIdAndDatasource("id_3", "allow") - ) - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_1", TaskState.SUCCESS, "deny"), + createTaskStatusPlus("id_2", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_3", TaskState.SUCCESS, "allow") + ) ); EasyMock.replay( @@ -887,7 +642,7 @@ public void testGetTasksFilterCompleteStateWithInterval() .getEntity(); Assert.assertEquals(2, responseObjects.size()); Assert.assertEquals("id_2", responseObjects.get(0).getId()); - Assert.assertTrue("DataSource Check", "allow".equals(responseObjects.get(0).getDataSource())); + Assert.assertEquals("DataSource Check", "allow", responseObjects.get(0).getDataSource()); } @Test @@ -910,11 +665,11 @@ public void testGetTasksRequiresDatasourceRead() ) ).andStubReturn( ImmutableList.of( - createTaskInfo("id_5", Datasources.WIKIPEDIA), - createTaskInfo("id_6", Datasources.BUZZFEED), - createTaskInfo("id_1", Datasources.WIKIPEDIA, TaskState.RUNNING, "test"), - createTaskInfo("id_4", Datasources.BUZZFEED, TaskState.RUNNING, "test") - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_5", TaskState.SUCCESS, Datasources.WIKIPEDIA), + createTaskStatusPlus("id_6", TaskState.SUCCESS, Datasources.BUZZFEED), + createTaskStatusPlus("id_1", TaskState.RUNNING, Datasources.WIKIPEDIA), + createTaskStatusPlus("id_4", TaskState.RUNNING, Datasources.BUZZFEED) + ) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( @@ -967,11 +722,11 @@ public void testGetTasksFilterByTaskTypeRequiresDatasourceRead() ) ).andStubReturn( ImmutableList.of( - createTaskInfo("id_5", Datasources.WIKIPEDIA), - createTaskInfo("id_6", Datasources.BUZZFEED), - createTaskInfo("id_1", Datasources.WIKIPEDIA, TaskState.RUNNING, "to-return"), - createTaskInfo("id_4", Datasources.WIKIPEDIA, TaskState.RUNNING, "test") - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_5", TaskState.SUCCESS, Datasources.WIKIPEDIA), + createTaskStatusPlus("id_6", TaskState.SUCCESS, Datasources.BUZZFEED), + createTaskStatusPlus("id_1", TaskState.RUNNING, Datasources.WIKIPEDIA, "to-return"), + createTaskStatusPlus("id_4", TaskState.RUNNING, Datasources.BUZZFEED) + ) ); EasyMock.>expect(taskRunner.getKnownTasks()).andReturn( @@ -1026,7 +781,7 @@ public void testGetTasksFilterByDatasourceRequiresReadAccess() } @Test - public void testGetNullCompleteTask() + public void testGetCompleteTasksOfAllDatasources() { expectAuthorizationTokenCheck(); EasyMock.expect( @@ -1039,28 +794,10 @@ public void testGetNullCompleteTask() ) ).andStubReturn( ImmutableList.of( - new TaskInfo( - "id_1", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_1"), - "allow", - null - ), - new TaskInfo<>( - "id_2", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_2"), - "deny", - getTaskWithIdAndDatasource("id_2", "deny") - ), - new TaskInfo<>( - "id_3", - DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.success("id_3"), - "allow", - getTaskWithIdAndDatasource("id_3", "allow") - ) - ).stream().map(this::toTaskStatusPlus).collect(Collectors.toList()) + createTaskStatusPlus("id_1", TaskState.SUCCESS, "allow"), + createTaskStatusPlus("id_2", TaskState.SUCCESS, "deny"), + createTaskStatusPlus("id_3", TaskState.SUCCESS, "allow") + ) ); EasyMock.replay( taskRunner, @@ -1075,9 +812,7 @@ public void testGetNullCompleteTask() .getEntity(); Assert.assertEquals(2, responseObjects.size()); Assert.assertEquals("id_1", responseObjects.get(0).getId()); - TaskStatusPlus tsp = responseObjects.get(0); - Assert.assertEquals(null, tsp.getType()); - Assert.assertTrue("DataSource Check", "allow".equals(responseObjects.get(0).getDataSource())); + Assert.assertEquals("DataSource Check", "allow", responseObjects.get(0).getDataSource()); } @Test @@ -1380,19 +1115,19 @@ public void testShutdownAllTasks() Optional.of(mockQueue) ).anyTimes(); EasyMock.expect(taskStorageQueryAdapter.getActiveTaskInfo("datasource")).andStubReturn(ImmutableList.of( - new TaskInfo( + new TaskInfo<>( "id_1", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_1"), "datasource", - getTaskWithIdAndDatasource("id_1", "datasource") + NoopTask.create("id_1", 1) ), - new TaskInfo( + new TaskInfo<>( "id_2", DateTime.now(ISOChronology.getInstanceUTC()), TaskStatus.success("id_2"), "datasource", - getTaskWithIdAndDatasource("id_2", "datasource") + NoopTask.create("id_2", 1) ) )); mockQueue.shutdown("id_1", "Shutdown request from user"); @@ -1680,7 +1415,7 @@ public void testGetTotalWorkerCapacityWithAutoScaleConfiguredAndProvisioningStra Optional.of(workerTaskRunner) ).anyTimes(); EasyMock.expect(provisioningStrategy.getExpectedWorkerCapacity(workerInfos)).andReturn(invalidExpectedCapacity).anyTimes(); - AutoScaler autoScaler = EasyMock.createMock(AutoScaler.class); + AutoScaler autoScaler = EasyMock.createMock(AutoScaler.class); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(maxNumWorkers); DefaultWorkerBehaviorConfig workerBehaviorConfig = new DefaultWorkerBehaviorConfig(null, autoScaler); @@ -1725,80 +1460,25 @@ private void expectAuthorizationTokenCheck(String username) EasyMock.expectLastCall().anyTimes(); } - private Task getTaskWithIdAndDatasource(String id, String datasource) - { - return getTaskWithIdAndDatasource(id, datasource, "test"); - } - - private Task getTaskWithIdAndDatasource(String id, String datasource, String taskType) - { - return new AbstractTask(id, datasource, null) - { - @Override - public String getType() - { - return taskType; - } - - @Override - public boolean isReady(TaskActionClient taskActionClient) - { - return false; - } - - @Override - public void stopGracefully(TaskConfig taskConfig) - { - } - - @Override - public TaskStatus run(TaskToolbox toolbox) - { - return null; - } - }; - } - - private TaskInfo createTaskInfo( - String taskId, - String datasource - ) + private TaskStatusPlus createTaskStatusPlus(String taskId, TaskState taskState, String datasource) { - return createTaskInfo(taskId, datasource, TaskState.SUCCESS, "test"); + return createTaskStatusPlus(taskId, taskState, datasource, "test"); } - private TaskInfo createTaskInfo( - String taskId, - String datasource, - TaskState state, - String taskType - ) + private TaskStatusPlus createTaskStatusPlus(String taskId, TaskState taskState, String datasource, String taskType) { - return new TaskInfo<>( + return new TaskStatusPlus( taskId, + null, + taskType, DateTime.now(ISOChronology.getInstanceUTC()), - TaskStatus.fromCode(taskId, state), - datasource, - getTaskWithIdAndDatasource(taskId, datasource, taskType) - ); - } - - private TaskStatusPlus toTaskStatusPlus(TaskInfo taskInfo) - { - Task task = taskInfo.getTask(); - TaskStatus status = taskInfo.getStatus(); - return new TaskStatusPlus( - taskInfo.getId(), - task == null ? null : task.getGroupId(), - task == null ? null : task.getType(), - taskInfo.getCreatedTime(), DateTimes.EPOCH, - status.getStatusCode(), - status.getStatusCode().isComplete() ? RunnerTaskState.NONE : RunnerTaskState.WAITING, - status.getDuration(), - status.getLocation(), - taskInfo.getDataSource(), - status.getErrorMsg() + taskState, + taskState.isComplete() ? RunnerTaskState.NONE : RunnerTaskState.WAITING, + 100L, + TaskLocation.unknown(), + datasource, + null ); } From e2b9e932c86fc56ef7222c4ed06b03321c0e5a38 Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 28 Apr 2022 15:31:52 +0530 Subject: [PATCH 12/16] Address review --- .../apache/druid/indexer/TaskMetadata.java | 110 ++++++++++++++++++ .../apache/druid/indexer/TaskStatusPlus.java | 27 +++++ .../MetadataStorageActionHandler.java | 11 +- .../metadata/MetadataStorageConnector.java | 2 +- .../sqlserver/SQLServerConnectorTest.java | 14 +-- .../postgresql/PostgreSQLConnectorTest.java | 14 +-- .../overlord/MetadataTaskStorage.java | 2 +- .../druid/metadata/SQLMetadataConnector.java | 60 +--------- .../SQLMetadataStorageActionHandler.java | 48 ++++++++ .../metadata/SQLMetadataConnectorTest.java | 77 ------------ .../SQLMetadataStorageActionHandlerTest.java | 4 +- 11 files changed, 207 insertions(+), 162 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/indexer/TaskMetadata.java diff --git a/core/src/main/java/org/apache/druid/indexer/TaskMetadata.java b/core/src/main/java/org/apache/druid/indexer/TaskMetadata.java new file mode 100644 index 000000000000..ef0d151ab3d1 --- /dev/null +++ b/core/src/main/java/org/apache/druid/indexer/TaskMetadata.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.indexer; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Model class containing the fields relevant to view tasks in the ingestion tab. + * These fields are extracted from the task payload for the new schema and this model can be used for migration as well. + */ +public class TaskMetadata +{ + + private final String id; + + @Nullable + private final String type; + + @Nullable + private final String groupId; + + @JsonCreator + public TaskMetadata( + @JsonProperty("id") String id, + @JsonProperty("groupId") @Nullable String groupId, + @JsonProperty("type") @Nullable String type // nullable for backward compatibility + ) + { + this.id = Preconditions.checkNotNull(id, "id"); + this.groupId = groupId; + this.type = type; + } + + @JsonProperty + public String getId() + { + return id; + } + + @Nullable + @JsonProperty + public String getGroupId() + { + return groupId; + } + + @Nullable + @JsonProperty + public String getType() + { + return type; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TaskMetadata that = (TaskMetadata) o; + return Objects.equals(getId(), that.getId()) && + Objects.equals(getGroupId(), that.getGroupId()) && + Objects.equals(getType(), that.getType()); + } + + @Override + public int hashCode() + { + return Objects.hash( + getId(), + getGroupId(), + getType() + ); + } + + @Override + public String toString() + { + return "TaskMetadata{" + + "id='" + id + '\'' + + ", groupId='" + groupId + '\'' + + ", type='" + type + '\'' + + '}'; + } +} diff --git a/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java b/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java index 75bbefa0eaf3..92875db23a2e 100644 --- a/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java +++ b/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.RE; import org.joda.time.DateTime; @@ -31,6 +32,7 @@ public class TaskStatusPlus { private final String id; + @Nullable private final String type; private final DateTime createdTime; private final DateTime queueInsertionTime; @@ -252,4 +254,29 @@ public String toString() ", errorMsg='" + errorMsg + '\'' + '}'; } + + /** + * Convert a TaskInfo pair of TaskMetadata and TaskStatus to a TaskStatusPlus + * Applicable only for completed or waiting tasks + * + * @param taskMetadataInfo TaskInfo pair + * @return corresponding TaskStatusPlus + */ + public static TaskStatusPlus fromTaskMetadataInfo(TaskInfo taskMetadataInfo) + { + TaskStatus status = taskMetadataInfo.getStatus(); + return new TaskStatusPlus( + taskMetadataInfo.getId(), + taskMetadataInfo.getTask().getGroupId(), + taskMetadataInfo.getTask().getType(), + taskMetadataInfo.getCreatedTime(), + DateTimes.EPOCH, + status.getStatusCode(), + status.getStatusCode().isComplete() ? RunnerTaskState.NONE : RunnerTaskState.WAITING, + status.getDuration(), + status.getLocation(), + taskMetadataInfo.getDataSource(), + status.getErrorMsg() + ); + } } diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index f25cedfbccc9..81c60ec9e520 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -109,7 +109,7 @@ List> getTaskInfos( * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata * store. For complete tasks, additional filters in {@code CompleteTaskLookup} can be applied. - * All lookups should be processed atomically if there are more than one lookup is given. + * All lookups should be processed atomically if more than one lookup is given. * * It is RECOMMENDED to set fetchPayload to false after task table migration of old data to new schmea has completed * @@ -201,4 +201,13 @@ default List> getTaskInfos( */ @Nullable Long getLockId(String entryId, LockType lock); + + /** + * Utility to migrate existing tasks to the new schema + * + * To be kicked off in a separate thread at MetadataTaskStorage startup. + * @param tasksTable + * @return + */ + boolean migrateTaskTable(String tasksTable); } diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java index d50d69d4937c..9211130b4041 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -87,7 +87,7 @@ default void exportTable( void createSupervisorsTable(); - boolean migrateTaskTable(); + String getTaskTableName(); void deleteAllRecords(String tableName); } diff --git a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java index aaaf9b10e1a2..a31159e88134 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java @@ -70,19 +70,7 @@ public void testLimitClause() SQLServerConnector connector = new SQLServerConnector( Suppliers.ofInstance(new MetadataStorageConnectorConfig()), Suppliers.ofInstance( - new MetadataStorageTablesConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + new MetadataStorageTablesConfig(null, null, null, null, null, null, null, null, null, null, null) ) ); Assert.assertEquals("FETCH NEXT 100 ROWS ONLY", connector.limitClause(100)); diff --git a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java index 07deb3108f8a..08f3c333a1fb 100644 --- a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java +++ b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java @@ -69,19 +69,7 @@ public void testLimitClause() PostgreSQLConnector connector = new PostgreSQLConnector( Suppliers.ofInstance(new MetadataStorageConnectorConfig()), Suppliers.ofInstance( - new MetadataStorageTablesConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + new MetadataStorageTablesConfig(null, null, null, null, null, null, null, null, null, null, null) ), new PostgreSQLConnectorConfig(), new PostgreSQLTablesConfig() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index b3551ab615a4..ae313cfcd9b7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -124,7 +124,7 @@ public void start() metadataStorageConnector.createTaskTables(); ExecutorService executorService = Executors.newSingleThreadExecutor(); taskMigrationCompleteFuture = executorService.submit(() -> { - return metadataStorageConnector.migrateTaskTable(); + return handler.migrateTaskTable(metadataStorageConnector.getTaskTableName()); }); } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index af33f3229a2e..9fa488101232 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -19,8 +19,6 @@ package org.apache.druid.metadata; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.base.Throwables; @@ -45,14 +43,12 @@ import org.skife.jdbi.v2.util.IntegerMapper; import javax.annotation.Nullable; -import java.io.IOException; import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.SQLRecoverableException; import java.sql.SQLTransientException; -import java.sql.Statement; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -390,14 +386,16 @@ public void alterEntryTable(final String tableName) @Override public Void withHandle(Handle handle) { + final Batch batch = handle.createBatch(); if (!tableContainsColumn(handle, tableName, "type")) { log.info("Adding column: type to table[%s]", tableName); - handle.execute(StringUtils.format("ALTER TABLE %1$s ADD COLUMN type VARCHAR(255)", tableName)); + batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN type VARCHAR(255) NULL", tableName)); } if (!tableContainsColumn(handle, tableName, "group_id")) { log.info("Adding column: group_id to table[%s]", tableName); - handle.execute(StringUtils.format("ALTER TABLE %1$s ADD COLUMN group_id VARCHAR(255)", tableName)); + batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN group_id VARCHAR(255) NULL", tableName)); } + batch.execute(); return null; } } @@ -409,57 +407,11 @@ public Void withHandle(Handle handle) } @Override - public boolean migrateTaskTable() + public String getTaskTableName() { final MetadataStorageTablesConfig tablesConfig = tablesConfigSupplier.get(); final String entryType = tablesConfig.getTaskEntryType(); - final String tableName = tablesConfig.getEntryTable(entryType); - return migrateTaskTable(tableName); - } - - public boolean migrateTaskTable(String tableName) - { - log.info("Populate fields task and group_id of task entry table [%s] from payload", tableName); - try { - retryWithHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) throws SQLException, IOException - { - ObjectMapper objectMapper = new ObjectMapper(); - Connection connection = handle.getConnection(); - Statement statement = connection.createStatement(ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_UPDATABLE); - boolean flag = true; - while (flag) { - // Should ideally use a cursor and sort by id for efficiency, but updates with ordering aren't allowed - String sql = StringUtils.format( - "SELECT * FROM %1$s WHERE active = false AND type IS null %2$s", - tableName, - limitClause(100) - ); - ResultSet resultSet = statement.executeQuery(sql); - flag = false; - while (resultSet.next()) { - ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); - resultSet.updateString("type", payload.get("type").asText()); - resultSet.updateString("group_id", payload.get("groupId").asText()); - resultSet.updateRow(); - flag = true; - } - } - statement.close(); - return null; - } - } - ); - log.info("Migration of tasks complete for table[%s]", tableName); - return true; - } - catch (Exception e) { - log.warn(e, "Exception migrating task table [%s]", tableName); - return false; - } + return tablesConfig.getEntryTable(entryType); } public void createLogTable(final String tableName, final String entryTypeName) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index 8fab6518890a..162c3ab671bf 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -52,8 +52,10 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.Statement; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -903,4 +905,50 @@ public Long getLockId(String entryId, LockType lock) .findAny() .orElse(null); } + + @Override + public boolean migrateTaskTable(String tableName) + { + log.info("Populate fields task and group_id of task entry table [%s] from payload", tableName); + try { + connector.retryWithHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws SQLException, IOException + { + ObjectMapper objectMapper = new ObjectMapper(); + Connection connection = handle.getConnection(); + Statement statement = connection.createStatement(ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_UPDATABLE); + boolean flag = true; + while (flag) { + // Should ideally use a cursor and sort by id for efficiency, but updates with ordering aren't allowed + String sql = StringUtils.format( + "SELECT * FROM %1$s WHERE active = false AND type IS null %2$s", + tableName, + connector.limitClause(100) + ); + ResultSet resultSet = statement.executeQuery(sql); + flag = false; + while (resultSet.next()) { + ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); + resultSet.updateString("type", payload.get("type").asText()); + resultSet.updateString("group_id", payload.get("groupId").asText()); + resultSet.updateRow(); + flag = true; + } + } + statement.close(); + return null; + } + } + ); + log.info("Migration of tasks complete for table[%s]", tableName); + return true; + } + catch (Exception e) { + log.warn(e, "Exception migrating task table [%s]", tableName); + return false; + } + } } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java index ed9ce91efa29..1c192da475d5 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -23,7 +23,6 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import org.apache.commons.dbcp2.BasicDataSource; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.junit.Assert; import org.junit.Before; @@ -33,16 +32,12 @@ import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.tweak.HandleCallback; -import java.nio.charset.StandardCharsets; -import java.sql.Blob; -import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.UUID; public class SQLMetadataConnectorTest @@ -113,37 +108,6 @@ public Void withHandle(Handle handle) } } - @Test - public void testMigrateTaskTable() - { - String table = tablesConfig.getTasksTable(); - connector.getDBI().withHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) - { - connector.prepareTaskEntryTable(table); - int numRows = 1000; - List ids = new ArrayList<>(); - for (int i = 0; i < numRows; i++) { - String id = insertIntoTasksTable(table, "type_" + i, "groupId_" + i); - Assert.assertNotEquals("", id); - ids.add(id); - } - - connector.migrateTaskTable(); - - for (int i = 0; i < numRows; i++) { - Assert.assertTrue(verifyTaskTypeAndGroupId(table, ids.get(i), "type_" + i, "groupId_" + i)); - } - return null; - } - } - ); - dropTable(table); - } - @Test public void testInsertOrUpdate() { @@ -297,47 +261,6 @@ public void testBasicDataSourceCreation() throws Exception Assert.assertEquals((long) dataSource.getDefaultQueryTimeout(), 30000); } - private String insertIntoTasksTable(String table, String type, String groupId) - { - String id = UUID.randomUUID().toString(); - String dummy = "dummyData"; - String createdDate = DateTimes.nowUtc().toString(); - try { - connector.retryWithHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) throws SQLException - { - String json = StringUtils.format("{\"type\":\"%1$s\",\"groupId\":\"%2$s\"}", type, groupId); - Blob payload = handle.getConnection().createBlob(); - payload.setBytes(1, json.getBytes(StandardCharsets.UTF_8)); - - String sql = StringUtils.format( - "INSERT INTO %1$s (id, created_date, datasource, active, payload, status_payload) VALUES (?,?,?,?,?,?)", - table - ); - PreparedStatement preparedStatement = handle.getConnection().prepareStatement(sql); - preparedStatement.setString(1, id); - preparedStatement.setString(2, createdDate); - preparedStatement.setString(3, dummy); - preparedStatement.setBoolean(4, false); - preparedStatement.setBlob(5, payload); - preparedStatement.setBlob(6, payload); - preparedStatement.execute(); - preparedStatement.close(); - - return null; - } - } - ); - return id; - } - catch (Exception e) { - return ""; - } - } - private boolean verifyTaskTypeAndGroupId(String table, String id, String type, String groupId) { try { diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 2c4f271d12a3..5daf90449f0a 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -69,7 +69,7 @@ public class SQLMetadataStorageActionHandlerTest private SQLMetadataStorageActionHandler, Map, Map, Map> handler; - final String entryTable = "entries"; + private final String entryTable = "entries"; @Before public void setUp() @@ -507,7 +507,7 @@ public void testGetTaskStatusPlusList() verify(activeAltered, taskStatusPlusList, true, true, true); // MIGRATION - derbyConnectorRule.getConnector().migrateTaskTable(entryTable); + handler.migrateTaskTable(entryTable); // Payload based fetch. task type and groupid will still be populated taskStatusPlusList = handler.getTaskStatusPlusList(taskLookups, null, true); From 48e8a89e608c3a309a04f11690d530b8a57ba772 Mon Sep 17 00:00:00 2001 From: Amatya Date: Mon, 2 May 2022 12:57:50 +0530 Subject: [PATCH 13/16] Address review --- .../MetadataStorageActionHandler.java | 38 ++- .../druid/indexing/common/task/Task.java | 18 ++ .../overlord/HeapMemoryTaskStorage.java | 45 +-- .../overlord/MetadataTaskStorage.java | 25 +- .../druid/metadata/SQLMetadataConnector.java | 4 +- .../SQLMetadataStorageActionHandler.java | 286 ++++++++++++------ .../SQLMetadataStorageActionHandlerTest.java | 166 ++++++---- 7 files changed, 365 insertions(+), 217 deletions(-) diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index 81c60ec9e520..b9681ffe041f 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -21,7 +21,7 @@ import com.google.common.base.Optional; import org.apache.druid.indexer.TaskInfo; -import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexer.TaskMetadata; import org.apache.druid.metadata.TaskLookup.TaskLookupType; import org.joda.time.DateTime; @@ -104,14 +104,40 @@ List> getTaskInfos( ); /** - * Returns a list of TaskStatusPlus for the tasks corresponding to the given filters + * This is the recommended method to fetch Tasks for the task view + * This utilizes the new type and group_id columns and should be utilized after migration + * Returns a list of TaskInfo for the tasks corresponding to the given filters + * The TaskInfo comprises the TaskMetadata which is significantly smaller than a Task, and the TaskStatus + * These are sufficient to create the TaskStatusPlus for a given Task, and prevent unnecessary memory usage * * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata * store. For complete tasks, additional filters in {@code CompleteTaskLookup} can be applied. * All lookups should be processed atomically if more than one lookup is given. * - * It is RECOMMENDED to set fetchPayload to false after task table migration of old data to new schmea has completed + * fetchPayload determines the query used to fetch from the tasks table + * If true, fetch the payload and deserialize it to obtain the above fields + * Else, use the newly created type and group_id columns in the query for task summaries + * + * @param taskLookups task lookup type and filters. + * @param datasource datasource filter + */ + List> getTaskMetadataInfos( + Map taskLookups, + @Nullable String datasource + ); + + /** + * Please use this method to fetch task for viewing on ingestion tab only before task migration + * This deserializes the payload column to get the required fields, and has a greater overhead + * Returns a list of TaskInfo for the tasks corresponding to the given filters + * The TaskInfo comprises the TaskMetadata which is significantly smaller than a Task, and the TaskStatus + * These are sufficient to create the TaskStatusPlus for a given Task, and prevent unnecessary memory usage + * + * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. + * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata + * store. For complete tasks, additional filters in {@code CompleteTaskLookup} can be applied. + * All lookups should be processed atomically if more than one lookup is given. * * fetchPayload determines the query used to fetch from the tasks table * If true, fetch the payload and deserialize it to obtain the above fields @@ -119,12 +145,10 @@ List> getTaskInfos( * * @param taskLookups task lookup type and filters. * @param datasource datasource filter - * @param fetchPayload indicates which query to use based on migration status */ - List getTaskStatusPlusList( + List> getTaskMetadataInfosFromPayload( Map taskLookups, - @Nullable String datasource, - boolean fetchPayload + @Nullable String datasource ); default List> getTaskInfos( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 5529aebc6423..cd3aeab2aa87 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.indexer.TaskInfo; +import org.apache.druid.indexer.TaskMetadata; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; @@ -241,4 +243,20 @@ default ContextValueType getContextValue(String key, ContextV final ContextValueType value = getContextValue(key); return value == null ? defaultValue : value; } + + default TaskMetadata getMetadata() + { + return new TaskMetadata(this.getId(), this.getGroupId(), this.getType()); + } + + static TaskInfo toTaskMetadataInfo(TaskInfo taskInfo) + { + return new TaskInfo<>( + taskInfo.getId(), + taskInfo.getCreatedTime(), + taskInfo.getStatus(), + taskInfo.getDataSource(), + taskInfo.getTask().getMetadata() + ); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index f59d25a391b0..304cb494a28b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -28,7 +28,6 @@ import com.google.common.collect.Ordering; import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; -import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; @@ -172,25 +171,6 @@ public List getActiveTasksByDatasource(String datasource) return listBuilder.build(); } - private TaskStatusPlus toTaskStatusPlus(TaskInfo taskInfo) - { - Task task = taskInfo.getTask(); - TaskStatus status = taskInfo.getStatus(); - return new TaskStatusPlus( - taskInfo.getId(), - task == null ? null : task.getGroupId(), - task == null ? null : task.getType(), - taskInfo.getCreatedTime(), - DateTimes.EPOCH, - status.getStatusCode(), - status.getStatusCode().isComplete() ? RunnerTaskState.NONE : RunnerTaskState.WAITING, - status.getDuration(), - status.getLocation(), - taskInfo.getDataSource(), - status.getErrorMsg() - ); - } - public List> getActiveTaskInfo(@Nullable String dataSource) { final ImmutableList.Builder> listBuilder = ImmutableList.builder(); @@ -260,27 +240,10 @@ public List getTaskStatusPlusList( @Nullable String datasource ) { - final List tasks = new ArrayList<>(); - taskLookups.forEach((type, lookup) -> { - if (type == TaskLookupType.COMPLETE) { - CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) lookup; - tasks.addAll( - getRecentlyCreatedAlreadyFinishedTaskInfo( - completeTaskLookup.hasTaskCreatedTimeFilter() - ? completeTaskLookup - : completeTaskLookup.withDurationBeforeNow(config.getRecentlyFinishedThreshold()), - datasource - ).stream() - .map(taskInfo -> toTaskStatusPlus(taskInfo)) - .collect(Collectors.toList()) - ); - } else { - tasks.addAll(getActiveTaskInfo(datasource).stream() - .map(taskInfo -> toTaskStatusPlus(taskInfo)) - .collect(Collectors.toList())); - } - }); - return tasks; + return getTaskInfos(taskLookups, datasource).stream() + .map(Task::toTaskMetadataInfo) + .map(TaskStatusPlus::fromTaskMetadataInfo) + .collect(Collectors.toList()); } private List> getRecentlyCreatedAlreadyFinishedTaskInfoSince( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index ae313cfcd9b7..28c2f7fc2e23 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -28,6 +28,7 @@ import com.google.common.collect.Maps; import com.google.inject.Inject; import org.apache.druid.indexer.TaskInfo; +import org.apache.druid.indexer.TaskMetadata; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; @@ -121,11 +122,12 @@ public MetadataTaskStorage( @LifecycleStart public void start() { + // Case where active tasks metadataStorageConnector.createTaskTables(); ExecutorService executorService = Executors.newSingleThreadExecutor(); - taskMigrationCompleteFuture = executorService.submit(() -> { - return handler.migrateTaskTable(metadataStorageConnector.getTaskTableName()); - }); + taskMigrationCompleteFuture = executorService.submit( + () -> handler.migrateTaskTable(metadataStorageConnector.getTaskTableName()) + ); } @LifecycleStop @@ -249,7 +251,7 @@ public List getTaskStatusPlusList( @Nullable String datasource ) { - Map theTaskLookups = processTaskLookups(taskLookups); + Map processedTaskLookups = processTaskLookups(taskLookups); boolean fetchPayload = true; if (taskMigrationCompleteFuture.isDone()) { try { @@ -259,8 +261,13 @@ public List getTaskStatusPlusList( log.info(e, "Exception getting task migration future"); } } + List> taskMetadataInfos = fetchPayload + ? handler.getTaskMetadataInfosFromPayload(processedTaskLookups, datasource) + : handler.getTaskMetadataInfos(processedTaskLookups, datasource); return Collections.unmodifiableList( - handler.getTaskStatusPlusList(theTaskLookups, datasource, fetchPayload) + taskMetadataInfos.stream() + .map(TaskStatusPlus::fromTaskMetadataInfo) + .collect(Collectors.toList()) ); } @@ -268,21 +275,21 @@ private Map processTaskLookups( Map taskLookups ) { - Map theTaskLookups = Maps.newHashMapWithExpectedSize(taskLookups.size()); + Map processedTaskLookups = Maps.newHashMapWithExpectedSize(taskLookups.size()); for (Entry entry : taskLookups.entrySet()) { if (entry.getKey() == TaskLookupType.COMPLETE) { CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); - theTaskLookups.put( + processedTaskLookups.put( entry.getKey(), completeTaskLookup.hasTaskCreatedTimeFilter() ? completeTaskLookup : completeTaskLookup.withDurationBeforeNow(config.getRecentlyFinishedThreshold()) ); } else { - theTaskLookups.put(entry.getKey(), entry.getValue()); + processedTaskLookups.put(entry.getKey(), entry.getValue()); } } - return theTaskLookups; + return processedTaskLookups; } @Override diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 9fa488101232..83b66dd3fe47 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -389,11 +389,11 @@ public Void withHandle(Handle handle) final Batch batch = handle.createBatch(); if (!tableContainsColumn(handle, tableName, "type")) { log.info("Adding column: type to table[%s]", tableName); - batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN type VARCHAR(255) NULL", tableName)); + batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN type VARCHAR(255)", tableName)); } if (!tableContainsColumn(handle, tableName, "group_id")) { log.info("Adding column: group_id to table[%s]", tableName); - batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN group_id VARCHAR(255) NULL", tableName)); + batch.add(StringUtils.format("ALTER TABLE %1$s ADD COLUMN group_id VARCHAR(255)", tableName)); } batch.execute(); return null; diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index 162c3ab671bf..4dda47d3c974 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -21,16 +21,14 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Maps; -import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskInfo; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexer.TaskMetadata; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; @@ -39,6 +37,7 @@ import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup; import org.apache.druid.metadata.TaskLookup.TaskLookupType; import org.joda.time.DateTime; +import org.skife.jdbi.v2.Batch; import org.skife.jdbi.v2.FoldController; import org.skife.jdbi.v2.Folder3; import org.skife.jdbi.v2.Handle; @@ -52,10 +51,8 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; -import java.sql.Statement; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -79,7 +76,9 @@ public abstract class SQLMetadataStorageActionHandler taskInfoMapper; - private final TaskStatusPlusMapper taskStatusPlusMapper; + private final TaskMetadataInfoMapper taskMetadataInfoMapper; + private final TaskMetadataInfoMapperFromPayload taskMetadataInfoMapperFromPayload; + private final TaskMetadataMapper taskMetadataMapper; @SuppressWarnings("PMD.UnnecessaryFullyQualifiedName") public SQLMetadataStorageActionHandler( @@ -106,7 +105,9 @@ public SQLMetadataStorageActionHandler( this.logTable = logTable; this.lockTable = lockTable; this.taskInfoMapper = new TaskInfoMapper<>(jsonMapper, entryType, statusType); - this.taskStatusPlusMapper = new TaskStatusPlusMapper(jsonMapper); + this.taskMetadataInfoMapper = new TaskMetadataInfoMapper(jsonMapper); + this.taskMetadataInfoMapperFromPayload = new TaskMetadataInfoMapperFromPayload(jsonMapper); + this.taskMetadataMapper = new TaskMetadataMapper(jsonMapper); } protected SQLMetadataConnector getConnector() @@ -325,16 +326,34 @@ public List> getTaskInfos( } @Override - public List getTaskStatusPlusList( + public List> getTaskMetadataInfosFromPayload( + Map taskLookups, + @Nullable String dataSource + ) + { + return getTaskMetadataInfos(taskLookups, dataSource, true); + } + + @Override + public List> getTaskMetadataInfos( + Map taskLookups, + @Nullable String dataSource + ) + { + return getTaskMetadataInfos(taskLookups, dataSource, false); + } + + public List> getTaskMetadataInfos( Map taskLookups, @Nullable String dataSource, boolean fetchPayload ) { - taskStatusPlusMapper.setUsePayload(fetchPayload); + ResultSetMapper> resultSetMapper = + fetchPayload ? taskMetadataInfoMapperFromPayload : taskMetadataInfoMapper; return getConnector().retryTransaction( (handle, status) -> { - final List taskStatusPlusList = new ArrayList<>(); + final List> taskMetadataInfos = new ArrayList<>(); for (Entry entry : taskLookups.entrySet()) { final Query> query; switch (entry.getKey()) { @@ -342,7 +361,7 @@ public List getTaskStatusPlusList( query = !fetchPayload ? createActiveTaskSummaryStreamingQuery(handle, dataSource) : createActiveTaskStreamingQuery(handle, dataSource); - taskStatusPlusList.addAll(query.map(taskStatusPlusMapper).list()); + taskMetadataInfos.addAll(query.map(resultSetMapper).list()); break; case COMPLETE: CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); @@ -351,13 +370,17 @@ public List getTaskStatusPlusList( query = !fetchPayload ? createCompletedTaskSummaryStreamingQuery(handle, priorTo, limit, dataSource) : createCompletedTaskStreamingQuery(handle, priorTo, limit, dataSource); - taskStatusPlusList.addAll(query.map(taskStatusPlusMapper).list()); + taskMetadataInfos.addAll(query.map(resultSetMapper).list()); break; default: throw new IAE("Unknown TaskLookupType: [%s]", entry.getKey()); } } - return taskStatusPlusList; + for (TaskInfo taskMetadataInfo : taskMetadataInfos) { + System.out.println(taskMetadataInfo.getTask()); + } + System.out.println("-------------------"); + return taskMetadataInfos; }, 3, SQLMetadataConnector.DEFAULT_MAX_TRIES @@ -548,72 +571,106 @@ private String getWhereClauseForActiveStatusesQuery(String dataSource) return sql; } - static class TaskStatusPlusMapper implements ResultSetMapper + class TaskMetadataInfoMapperFromPayload implements ResultSetMapper> { private final ObjectMapper objectMapper; - private boolean usePayload; - TaskStatusPlusMapper(ObjectMapper objectMapper) + TaskMetadataInfoMapperFromPayload(ObjectMapper objectMapper) { this.objectMapper = objectMapper; - this.usePayload = true; } - void setUsePayload(boolean usePayload) + @Override + public TaskInfo map(int index, ResultSet resultSet, StatementContext context) + throws SQLException { - this.usePayload = usePayload; + return toTaskMetadataInfo(objectMapper, resultSet, true); + } + } + + class TaskMetadataInfoMapper implements ResultSetMapper> + { + private final ObjectMapper objectMapper; + + TaskMetadataInfoMapper(ObjectMapper objectMapper) + { + this.objectMapper = objectMapper; } @Override - public TaskStatusPlus map(int index, ResultSet resultSet, StatementContext context) + public TaskInfo map(int index, ResultSet resultSet, StatementContext context) throws SQLException { - String type; - String groupId; - if (usePayload) { - try { - ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); - type = payload.get("type").asText(); - groupId = payload.get("groupId").asText(); - } - catch (IOException e) { - log.error(e, "Encountered exception while deserializing task payload"); - throw new SQLException(e); - } - } else { - type = resultSet.getString("type"); - groupId = resultSet.getString("group_id"); - } + return toTaskMetadataInfo(objectMapper, resultSet, false); + } + } - TaskState statusCode; - Long duration; - TaskLocation location; - String errorMsg; + private TaskInfo toTaskMetadataInfo(ObjectMapper objectMapper, + ResultSet resultSet, + boolean usePayload + ) throws SQLException + { + String type; + String groupId; + if (usePayload) { try { - ObjectNode status = objectMapper.readValue(resultSet.getBytes("status_payload"), ObjectNode.class); - statusCode = objectMapper.convertValue(status.get("status"), TaskState.class); - duration = status.get("duration").asLong(); - location = objectMapper.convertValue(status.get("location"), TaskLocation.class); - errorMsg = status.get("errorMsg").asText(); + ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); + type = payload.get("type").asText(); + groupId = payload.get("groupId").asText(); } catch (IOException e) { - log.error(e, "Encountered exception while deserializing task status_payload"); + log.error(e, "Encountered exception while deserializing task payload"); throw new SQLException(e); } + } else { + type = resultSet.getString("type"); + groupId = resultSet.getString("group_id"); + } - return new TaskStatusPlus( - resultSet.getString("id"), - groupId, - type, - DateTimes.of(resultSet.getString("created_date")), - DateTimes.EPOCH, - statusCode, - statusCode.isComplete() ? RunnerTaskState.NONE : RunnerTaskState.WAITING, - duration, - location, - resultSet.getString("datasource"), - errorMsg - ); + String id = resultSet.getString("id"); + DateTime createdTime = DateTimes.of(resultSet.getString("created_date")); + StatusType status; + try { + status = objectMapper.readValue(resultSet.getBytes("status_payload"), statusType); + } + catch (IOException e) { + log.error(e, "Encountered exception while deserializing task status_payload"); + throw new SQLException(e); + } + String datasource = resultSet.getString("datasource"); + TaskMetadata taskMetadata = new TaskMetadata(id, groupId, type); + + return new TaskInfo<>(id, createdTime, status, datasource, taskMetadata); + } + + static class TaskMetadataMapper implements ResultSetMapper + { + private final ObjectMapper objectMapper; + + TaskMetadataMapper(ObjectMapper objectMapper) + { + this.objectMapper = objectMapper; + } + + @Override + public TaskMetadata map(int index, ResultSet resultSet, StatementContext context) + throws SQLException + { + try { + ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); + // If field is absent (older task version), use blank string to avoid a loop of migration of such tasks. + JsonNode type = payload.get("type"); + JsonNode groupId = payload.get("groupId"); + return new TaskMetadata( + resultSet.getString("id"), + groupId == null ? "" : groupId.asText(), + type == null ? "" : type.asText() + ); + } + catch (IOException e) { + log.error(e, "Encountered exception while deserializing task payload"); + throw new SQLException(e); + } } } @@ -906,49 +963,80 @@ public Long getLockId(String entryId, LockType lock) .orElse(null); } - @Override - public boolean migrateTaskTable(String tableName) + private List fetchTaskMetadatas(String tableName, String id, int limit) { - log.info("Populate fields task and group_id of task entry table [%s] from payload", tableName); - try { - connector.retryWithHandle( - new HandleCallback() + List taskMetadatas = new ArrayList<>(); + connector.retryWithHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) { - @Override - public Void withHandle(Handle handle) throws SQLException, IOException - { - ObjectMapper objectMapper = new ObjectMapper(); - Connection connection = handle.getConnection(); - Statement statement = connection.createStatement(ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_UPDATABLE); - boolean flag = true; - while (flag) { - // Should ideally use a cursor and sort by id for efficiency, but updates with ordering aren't allowed - String sql = StringUtils.format( - "SELECT * FROM %1$s WHERE active = false AND type IS null %2$s", - tableName, - connector.limitClause(100) - ); - ResultSet resultSet = statement.executeQuery(sql); - flag = false; - while (resultSet.next()) { - ObjectNode payload = objectMapper.readValue(resultSet.getBytes("payload"), ObjectNode.class); - resultSet.updateString("type", payload.get("type").asText()); - resultSet.updateString("group_id", payload.get("groupId").asText()); - resultSet.updateRow(); - flag = true; - } - } - statement.close(); - return null; + String sql = StringUtils.format( + "SELECT * FROM %1$s WHERE id > '%2$s' AND type IS null ORDER BY id %3$s", + tableName, + id, + connector.limitClause(limit) + ); + Query> query = handle.createQuery(sql); + taskMetadatas.addAll(query.map(taskMetadataMapper).list()); + return null; + } + } + ); + return taskMetadatas; + } + + private void updateTaskMetadatas(String tasksTable, List taskMetadatas) + { + connector.retryWithHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) + { + Batch batch = handle.createBatch(); + String sql = "UPDATE %1$s SET type = '%2$s', group_id = '%3$s' WHERE id = '%4$s'"; + for (TaskMetadata metadata : taskMetadatas) { + batch.add(StringUtils.format(sql, tasksTable, metadata.getType(), metadata.getGroupId(), metadata.getId()) + ); } + batch.execute(); + return null; } - ); - log.info("Migration of tasks complete for table[%s]", tableName); - return true; - } - catch (Exception e) { - log.warn(e, "Exception migrating task table [%s]", tableName); - return false; + } + ); + } + + + @Override + public boolean migrateTaskTable(String tableName) + { + log.info("Populate fields task and group_id of task entry table [%s] from payload", tableName); + String id = ""; + int limit = 100; + while (true) { + List taskMetadatas; + try { + taskMetadatas = fetchTaskMetadatas(tableName, id, limit); + } + catch (Exception e) { + log.warn(e, "Task migration failed while reading entries from task table"); + return false; + } + if (taskMetadatas.isEmpty()) { + break; + } + try { + updateTaskMetadatas(tableName, taskMetadatas); + } + catch (Exception e) { + log.warn(e, "Task migration failed while updating entries in task table"); + return false; + } + id = taskMetadatas.get(taskMetadatas.size() - 1).getId(); } + log.info("Task migration for table [%s] successful", tableName); + return true; } } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 5daf90449f0a..32118e18ea39 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -25,11 +25,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskMetadata; import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; @@ -45,7 +44,11 @@ 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 java.sql.ResultSet; +import java.sql.SQLException; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -466,69 +469,104 @@ public void testRemoveTasksOlderThan() throws Exception Assert.assertEquals(1, handler.getLogs(entryId3).size()); } + @Test + public void testMigration() + { + int active = 1234; + for (int i = 0; i < active; i++) { + insertTaskInfo(createRandomTaskInfo(true), false); + } + + int completed = 2345; + for (int i = 0; i < completed; i++) { + insertTaskInfo(createRandomTaskInfo(false), false); + } + + Assert.assertEquals(active + completed, getUnmigratedTaskCount().intValue()); + + handler.migrateTaskTable(entryTable); + + Assert.assertEquals(0, getUnmigratedTaskCount().intValue()); + } + @Test public void testGetTaskStatusPlusList() { // SETUP - TaskInfo, Map> activeUnaltered = getRandomTaskInfo(true); + TaskInfo, Map> activeUnaltered = createRandomTaskInfo(true); insertTaskInfo(activeUnaltered, false); - TaskInfo, Map> completedUnaltered = getRandomTaskInfo(false); + TaskInfo, Map> completedUnaltered = createRandomTaskInfo(false); insertTaskInfo(completedUnaltered, false); - TaskInfo, Map> activeAltered = getRandomTaskInfo(true); + TaskInfo, Map> activeAltered = createRandomTaskInfo(true); insertTaskInfo(activeAltered, true); - TaskInfo, Map> completedAltered = getRandomTaskInfo(false); + TaskInfo, Map> completedAltered = createRandomTaskInfo(false); insertTaskInfo(completedAltered, true); Map taskLookups = new HashMap<>(); taskLookups.put(TaskLookup.TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()); taskLookups.put(TaskLookup.TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, Duration.millis(86400000))); - List taskStatusPlusList; + List>> taskMetadataInfos; // BEFORE MIGRATION // Payload based fetch. task type and groupid will be populated - taskStatusPlusList = handler.getTaskStatusPlusList(taskLookups, null, true); - Assert.assertEquals(4, taskStatusPlusList.size()); - verify(completedUnaltered, taskStatusPlusList, false, false, true); - verify(completedAltered, taskStatusPlusList, false, true, false); - verify(activeUnaltered, taskStatusPlusList, true, false, false); - verify(activeAltered, taskStatusPlusList, true, true, false); + taskMetadataInfos = handler.getTaskMetadataInfos(taskLookups, null, true); + Assert.assertEquals(4, taskMetadataInfos.size()); + verifyTaskInfoToMetadataInfo(completedUnaltered, taskMetadataInfos, false); + verifyTaskInfoToMetadataInfo(completedAltered, taskMetadataInfos, false); + verifyTaskInfoToMetadataInfo(activeUnaltered, taskMetadataInfos, false); + verifyTaskInfoToMetadataInfo(activeAltered, taskMetadataInfos, false); // New columns based fetch before migration is complete. type and payload are null when altered = false - taskStatusPlusList = handler.getTaskStatusPlusList(taskLookups, null, false); - Assert.assertEquals(4, taskStatusPlusList.size()); - verify(completedUnaltered, taskStatusPlusList, false, false, true); - verify(completedAltered, taskStatusPlusList, false, true, true); - verify(activeUnaltered, taskStatusPlusList, true, false, true); - verify(activeAltered, taskStatusPlusList, true, true, true); + taskMetadataInfos = handler.getTaskMetadataInfos(taskLookups, null, false); + Assert.assertEquals(4, taskMetadataInfos.size()); + verifyTaskInfoToMetadataInfo(completedUnaltered, taskMetadataInfos, true); + verifyTaskInfoToMetadataInfo(completedAltered, taskMetadataInfos, false); + verifyTaskInfoToMetadataInfo(activeUnaltered, taskMetadataInfos, true); + verifyTaskInfoToMetadataInfo(activeAltered, taskMetadataInfos, false); // MIGRATION handler.migrateTaskTable(entryTable); - // Payload based fetch. task type and groupid will still be populated - taskStatusPlusList = handler.getTaskStatusPlusList(taskLookups, null, true); - Assert.assertEquals(4, taskStatusPlusList.size()); - verify(completedUnaltered, taskStatusPlusList, false, false, false); - verify(completedAltered, taskStatusPlusList, false, true, false); - verify(activeUnaltered, taskStatusPlusList, true, false, false); - verify(activeAltered, taskStatusPlusList, true, true, false); - - // New columns based fetch before migration is complete. - // type and payload are not null for completed task but are still null for active ones since they aren't migrated - // An active task will be eventually updated on its own due to insertion - taskStatusPlusList = handler.getTaskStatusPlusList(taskLookups, null, false); - Assert.assertEquals(4, taskStatusPlusList.size()); - verify(completedUnaltered, taskStatusPlusList, false, false, false); - verify(completedAltered, taskStatusPlusList, false, true, false); - verify(activeUnaltered, taskStatusPlusList, true, false, true); - verify(activeAltered, taskStatusPlusList, true, true, false); + // Payload based fetch. task type and groupid will still be populated in tasks tab + taskMetadataInfos = handler.getTaskMetadataInfos(taskLookups, null, true); + Assert.assertEquals(4, taskMetadataInfos.size()); + verifyTaskInfoToMetadataInfo(completedUnaltered, taskMetadataInfos, false); + verifyTaskInfoToMetadataInfo(completedAltered, taskMetadataInfos, false); + verifyTaskInfoToMetadataInfo(activeUnaltered, taskMetadataInfos, false); + verifyTaskInfoToMetadataInfo(activeAltered, taskMetadataInfos, false); + + // New columns based fetch after migration is complete. All data must be populated in the tasks table + taskMetadataInfos = handler.getTaskMetadataInfos(taskLookups, null, false); + Assert.assertEquals(4, taskMetadataInfos.size()); + verifyTaskInfoToMetadataInfo(completedUnaltered, taskMetadataInfos, false); + verifyTaskInfoToMetadataInfo(completedAltered, taskMetadataInfos, false); + verifyTaskInfoToMetadataInfo(activeUnaltered, taskMetadataInfos, false); + verifyTaskInfoToMetadataInfo(activeAltered, taskMetadataInfos, false); } - private TaskInfo, Map> getRandomTaskInfo(boolean active) + private Integer getUnmigratedTaskCount() + { + return handler.getConnector().retryWithHandle( + new HandleCallback() + { + @Override + public Integer withHandle(Handle handle) throws SQLException + { + String sql = String.format("SELECT COUNT(*) FROM %s WHERE type is NULL or group_id is NULL", entryTable); + ResultSet resultSet = handle.getConnection().createStatement().executeQuery(sql); + resultSet.next(); + return resultSet.getInt(1); + } + } + ); + } + + private TaskInfo, Map> createRandomTaskInfo(boolean active) { String id = UUID.randomUUID().toString(); DateTime createdTime = DateTime.now(DateTimeZone.UTC); @@ -577,37 +615,47 @@ private void insertTaskInfo(TaskInfo, Map> t } } - private void verify(TaskInfo, Map> taskInfo, List taskStatusPlusList, - boolean active, boolean altered, boolean nullNewColumns) + private void verifyTaskInfoToMetadataInfo(TaskInfo, Map> taskInfo, + List>> taskMetadataInfos, + boolean nullNewColumns) { - for (TaskStatusPlus taskStatusPlus : taskStatusPlusList) { - if (taskStatusPlus.getId().equals(taskInfo.getId())) { - verify(taskInfo, taskStatusPlus, active, altered, nullNewColumns); + for (TaskInfo> taskMetadataInfo : taskMetadataInfos) { + if (taskMetadataInfo.getId().equals(taskInfo.getId())) { + verifyTaskInfoToMetadataInfo(taskInfo, taskMetadataInfo, nullNewColumns); } return; } Assert.fail(); } - private void verify(TaskInfo, Map> taskInfo, TaskStatusPlus taskStatusPlus, - boolean active, boolean altered, boolean nullNewColumns) + private void verifyTaskInfoToMetadataInfo(TaskInfo, Map> taskInfo, + TaskInfo> taskMetadataInfo, + boolean nullNewColumns) { - Assert.assertEquals(taskInfo.getId(), taskStatusPlus.getId()); - Assert.assertEquals(taskInfo.getCreatedTime(), taskStatusPlus.getCreatedTime()); - Assert.assertEquals(taskInfo.getCreatedTime(), taskStatusPlus.getCreatedTime()); - Assert.assertEquals(DateTimes.EPOCH, taskStatusPlus.getQueueInsertionTime()); - Assert.assertEquals(active ? TaskState.RUNNING : TaskState.SUCCESS, taskStatusPlus.getStatusCode()); - Assert.assertEquals(!active ? RunnerTaskState.NONE : RunnerTaskState.WAITING, taskStatusPlus.getRunnerStatusCode()); - Assert.assertEquals(taskInfo.getStatus().get("duration"), taskStatusPlus.getDuration()); - Assert.assertEquals(taskInfo.getStatus().get("location"), taskStatusPlus.getLocation()); - Assert.assertEquals(taskInfo.getDataSource(), taskStatusPlus.getDataSource()); - Assert.assertEquals(taskInfo.getStatus().get("errorMsg"), taskStatusPlus.getErrorMsg()); - if (!altered && nullNewColumns) { - Assert.assertNull(taskStatusPlus.getType()); - Assert.assertNull(taskStatusPlus.getGroupId()); + Assert.assertEquals(taskInfo.getId(), taskMetadataInfo.getId()); + Assert.assertEquals(taskInfo.getCreatedTime(), taskMetadataInfo.getCreatedTime()); + Assert.assertEquals(taskInfo.getDataSource(), taskMetadataInfo.getDataSource()); + + verifyTaskStatus(taskInfo.getStatus(), taskMetadataInfo.getStatus()); + + Map task = taskInfo.getTask(); + TaskMetadata taskMetadata = taskMetadataInfo.getTask(); + Assert.assertEquals(task.get("id"), taskMetadata.getId()); + if (nullNewColumns) { + Assert.assertNull(taskMetadata.getGroupId()); + Assert.assertNull(taskMetadata.getType()); } else { - Assert.assertEquals(taskInfo.getTask().get("type"), taskStatusPlus.getType()); - Assert.assertEquals(taskInfo.getTask().get("groupId"), taskStatusPlus.getGroupId()); + Assert.assertEquals(task.get("groupId"), taskMetadata.getGroupId()); + Assert.assertEquals(task.get("type"), taskMetadata.getType()); } } + + private void verifyTaskStatus(Map expected, Map actual) + { + Assert.assertEquals(expected.get("id"), actual.get("id")); + Assert.assertEquals(expected.get("duration"), actual.get("duration")); + Assert.assertEquals(expected.get("errorMsg"), actual.get("errorMsg")); + Assert.assertEquals(expected.get("status").toString(), actual.get("status")); + Assert.assertEquals(expected.get("location"), JSON_MAPPER.convertValue(actual.get("location"), TaskLocation.class)); + } } From 402dd124f64cbff72b0d1730c2289bd1c5e5d4bf Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 15 Jun 2022 18:42:35 +0530 Subject: [PATCH 14/16] Incorporate feedback --- ...{TaskMetadata.java => TaskIdentifier.java} | 10 +- .../apache/druid/indexer/TaskStatusPlus.java | 20 +-- .../MetadataStorageActionHandler.java | 38 ++---- .../metadata/MetadataStorageConnector.java | 2 - .../druid/indexing/common/task/Task.java | 8 +- .../overlord/HeapMemoryTaskStorage.java | 4 +- .../overlord/MetadataTaskStorage.java | 12 +- .../druid/indexing/overlord/TaskStorage.java | 4 +- .../results/auth_test_sys_schema_tasks.json | 4 +- .../druid/metadata/SQLMetadataConnector.java | 10 +- .../SQLMetadataStorageActionHandler.java | 123 +++++++++--------- .../SQLMetadataStorageActionHandlerTest.java | 39 +++--- 12 files changed, 131 insertions(+), 143 deletions(-) rename core/src/main/java/org/apache/druid/indexer/{TaskMetadata.java => TaskIdentifier.java} (92%) diff --git a/core/src/main/java/org/apache/druid/indexer/TaskMetadata.java b/core/src/main/java/org/apache/druid/indexer/TaskIdentifier.java similarity index 92% rename from core/src/main/java/org/apache/druid/indexer/TaskMetadata.java rename to core/src/main/java/org/apache/druid/indexer/TaskIdentifier.java index ef0d151ab3d1..753de56328bf 100644 --- a/core/src/main/java/org/apache/druid/indexer/TaskMetadata.java +++ b/core/src/main/java/org/apache/druid/indexer/TaskIdentifier.java @@ -27,10 +27,10 @@ import java.util.Objects; /** - * Model class containing the fields relevant to view tasks in the ingestion tab. + * Model class containing the id, type and groupId of a task * These fields are extracted from the task payload for the new schema and this model can be used for migration as well. */ -public class TaskMetadata +public class TaskIdentifier { private final String id; @@ -42,7 +42,7 @@ public class TaskMetadata private final String groupId; @JsonCreator - public TaskMetadata( + public TaskIdentifier( @JsonProperty("id") String id, @JsonProperty("groupId") @Nullable String groupId, @JsonProperty("type") @Nullable String type // nullable for backward compatibility @@ -82,7 +82,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - TaskMetadata that = (TaskMetadata) o; + TaskIdentifier that = (TaskIdentifier) o; return Objects.equals(getId(), that.getId()) && Objects.equals(getGroupId(), that.getGroupId()) && Objects.equals(getType(), that.getType()); @@ -101,7 +101,7 @@ public int hashCode() @Override public String toString() { - return "TaskMetadata{" + + return "TaskIdentifier{" + "id='" + id + '\'' + ", groupId='" + groupId + '\'' + ", type='" + type + '\'' + diff --git a/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java b/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java index 92875db23a2e..af04db211138 100644 --- a/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java +++ b/core/src/main/java/org/apache/druid/indexer/TaskStatusPlus.java @@ -256,26 +256,26 @@ public String toString() } /** - * Convert a TaskInfo pair of TaskMetadata and TaskStatus to a TaskStatusPlus - * Applicable only for completed or waiting tasks + * Convert a TaskInfo class of TaskIdentifier and TaskStatus to a TaskStatusPlus + * Applicable only for completed or waiting tasks since a TaskInfo doesn't have the exhaustive info for running tasks * - * @param taskMetadataInfo TaskInfo pair + * @param taskIdentifierInfo TaskInfo pair * @return corresponding TaskStatusPlus */ - public static TaskStatusPlus fromTaskMetadataInfo(TaskInfo taskMetadataInfo) + public static TaskStatusPlus fromTaskIdentifierInfo(TaskInfo taskIdentifierInfo) { - TaskStatus status = taskMetadataInfo.getStatus(); + TaskStatus status = taskIdentifierInfo.getStatus(); return new TaskStatusPlus( - taskMetadataInfo.getId(), - taskMetadataInfo.getTask().getGroupId(), - taskMetadataInfo.getTask().getType(), - taskMetadataInfo.getCreatedTime(), + taskIdentifierInfo.getId(), + taskIdentifierInfo.getTask().getGroupId(), + taskIdentifierInfo.getTask().getType(), + taskIdentifierInfo.getCreatedTime(), DateTimes.EPOCH, status.getStatusCode(), status.getStatusCode().isComplete() ? RunnerTaskState.NONE : RunnerTaskState.WAITING, status.getDuration(), status.getLocation(), - taskMetadataInfo.getDataSource(), + taskIdentifierInfo.getDataSource(), status.getErrorMsg() ); } diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index b9681ffe041f..099961c0bed6 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -20,8 +20,8 @@ package org.apache.druid.metadata; import com.google.common.base.Optional; +import org.apache.druid.indexer.TaskIdentifier; import org.apache.druid.indexer.TaskInfo; -import org.apache.druid.indexer.TaskMetadata; import org.apache.druid.metadata.TaskLookup.TaskLookupType; import org.joda.time.DateTime; @@ -104,49 +104,37 @@ List> getTaskInfos( ); /** - * This is the recommended method to fetch Tasks for the task view - * This utilizes the new type and group_id columns and should be utilized after migration - * Returns a list of TaskInfo for the tasks corresponding to the given filters - * The TaskInfo comprises the TaskMetadata which is significantly smaller than a Task, and the TaskStatus - * These are sufficient to create the TaskStatusPlus for a given Task, and prevent unnecessary memory usage + * Returns the statuses of the specified tasks. Implementations of this method must not + * read the task payload from the underlying storage as it may increase memory usage. * * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata * store. For complete tasks, additional filters in {@code CompleteTaskLookup} can be applied. * All lookups should be processed atomically if more than one lookup is given. * - * fetchPayload determines the query used to fetch from the tasks table - * If true, fetch the payload and deserialize it to obtain the above fields - * Else, use the newly created type and group_id columns in the query for task summaries - * * @param taskLookups task lookup type and filters. * @param datasource datasource filter */ - List> getTaskMetadataInfos( + List> getTaskStatusList( Map taskLookups, @Nullable String datasource ); /** - * Please use this method to fetch task for viewing on ingestion tab only before task migration - * This deserializes the payload column to get the required fields, and has a greater overhead - * Returns a list of TaskInfo for the tasks corresponding to the given filters - * The TaskInfo comprises the TaskMetadata which is significantly smaller than a Task, and the TaskStatus - * These are sufficient to create the TaskStatusPlus for a given Task, and prevent unnecessary memory usage + * Returns the statuses of the specified tasks. Implementations of this method may + * read from the corresponding task payloads to retrieve task information. + * + * This method is deprecated and {@link #getTaskStatusList} should be used instead. * * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata * store. For complete tasks, additional filters in {@code CompleteTaskLookup} can be applied. * All lookups should be processed atomically if more than one lookup is given. * - * fetchPayload determines the query used to fetch from the tasks table - * If true, fetch the payload and deserialize it to obtain the above fields - * Else, use the newly created type and group_id columns in the query for task summaries - * * @param taskLookups task lookup type and filters. * @param datasource datasource filter */ - List> getTaskMetadataInfosFromPayload( + List> getTaskStatusListFromPayload( Map taskLookups, @Nullable String datasource ); @@ -227,11 +215,11 @@ default List> getTaskInfos( Long getLockId(String entryId, LockType lock); /** - * Utility to migrate existing tasks to the new schema + * Utility to migrate existing tasks to the new schema by populating type and groupId * * To be kicked off in a separate thread at MetadataTaskStorage startup. - * @param tasksTable - * @return + * + * @return true if migration was successful */ - boolean migrateTaskTable(String tasksTable); + boolean populateTaskTypeAndGroupId(); } diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java index 9211130b4041..45fb6639082c 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -87,7 +87,5 @@ default void exportTable( void createSupervisorsTable(); - String getTaskTableName(); - void deleteAllRecords(String tableName); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index cd3aeab2aa87..65f75ee513b3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.indexer.TaskIdentifier; import org.apache.druid.indexer.TaskInfo; -import org.apache.druid.indexer.TaskMetadata; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; @@ -244,12 +244,12 @@ default ContextValueType getContextValue(String key, ContextV return value == null ? defaultValue : value; } - default TaskMetadata getMetadata() + default TaskIdentifier getMetadata() { - return new TaskMetadata(this.getId(), this.getGroupId(), this.getType()); + return new TaskIdentifier(this.getId(), this.getGroupId(), this.getType()); } - static TaskInfo toTaskMetadataInfo(TaskInfo taskInfo) + static TaskInfo toTaskIdentifierInfo(TaskInfo taskInfo) { return new TaskInfo<>( taskInfo.getId(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index 304cb494a28b..8ef5970982ad 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -241,8 +241,8 @@ public List getTaskStatusPlusList( ) { return getTaskInfos(taskLookups, datasource).stream() - .map(Task::toTaskMetadataInfo) - .map(TaskStatusPlus::fromTaskMetadataInfo) + .map(Task::toTaskIdentifierInfo) + .map(TaskStatusPlus::fromTaskIdentifierInfo) .collect(Collectors.toList()); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 28c2f7fc2e23..01e7a6594e1d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -27,8 +27,8 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.inject.Inject; +import org.apache.druid.indexer.TaskIdentifier; import org.apache.druid.indexer.TaskInfo; -import org.apache.druid.indexer.TaskMetadata; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskLock; @@ -126,7 +126,7 @@ public void start() metadataStorageConnector.createTaskTables(); ExecutorService executorService = Executors.newSingleThreadExecutor(); taskMigrationCompleteFuture = executorService.submit( - () -> handler.migrateTaskTable(metadataStorageConnector.getTaskTableName()) + () -> handler.populateTaskTypeAndGroupId() ); } @@ -261,12 +261,12 @@ public List getTaskStatusPlusList( log.info(e, "Exception getting task migration future"); } } - List> taskMetadataInfos = fetchPayload - ? handler.getTaskMetadataInfosFromPayload(processedTaskLookups, datasource) - : handler.getTaskMetadataInfos(processedTaskLookups, datasource); + List> taskMetadataInfos = fetchPayload + ? handler.getTaskStatusListFromPayload(processedTaskLookups, datasource) + : handler.getTaskStatusList(processedTaskLookups, datasource); return Collections.unmodifiableList( taskMetadataInfos.stream() - .map(TaskStatusPlus::fromTaskMetadataInfo) + .map(TaskStatusPlus::fromTaskIdentifierInfo) .collect(Collectors.toList()) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java index 2d5053b6e682..e9b0af24057f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java @@ -152,8 +152,8 @@ public interface TaskStorage List getActiveTasksByDatasource(String datasource); /** - * Returns tasks stored in the storage facility as a List of TaskStatusPlus - * particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. + * Returns the status of tasks in metadata storage as TaskStatusPlus + * No particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. * * The returned list can contain active tasks and complete tasks depending on the {@code taskLookups} parameter. * See {@link TaskLookup} for more details of active and complete tasks. diff --git a/integration-tests/src/test/resources/results/auth_test_sys_schema_tasks.json b/integration-tests/src/test/resources/results/auth_test_sys_schema_tasks.json index 3f0e48fca71e..fe09abad5c8d 100644 --- a/integration-tests/src/test/resources/results/auth_test_sys_schema_tasks.json +++ b/integration-tests/src/test/resources/results/auth_test_sys_schema_tasks.json @@ -1,8 +1,8 @@ [ { "task_id": "index_auth_test_2030-04-30T01:13:31.893Z", - "type": null, - "group_id": null, + "group_id": "", + "type": "", "datasource": "auth_test", "created_time": "2030-04-30T01:13:31.893Z", "queue_insertion_time": "1970-01-01T00:00:00.000Z", diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 83b66dd3fe47..781a4b99c8f9 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -377,7 +377,7 @@ tableName, getPayloadType(), getCollation() ); } - public void alterEntryTable(final String tableName) + private void alterEntryTable(final String tableName) { try { retryWithHandle( @@ -406,14 +406,6 @@ public Void withHandle(Handle handle) } } - @Override - public String getTaskTableName() - { - final MetadataStorageTablesConfig tablesConfig = tablesConfigSupplier.get(); - final String entryType = tablesConfig.getTaskEntryType(); - return tablesConfig.getEntryTable(entryType); - } - public void createLogTable(final String tableName, final String entryTypeName) { createTable( diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index 4dda47d3c974..f3830b7cb3cb 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -27,8 +27,8 @@ import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Maps; +import org.apache.druid.indexer.TaskIdentifier; import org.apache.druid.indexer.TaskInfo; -import org.apache.druid.indexer.TaskMetadata; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; @@ -76,9 +76,9 @@ public abstract class SQLMetadataStorageActionHandler taskInfoMapper; - private final TaskMetadataInfoMapper taskMetadataInfoMapper; - private final TaskMetadataInfoMapperFromPayload taskMetadataInfoMapperFromPayload; - private final TaskMetadataMapper taskMetadataMapper; + private final TaskStatusMapper taskStatusMapper; + private final TaskStatusMapperFromPayload taskStatusMapperFromPayload; + private final TaskIdentifierMapper taskIdentifierMapper; @SuppressWarnings("PMD.UnnecessaryFullyQualifiedName") public SQLMetadataStorageActionHandler( @@ -105,9 +105,9 @@ public SQLMetadataStorageActionHandler( this.logTable = logTable; this.lockTable = lockTable; this.taskInfoMapper = new TaskInfoMapper<>(jsonMapper, entryType, statusType); - this.taskMetadataInfoMapper = new TaskMetadataInfoMapper(jsonMapper); - this.taskMetadataInfoMapperFromPayload = new TaskMetadataInfoMapperFromPayload(jsonMapper); - this.taskMetadataMapper = new TaskMetadataMapper(jsonMapper); + this.taskStatusMapper = new TaskStatusMapper(jsonMapper); + this.taskStatusMapperFromPayload = new TaskStatusMapperFromPayload(jsonMapper); + this.taskIdentifierMapper = new TaskIdentifierMapper(jsonMapper); } protected SQLMetadataConnector getConnector() @@ -326,60 +326,56 @@ public List> getTaskInfos( } @Override - public List> getTaskMetadataInfosFromPayload( + public List> getTaskStatusListFromPayload( Map taskLookups, @Nullable String dataSource ) { - return getTaskMetadataInfos(taskLookups, dataSource, true); + return getTaskStatusList(taskLookups, dataSource, true); } @Override - public List> getTaskMetadataInfos( + public List> getTaskStatusList( Map taskLookups, @Nullable String dataSource ) { - return getTaskMetadataInfos(taskLookups, dataSource, false); + return getTaskStatusList(taskLookups, dataSource, false); } - public List> getTaskMetadataInfos( + public List> getTaskStatusList( Map taskLookups, @Nullable String dataSource, boolean fetchPayload ) { - ResultSetMapper> resultSetMapper = - fetchPayload ? taskMetadataInfoMapperFromPayload : taskMetadataInfoMapper; + ResultSetMapper> resultSetMapper = + fetchPayload ? taskStatusMapperFromPayload : taskStatusMapper; return getConnector().retryTransaction( (handle, status) -> { - final List> taskMetadataInfos = new ArrayList<>(); + final List> taskMetadataInfos = new ArrayList<>(); for (Entry entry : taskLookups.entrySet()) { final Query> query; switch (entry.getKey()) { case ACTIVE: - query = !fetchPayload - ? createActiveTaskSummaryStreamingQuery(handle, dataSource) - : createActiveTaskStreamingQuery(handle, dataSource); + query = fetchPayload + ? createActiveTaskStreamingQuery(handle, dataSource) + : createActiveTaskSummaryStreamingQuery(handle, dataSource); taskMetadataInfos.addAll(query.map(resultSetMapper).list()); break; case COMPLETE: CompleteTaskLookup completeTaskLookup = (CompleteTaskLookup) entry.getValue(); DateTime priorTo = completeTaskLookup.getTasksCreatedPriorTo(); Integer limit = completeTaskLookup.getMaxTaskStatuses(); - query = !fetchPayload - ? createCompletedTaskSummaryStreamingQuery(handle, priorTo, limit, dataSource) - : createCompletedTaskStreamingQuery(handle, priorTo, limit, dataSource); + query = fetchPayload + ? createCompletedTaskStreamingQuery(handle, priorTo, limit, dataSource) + : createCompletedTaskSummaryStreamingQuery(handle, priorTo, limit, dataSource); taskMetadataInfos.addAll(query.map(resultSetMapper).list()); break; default: throw new IAE("Unknown TaskLookupType: [%s]", entry.getKey()); } } - for (TaskInfo taskMetadataInfo : taskMetadataInfos) { - System.out.println(taskMetadataInfo.getTask()); - } - System.out.println("-------------------"); return taskMetadataInfos; }, 3, @@ -396,7 +392,7 @@ public List> getTaskMetadataInfos( * @param dataSource datasource to which the tasks belong. null if we don't want to filter * @return Query object for TaskStatusPlus for completed tasks of interest */ - protected Query> createCompletedTaskSummaryStreamingQuery( + private Query> createCompletedTaskSummaryStreamingQuery( Handle handle, DateTime timestamp, @Nullable Integer maxNumStatuses, @@ -444,7 +440,7 @@ protected Query> createCompletedTaskSummaryStreamingQuery( * @param dataSource datasource to which the tasks belong. null if we don't want to filter * @return Query object for completed TaskInfos of interest */ - protected Query> createCompletedTaskStreamingQuery( + private Query> createCompletedTaskStreamingQuery( Handle handle, DateTime timestamp, @Nullable Integer maxNumStatuses, @@ -571,43 +567,43 @@ private String getWhereClauseForActiveStatusesQuery(String dataSource) return sql; } - class TaskMetadataInfoMapperFromPayload implements ResultSetMapper> + private class TaskStatusMapperFromPayload implements ResultSetMapper> { private final ObjectMapper objectMapper; - TaskMetadataInfoMapperFromPayload(ObjectMapper objectMapper) + TaskStatusMapperFromPayload(ObjectMapper objectMapper) { this.objectMapper = objectMapper; } @Override - public TaskInfo map(int index, ResultSet resultSet, StatementContext context) + public TaskInfo map(int index, ResultSet resultSet, StatementContext context) throws SQLException { - return toTaskMetadataInfo(objectMapper, resultSet, true); + return toTaskIdentifierInfo(objectMapper, resultSet, true); } } - class TaskMetadataInfoMapper implements ResultSetMapper> + private class TaskStatusMapper implements ResultSetMapper> { private final ObjectMapper objectMapper; - TaskMetadataInfoMapper(ObjectMapper objectMapper) + TaskStatusMapper(ObjectMapper objectMapper) { this.objectMapper = objectMapper; } @Override - public TaskInfo map(int index, ResultSet resultSet, StatementContext context) + public TaskInfo map(int index, ResultSet resultSet, StatementContext context) throws SQLException { - return toTaskMetadataInfo(objectMapper, resultSet, false); + return toTaskIdentifierInfo(objectMapper, resultSet, false); } } - private TaskInfo toTaskMetadataInfo(ObjectMapper objectMapper, - ResultSet resultSet, - boolean usePayload + private TaskInfo toTaskIdentifierInfo(ObjectMapper objectMapper, + ResultSet resultSet, + boolean usePayload ) throws SQLException { String type; @@ -638,22 +634,22 @@ private TaskInfo toTaskMetadataInfo(ObjectMapper objec throw new SQLException(e); } String datasource = resultSet.getString("datasource"); - TaskMetadata taskMetadata = new TaskMetadata(id, groupId, type); + TaskIdentifier taskIdentifier = new TaskIdentifier(id, groupId, type); - return new TaskInfo<>(id, createdTime, status, datasource, taskMetadata); + return new TaskInfo<>(id, createdTime, status, datasource, taskIdentifier); } - static class TaskMetadataMapper implements ResultSetMapper + static class TaskIdentifierMapper implements ResultSetMapper { private final ObjectMapper objectMapper; - TaskMetadataMapper(ObjectMapper objectMapper) + TaskIdentifierMapper(ObjectMapper objectMapper) { this.objectMapper = objectMapper; } @Override - public TaskMetadata map(int index, ResultSet resultSet, StatementContext context) + public TaskIdentifier map(int index, ResultSet resultSet, StatementContext context) throws SQLException { try { @@ -661,7 +657,7 @@ public TaskMetadata map(int index, ResultSet resultSet, StatementContext context // If field is absent (older task version), use blank string to avoid a loop of migration of such tasks. JsonNode type = payload.get("type"); JsonNode groupId = payload.get("groupId"); - return new TaskMetadata( + return new TaskIdentifier( resultSet.getString("id"), groupId == null ? "" : groupId.asText(), type == null ? "" : type.asText() @@ -963,9 +959,9 @@ public Long getLockId(String entryId, LockType lock) .orElse(null); } - private List fetchTaskMetadatas(String tableName, String id, int limit) + private List fetchTaskMetadatas(String tableName, String id, int limit) { - List taskMetadatas = new ArrayList<>(); + List taskIdentifiers = new ArrayList<>(); connector.retryWithHandle( new HandleCallback() { @@ -979,15 +975,15 @@ public Void withHandle(Handle handle) connector.limitClause(limit) ); Query> query = handle.createQuery(sql); - taskMetadatas.addAll(query.map(taskMetadataMapper).list()); + taskIdentifiers.addAll(query.map(taskIdentifierMapper).list()); return null; } } ); - return taskMetadatas; + return taskIdentifiers; } - private void updateTaskMetadatas(String tasksTable, List taskMetadatas) + private void updateTaskMetadatas(String tasksTable, List taskIdentifiers) { connector.retryWithHandle( new HandleCallback() @@ -997,7 +993,7 @@ public Void withHandle(Handle handle) { Batch batch = handle.createBatch(); String sql = "UPDATE %1$s SET type = '%2$s', group_id = '%3$s' WHERE id = '%4$s'"; - for (TaskMetadata metadata : taskMetadatas) { + for (TaskIdentifier metadata : taskIdentifiers) { batch.add(StringUtils.format(sql, tasksTable, metadata.getType(), metadata.getGroupId(), metadata.getId()) ); } @@ -1010,33 +1006,44 @@ public Void withHandle(Handle handle) @Override - public boolean migrateTaskTable(String tableName) + public boolean populateTaskTypeAndGroupId() { - log.info("Populate fields task and group_id of task entry table [%s] from payload", tableName); + log.info("Populate fields task and group_id of task entry table [%s] from payload", entryTable); String id = ""; int limit = 100; + int count = 0; while (true) { - List taskMetadatas; + List taskIdentifiers; try { - taskMetadatas = fetchTaskMetadatas(tableName, id, limit); + taskIdentifiers = fetchTaskMetadatas(entryTable, id, limit); } catch (Exception e) { log.warn(e, "Task migration failed while reading entries from task table"); return false; } - if (taskMetadatas.isEmpty()) { + if (taskIdentifiers.isEmpty()) { break; } try { - updateTaskMetadatas(tableName, taskMetadatas); + updateTaskMetadatas(entryTable, taskIdentifiers); + count += taskIdentifiers.size(); + log.info("Successfully updated type and groupId for [%d] tasks", count); } catch (Exception e) { log.warn(e, "Task migration failed while updating entries in task table"); return false; } - id = taskMetadatas.get(taskMetadatas.size() - 1).getId(); + id = taskIdentifiers.get(taskIdentifiers.size() - 1).getId(); + + try { + Thread.sleep(1000); + } + catch (InterruptedException e) { + log.info("Interrupted, exiting!"); + Thread.currentThread().interrupt(); + } } - log.info("Task migration for table [%s] successful", tableName); + log.info("Task migration for table [%s] successful", entryTable); return true; } } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 32118e18ea39..3f7e6bfe218f 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -25,9 +25,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexer.TaskIdentifier; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskMetadata; import org.apache.druid.indexer.TaskState; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -52,6 +52,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Random; import java.util.UUID; @@ -484,7 +485,7 @@ public void testMigration() Assert.assertEquals(active + completed, getUnmigratedTaskCount().intValue()); - handler.migrateTaskTable(entryTable); + handler.populateTaskTypeAndGroupId(); Assert.assertEquals(0, getUnmigratedTaskCount().intValue()); } @@ -509,12 +510,12 @@ public void testGetTaskStatusPlusList() taskLookups.put(TaskLookup.TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()); taskLookups.put(TaskLookup.TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, Duration.millis(86400000))); - List>> taskMetadataInfos; + List>> taskMetadataInfos; // BEFORE MIGRATION // Payload based fetch. task type and groupid will be populated - taskMetadataInfos = handler.getTaskMetadataInfos(taskLookups, null, true); + taskMetadataInfos = handler.getTaskStatusList(taskLookups, null, true); Assert.assertEquals(4, taskMetadataInfos.size()); verifyTaskInfoToMetadataInfo(completedUnaltered, taskMetadataInfos, false); verifyTaskInfoToMetadataInfo(completedAltered, taskMetadataInfos, false); @@ -522,7 +523,7 @@ public void testGetTaskStatusPlusList() verifyTaskInfoToMetadataInfo(activeAltered, taskMetadataInfos, false); // New columns based fetch before migration is complete. type and payload are null when altered = false - taskMetadataInfos = handler.getTaskMetadataInfos(taskLookups, null, false); + taskMetadataInfos = handler.getTaskStatusList(taskLookups, null, false); Assert.assertEquals(4, taskMetadataInfos.size()); verifyTaskInfoToMetadataInfo(completedUnaltered, taskMetadataInfos, true); verifyTaskInfoToMetadataInfo(completedAltered, taskMetadataInfos, false); @@ -530,10 +531,10 @@ public void testGetTaskStatusPlusList() verifyTaskInfoToMetadataInfo(activeAltered, taskMetadataInfos, false); // MIGRATION - handler.migrateTaskTable(entryTable); + handler.populateTaskTypeAndGroupId(); // Payload based fetch. task type and groupid will still be populated in tasks tab - taskMetadataInfos = handler.getTaskMetadataInfos(taskLookups, null, true); + taskMetadataInfos = handler.getTaskStatusList(taskLookups, null, true); Assert.assertEquals(4, taskMetadataInfos.size()); verifyTaskInfoToMetadataInfo(completedUnaltered, taskMetadataInfos, false); verifyTaskInfoToMetadataInfo(completedAltered, taskMetadataInfos, false); @@ -541,7 +542,7 @@ public void testGetTaskStatusPlusList() verifyTaskInfoToMetadataInfo(activeAltered, taskMetadataInfos, false); // New columns based fetch after migration is complete. All data must be populated in the tasks table - taskMetadataInfos = handler.getTaskMetadataInfos(taskLookups, null, false); + taskMetadataInfos = handler.getTaskStatusList(taskLookups, null, false); Assert.assertEquals(4, taskMetadataInfos.size()); verifyTaskInfoToMetadataInfo(completedUnaltered, taskMetadataInfos, false); verifyTaskInfoToMetadataInfo(completedAltered, taskMetadataInfos, false); @@ -557,7 +558,9 @@ private Integer getUnmigratedTaskCount() @Override public Integer withHandle(Handle handle) throws SQLException { - String sql = String.format("SELECT COUNT(*) FROM %s WHERE type is NULL or group_id is NULL", entryTable); + String sql = String.format(Locale.ENGLISH, + "SELECT COUNT(*) FROM %s WHERE type is NULL or group_id is NULL", + entryTable); ResultSet resultSet = handle.getConnection().createStatement().executeQuery(sql); resultSet.next(); return resultSet.getInt(1); @@ -616,10 +619,10 @@ private void insertTaskInfo(TaskInfo, Map> t } private void verifyTaskInfoToMetadataInfo(TaskInfo, Map> taskInfo, - List>> taskMetadataInfos, + List>> taskMetadataInfos, boolean nullNewColumns) { - for (TaskInfo> taskMetadataInfo : taskMetadataInfos) { + for (TaskInfo> taskMetadataInfo : taskMetadataInfos) { if (taskMetadataInfo.getId().equals(taskInfo.getId())) { verifyTaskInfoToMetadataInfo(taskInfo, taskMetadataInfo, nullNewColumns); } @@ -629,7 +632,7 @@ private void verifyTaskInfoToMetadataInfo(TaskInfo, Map, Map> taskInfo, - TaskInfo> taskMetadataInfo, + TaskInfo> taskMetadataInfo, boolean nullNewColumns) { Assert.assertEquals(taskInfo.getId(), taskMetadataInfo.getId()); @@ -639,14 +642,14 @@ private void verifyTaskInfoToMetadataInfo(TaskInfo, Map task = taskInfo.getTask(); - TaskMetadata taskMetadata = taskMetadataInfo.getTask(); - Assert.assertEquals(task.get("id"), taskMetadata.getId()); + TaskIdentifier taskIdentifier = taskMetadataInfo.getTask(); + Assert.assertEquals(task.get("id"), taskIdentifier.getId()); if (nullNewColumns) { - Assert.assertNull(taskMetadata.getGroupId()); - Assert.assertNull(taskMetadata.getType()); + Assert.assertNull(taskIdentifier.getGroupId()); + Assert.assertNull(taskIdentifier.getType()); } else { - Assert.assertEquals(task.get("groupId"), taskMetadata.getGroupId()); - Assert.assertEquals(task.get("type"), taskMetadata.getType()); + Assert.assertEquals(task.get("groupId"), taskIdentifier.getGroupId()); + Assert.assertEquals(task.get("type"), taskIdentifier.getType()); } } From aa827871c5f19d5e410c37c5bf32aeaa24a5f36c Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 16 Jun 2022 13:10:38 +0530 Subject: [PATCH 15/16] Refactor for cleaner API --- .../MetadataStorageActionHandler.java | 30 +------------ .../overlord/MetadataTaskStorage.java | 25 ++--------- .../SQLMetadataStorageActionHandler.java | 45 +++++++++++++------ .../SQLMetadataStorageActionHandlerTest.java | 7 ++- 4 files changed, 42 insertions(+), 65 deletions(-) diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index 099961c0bed6..5e161e9cdd46 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -104,8 +104,7 @@ List> getTaskInfos( ); /** - * Returns the statuses of the specified tasks. Implementations of this method must not - * read the task payload from the underlying storage as it may increase memory usage. + * Returns the statuses of the specified tasks. * * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata @@ -120,25 +119,6 @@ List> getTaskStatusList( @Nullable String datasource ); - /** - * Returns the statuses of the specified tasks. Implementations of this method may - * read from the corresponding task payloads to retrieve task information. - * - * This method is deprecated and {@link #getTaskStatusList} should be used instead. - * - * If {@code taskLookups} includes {@link TaskLookupType#ACTIVE}, it returns all active tasks in the metadata store. - * If {@code taskLookups} includes {@link TaskLookupType#COMPLETE}, it returns all complete tasks in the metadata - * store. For complete tasks, additional filters in {@code CompleteTaskLookup} can be applied. - * All lookups should be processed atomically if more than one lookup is given. - * - * @param taskLookups task lookup type and filters. - * @param datasource datasource filter - */ - List> getTaskStatusListFromPayload( - Map taskLookups, - @Nullable String datasource - ); - default List> getTaskInfos( TaskLookup taskLookup, @Nullable String datasource @@ -214,12 +194,4 @@ default List> getTaskInfos( @Nullable Long getLockId(String entryId, LockType lock); - /** - * Utility to migrate existing tasks to the new schema by populating type and groupId - * - * To be kicked off in a separate thread at MetadataTaskStorage startup. - * - * @return true if migration was successful - */ - boolean populateTaskTypeAndGroupId(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 01e7a6594e1d..19aec9dab950 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -64,8 +64,6 @@ public class MetadataTaskStorage implements TaskStorage { - private Future taskMigrationCompleteFuture; - private static final MetadataStorageActionHandlerTypes TASK_TYPES = new MetadataStorageActionHandlerTypes() { @Override @@ -124,10 +122,6 @@ public void start() { // Case where active tasks metadataStorageConnector.createTaskTables(); - ExecutorService executorService = Executors.newSingleThreadExecutor(); - taskMigrationCompleteFuture = executorService.submit( - () -> handler.populateTaskTypeAndGroupId() - ); } @LifecycleStop @@ -252,22 +246,11 @@ public List getTaskStatusPlusList( ) { Map processedTaskLookups = processTaskLookups(taskLookups); - boolean fetchPayload = true; - if (taskMigrationCompleteFuture.isDone()) { - try { - fetchPayload = !taskMigrationCompleteFuture.get(); - } - catch (Exception e) { - log.info(e, "Exception getting task migration future"); - } - } - List> taskMetadataInfos = fetchPayload - ? handler.getTaskStatusListFromPayload(processedTaskLookups, datasource) - : handler.getTaskStatusList(processedTaskLookups, datasource); return Collections.unmodifiableList( - taskMetadataInfos.stream() - .map(TaskStatusPlus::fromTaskIdentifierInfo) - .collect(Collectors.toList()) + handler.getTaskStatusList(processedTaskLookups, datasource) + .stream() + .map(TaskStatusPlus::fromTaskIdentifierInfo) + .collect(Collectors.toList()) ); } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index f3830b7cb3cb..88e66c417cc1 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Maps; @@ -57,6 +58,9 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; public abstract class SQLMetadataStorageActionHandler implements MetadataStorageActionHandler @@ -80,6 +84,8 @@ public abstract class SQLMetadataStorageActionHandler taskMigrationCompleteFuture; + @SuppressWarnings("PMD.UnnecessaryFullyQualifiedName") public SQLMetadataStorageActionHandler( final SQLMetadataConnector connector, @@ -325,25 +331,32 @@ public List> getTaskInfos( ); } - @Override - public List> getTaskStatusListFromPayload( - Map taskLookups, - @Nullable String dataSource - ) - { - return getTaskStatusList(taskLookups, dataSource, true); - } - @Override public List> getTaskStatusList( Map taskLookups, @Nullable String dataSource ) { - return getTaskStatusList(taskLookups, dataSource, false); + if (taskMigrationCompleteFuture == null) { + ExecutorService executorService = Executors.newSingleThreadExecutor(); + taskMigrationCompleteFuture = executorService.submit( + () -> populateTaskTypeAndGroupId() + ); + } + boolean fetchPayload = true; + if (taskMigrationCompleteFuture.isDone()) { + try { + fetchPayload = !taskMigrationCompleteFuture.get(); + } + catch (Exception e) { + log.info(e, "Exception getting task migration future"); + } + } + return getTaskStatusList(taskLookups, dataSource, fetchPayload); } - public List> getTaskStatusList( + @VisibleForTesting + List> getTaskStatusList( Map taskLookups, @Nullable String dataSource, boolean fetchPayload @@ -1004,9 +1017,13 @@ public Void withHandle(Handle handle) ); } - - @Override - public boolean populateTaskTypeAndGroupId() + /** + * Utility to migrate existing tasks to the new schema by populating type and groupId synchronously + * + * @return true if successful + */ + @VisibleForTesting + boolean populateTaskTypeAndGroupId() { log.info("Populate fields task and group_id of task entry table [%s] from payload", entryTable); String id = ""; diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 3f7e6bfe218f..99681506618b 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -490,8 +490,13 @@ public void testMigration() Assert.assertEquals(0, getUnmigratedTaskCount().intValue()); } - @Test public void testGetTaskStatusPlusList() + { + + } + + @Test + public void testGetTaskStatusPlusListInternal() { // SETUP TaskInfo, Map> activeUnaltered = createRandomTaskInfo(true); From f1c1151eab07e785e98a127dbb8a38630d1aa1af Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 16 Jun 2022 16:45:39 +0530 Subject: [PATCH 16/16] migration at startup and fix checkstyle --- .../metadata/MetadataStorageActionHandler.java | 5 +++++ .../indexing/overlord/MetadataTaskStorage.java | 7 ++----- .../SQLMetadataStorageActionHandler.java | 17 ++++++++++------- .../SQLMetadataStorageActionHandlerTest.java | 5 ----- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index 5e161e9cdd46..ca6ede6d32f0 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -194,4 +194,9 @@ default List> getTaskInfos( @Nullable Long getLockId(String entryId, LockType lock); + /** + * Utility to migrate existing tasks to the new schema by populating type and groupId asynchronously + */ + void populateTaskTypeAndGroupIdAsync(); + } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 19aec9dab950..7c7fbd3f53c8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -27,7 +27,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.inject.Inject; -import org.apache.druid.indexer.TaskIdentifier; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; @@ -56,9 +55,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.stream.Collectors; public class MetadataTaskStorage implements TaskStorage @@ -120,8 +116,9 @@ public MetadataTaskStorage( @LifecycleStart public void start() { - // Case where active tasks metadataStorageConnector.createTaskTables(); + // begins migration of existing tasks to new schema + handler.populateTaskTypeAndGroupIdAsync(); } @LifecycleStop diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index 88e66c417cc1..436af7ad694a 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -337,14 +337,8 @@ public List> getTaskStatusList( @Nullable String dataSource ) { - if (taskMigrationCompleteFuture == null) { - ExecutorService executorService = Executors.newSingleThreadExecutor(); - taskMigrationCompleteFuture = executorService.submit( - () -> populateTaskTypeAndGroupId() - ); - } boolean fetchPayload = true; - if (taskMigrationCompleteFuture.isDone()) { + if (taskMigrationCompleteFuture != null && taskMigrationCompleteFuture.isDone()) { try { fetchPayload = !taskMigrationCompleteFuture.get(); } @@ -1017,6 +1011,15 @@ public Void withHandle(Handle handle) ); } + @Override + public void populateTaskTypeAndGroupIdAsync() + { + ExecutorService executorService = Executors.newSingleThreadExecutor(); + taskMigrationCompleteFuture = executorService.submit( + () -> populateTaskTypeAndGroupId() + ); + } + /** * Utility to migrate existing tasks to the new schema by populating type and groupId synchronously * diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 99681506618b..f89ae9b0ebda 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -490,11 +490,6 @@ public void testMigration() Assert.assertEquals(0, getUnmigratedTaskCount().intValue()); } - public void testGetTaskStatusPlusList() - { - - } - @Test public void testGetTaskStatusPlusListInternal() {