Skip to content
This repository has been archived by the owner on Dec 13, 2023. It is now read-only.

Commit

Permalink
Refactor task search v1 and workflow search v1 APIs to return documen…
Browse files Browse the repository at this point in the history
…ts from ES directly (#3283)
  • Loading branch information
jxu-nflx authored Oct 17, 2022
1 parent 49a1076 commit 145ab53
Show file tree
Hide file tree
Showing 10 changed files with 294 additions and 103 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -439,7 +439,7 @@ && getStatus() == that.getStatus()
&& Objects.equals(getTaskDefName(), that.getTaskDefName())
&& getTaskType().equals(that.getTaskType())
&& getTaskId().equals(that.getTaskId())
&& getDomain().equals(that.getDomain());
&& Objects.equals(getDomain(), that.getDomain());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -623,11 +623,21 @@ public SearchResult<String> searchWorkflows(
return indexDAO.searchWorkflows(query, freeText, start, count, sort);
}

public SearchResult<WorkflowSummary> searchWorkflowSummary(
String query, String freeText, int start, int count, List<String> sort) {
return indexDAO.searchWorkflowSummary(query, freeText, start, count, sort);
}

public SearchResult<String> searchTasks(
String query, String freeText, int start, int count, List<String> sort) {
return indexDAO.searchTasks(query, freeText, start, count, sort);
}

public SearchResult<TaskSummary> searchTaskSummary(
String query, String freeText, int start, int count, List<String> sort) {
return indexDAO.searchTaskSummary(query, freeText, start, count, sort);
}

public List<TaskExecLog> getTaskExecutionLogs(String taskId) {
return properties.isTaskExecLogIndexingEnabled()
? indexDAO.getTaskExecutionLogs(taskId)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,12 +55,24 @@ public SearchResult<String> searchWorkflows(
return new SearchResult<>(0, Collections.emptyList());
}

@Override
public SearchResult<WorkflowSummary> searchWorkflowSummary(
String query, String freeText, int start, int count, List<String> sort) {
return new SearchResult<>(0, Collections.emptyList());
}

@Override
public SearchResult<String> searchTasks(
String query, String freeText, int start, int count, List<String> sort) {
return new SearchResult<>(0, Collections.emptyList());
}

@Override
public SearchResult<TaskSummary> searchTaskSummary(
String query, String freeText, int start, int count, List<String> sort) {
return new SearchResult<>(0, Collections.emptyList());
}

@Override
public void removeWorkflow(String workflowId) {}

Expand Down
24 changes: 23 additions & 1 deletion core/src/main/java/com/netflix/conductor/dao/IndexDAO.java
Original file line number Diff line number Diff line change
Expand Up @@ -65,17 +65,39 @@ public interface IndexDAO {
SearchResult<String> searchWorkflows(
String query, String freeText, int start, int count, List<String> sort);

/**
* @param query SQL like query for workflow search parameters.
* @param freeText Additional query in free text. Lucene syntax
* @param start start start index for pagination
* @param count count # of workflow ids to be returned
* @param sort sort options
* @return List of workflows for the matching query
*/
SearchResult<WorkflowSummary> searchWorkflowSummary(
String query, String freeText, int start, int count, List<String> sort);

/**
* @param query SQL like query for task search parameters.
* @param freeText Additional query in free text. Lucene syntax
* @param start start start index for pagination
* @param count count # of task ids to be returned
* @param sort sort options
* @return List of workflow ids for the matching query
* @return List of task ids for the matching query
*/
SearchResult<String> searchTasks(
String query, String freeText, int start, int count, List<String> sort);

/**
* @param query SQL like query for task search parameters.
* @param freeText Additional query in free text. Lucene syntax
* @param start start start index for pagination
* @param count count # of task ids to be returned
* @param sort sort options
* @return List of tasks for the matching query
*/
SearchResult<TaskSummary> searchTaskSummary(
String query, String freeText, int start, int count, List<String> sort);

/**
* Remove the workflow index
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -384,28 +384,7 @@ public void removeWorkflow(String workflowId, boolean archiveWorkflow) {

public SearchResult<WorkflowSummary> search(
String query, String freeText, int start, int size, List<String> sortOptions) {

SearchResult<String> result =
executionDAOFacade.searchWorkflows(query, freeText, start, size, sortOptions);
List<WorkflowSummary> workflows =
result.getResults().stream()
.parallel()
.map(
workflowId -> {
try {
return new WorkflowSummary(
executionDAOFacade.getWorkflow(workflowId, false));
} catch (Exception e) {
LOGGER.error(
"Error fetching workflow by id: {}", workflowId, e);
return null;
}
})
.filter(Objects::nonNull)
.collect(Collectors.toList());
int missing = result.getResults().size() - workflows.size();
long totalHits = result.getTotalHits() - missing;
return new SearchResult<>(totalHits, workflows);
return executionDAOFacade.searchWorkflowSummary(query, freeText, start, size, sortOptions);
}

public SearchResult<Workflow> searchV2(
Expand Down Expand Up @@ -436,7 +415,7 @@ public SearchResult<Workflow> searchV2(
public SearchResult<WorkflowSummary> searchWorkflowByTasks(
String query, String freeText, int start, int size, List<String> sortOptions) {
SearchResult<TaskSummary> taskSummarySearchResult =
searchTasks(query, freeText, start, size, sortOptions);
searchTaskSummary(query, freeText, start, size, sortOptions);
List<WorkflowSummary> workflowSummaries =
taskSummarySearchResult.getResults().stream()
.parallel()
Expand Down Expand Up @@ -514,14 +493,20 @@ public SearchResult<TaskSummary> searchTasks(
return new SearchResult<>(totalHits, workflows);
}

public SearchResult<TaskSummary> searchTaskSummary(
String query, String freeText, int start, int size, List<String> sortOptions) {
return executionDAOFacade.searchTaskSummary(query, freeText, start, size, sortOptions);
}

public SearchResult<TaskSummary> getSearchTasks(
String query,
String freeText,
int start,
/*@Max(value = MAX_SEARCH_SIZE, message = "Cannot return more than {value} workflows." +
" Please use pagination.")*/ int size,
String sortString) {
return searchTasks(query, freeText, start, size, Utils.convertStringToList(sortString));
return searchTaskSummary(
query, freeText, start, size, Utils.convertStringToList(sortString));
}

public SearchResult<Task> getSearchTasksV2(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,12 +86,13 @@ public void setup() {

@Test
public void workflowSearchTest() {
when(executionDAOFacade.searchWorkflows("query", "*", 0, 2, sort))
when(executionDAOFacade.searchWorkflowSummary("query", "*", 0, 2, sort))
.thenReturn(
new SearchResult<>(
2,
Arrays.asList(
workflow1.getWorkflowId(), workflow2.getWorkflowId())));
new WorkflowSummary(workflow1),
new WorkflowSummary(workflow2))));
when(executionDAOFacade.getWorkflow(workflow1.getWorkflowId(), false))
.thenReturn(workflow1);
when(executionDAOFacade.getWorkflow(workflow2.getWorkflowId(), false))
Expand All @@ -104,25 +105,6 @@ public void workflowSearchTest() {
assertEquals(workflow2.getWorkflowId(), searchResult.getResults().get(1).getWorkflowId());
}

@Test
public void workflowSearchExceptionTest() {
when(executionDAOFacade.searchWorkflows("query", "*", 0, 2, sort))
.thenReturn(
new SearchResult<>(
2,
Arrays.asList(
workflow1.getWorkflowId(), workflow2.getWorkflowId())));
when(executionDAOFacade.getWorkflow(workflow1.getWorkflowId(), false))
.thenReturn(workflow1);
when(executionDAOFacade.getWorkflow(workflow2.getWorkflowId(), false))
.thenThrow(new RuntimeException());
SearchResult<WorkflowSummary> searchResult =
executionService.search("query", "*", 0, 2, sort);
assertEquals(1, searchResult.getTotalHits());
assertEquals(1, searchResult.getResults().size());
assertEquals(workflow1.getWorkflowId(), searchResult.getResults().get(0).getWorkflowId());
}

@Test
public void workflowSearchV2Test() {
when(executionDAOFacade.searchWorkflows("query", "*", 0, 2, sort))
Expand Down Expand Up @@ -159,14 +141,13 @@ public void workflowSearchV2ExceptionTest() {

@Test
public void workflowSearchByTasksTest() {
when(executionDAOFacade.searchTasks("query", "*", 0, 2, sort))
when(executionDAOFacade.searchTaskSummary("query", "*", 0, 2, sort))
.thenReturn(
new SearchResult<>(
2,
Arrays.asList(
taskWorkflow1.getTaskId(), taskWorkflow2.getTaskId())));
when(executionDAOFacade.getTask(taskWorkflow1.getTaskId())).thenReturn(taskWorkflow1);
when(executionDAOFacade.getTask(taskWorkflow2.getTaskId())).thenReturn(taskWorkflow2);
new TaskSummary(taskWorkflow1),
new TaskSummary(taskWorkflow2))));
when(executionDAOFacade.getWorkflow(workflow1.getWorkflowId(), false))
.thenReturn(workflow1);
when(executionDAOFacade.getWorkflow(workflow2.getWorkflowId(), false))
Expand All @@ -181,17 +162,17 @@ public void workflowSearchByTasksTest() {

@Test
public void workflowSearchByTasksExceptionTest() {
when(executionDAOFacade.searchTasks("query", "*", 0, 2, sort))
when(executionDAOFacade.searchTaskSummary("query", "*", 0, 2, sort))
.thenReturn(
new SearchResult<>(
2,
Arrays.asList(
taskWorkflow1.getTaskId(), taskWorkflow2.getTaskId())));
when(executionDAOFacade.getTask(taskWorkflow1.getTaskId())).thenReturn(taskWorkflow1);
when(executionDAOFacade.getTask(taskWorkflow2.getTaskId()))
.thenThrow(new RuntimeException());
new TaskSummary(taskWorkflow1),
new TaskSummary(taskWorkflow2))));
when(executionDAOFacade.getWorkflow(workflow1.getWorkflowId(), false))
.thenReturn(workflow1);
when(executionDAOFacade.getTask(workflow2.getWorkflowId()))
.thenThrow(new RuntimeException());
SearchResult<WorkflowSummary> searchResult =
executionService.searchWorkflowByTasks("query", "*", 0, 2, sort);
assertEquals(1, searchResult.getTotalHits());
Expand Down Expand Up @@ -240,11 +221,10 @@ public void workflowSearchByTasksV2ExceptionTest() {

@Test
public void TaskSearchTest() {
List<String> taskList = Arrays.asList(taskWorkflow1.getTaskId(), taskWorkflow2.getTaskId());
when(executionDAOFacade.searchTasks("query", "*", 0, 2, sort))
List<TaskSummary> taskList =
Arrays.asList(new TaskSummary(taskWorkflow1), new TaskSummary(taskWorkflow2));
when(executionDAOFacade.searchTaskSummary("query", "*", 0, 2, sort))
.thenReturn(new SearchResult<>(2, taskList));
when(executionDAOFacade.getTask(taskWorkflow1.getTaskId())).thenReturn(taskWorkflow1);
when(executionDAOFacade.getTask(taskWorkflow2.getTaskId())).thenReturn(taskWorkflow2);
SearchResult<TaskSummary> searchResult =
executionService.getSearchTasks("query", "*", 0, 2, "Sort");
assertEquals(2, searchResult.getTotalHits());
Expand All @@ -253,21 +233,6 @@ public void TaskSearchTest() {
assertEquals(taskWorkflow2.getTaskId(), searchResult.getResults().get(1).getTaskId());
}

@Test
public void TaskSearchExceptionTest() {
List<String> taskList = Arrays.asList(taskWorkflow1.getTaskId(), taskWorkflow2.getTaskId());
when(executionDAOFacade.searchTasks("query", "*", 0, 2, sort))
.thenReturn(new SearchResult<>(2, taskList));
when(executionDAOFacade.getTask(taskWorkflow1.getTaskId())).thenReturn(taskWorkflow1);
when(executionDAOFacade.getTask(taskWorkflow2.getTaskId()))
.thenThrow(new RuntimeException());
SearchResult<TaskSummary> searchResult =
executionService.getSearchTasks("query", "*", 0, 2, "Sort");
assertEquals(1, searchResult.getTotalHits());
assertEquals(1, searchResult.getResults().size());
assertEquals(taskWorkflow1.getTaskId(), searchResult.getResults().get(0).getTaskId());
}

@Test
public void TaskSearchV2Test() {
when(executionDAOFacade.searchTasks("query", "*", 0, 2, sort))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@
import com.netflix.conductor.es6.dao.query.parser.internal.ParserException;
import com.netflix.conductor.metrics.Monitors;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.type.MapType;
import com.fasterxml.jackson.databind.type.TypeFactory;
Expand Down Expand Up @@ -657,7 +658,21 @@ private void updateWithRetry(BulkRequestBuilderWrapper request, String docType)
@Override
public SearchResult<String> searchWorkflows(
String query, String freeText, int start, int count, List<String> sort) {
return search(query, start, count, sort, freeText, WORKFLOW_DOC_TYPE);
return search(query, start, count, sort, freeText, WORKFLOW_DOC_TYPE, true, String.class);
}

@Override
public SearchResult<WorkflowSummary> searchWorkflowSummary(
String query, String freeText, int start, int count, List<String> sort) {
return search(
query,
start,
count,
sort,
freeText,
WORKFLOW_DOC_TYPE,
false,
WorkflowSummary.class);
}

@Override
Expand All @@ -668,7 +683,13 @@ public long getWorkflowCount(String query, String freeText) {
@Override
public SearchResult<String> searchTasks(
String query, String freeText, int start, int count, List<String> sort) {
return search(query, start, count, sort, freeText, TASK_DOC_TYPE);
return search(query, start, count, sort, freeText, TASK_DOC_TYPE, true, String.class);
}

@Override
public SearchResult<TaskSummary> searchTaskSummary(
String query, String freeText, int start, int count, List<String> sort) {
return search(query, start, count, sort, freeText, TASK_DOC_TYPE, false, TaskSummary.class);
}

@Override
Expand Down Expand Up @@ -776,13 +797,15 @@ private long count(String structuredQuery, String freeTextQuery, String docType)
}
}

private SearchResult<String> search(
private <T> SearchResult<T> search(
String structuredQuery,
int start,
int size,
List<String> sortOptions,
String freeTextQuery,
String docType) {
String docType,
boolean idOnly,
Class<T> clazz) {
try {
docType = StringUtils.isBlank(docTypeOverride) ? docType : docTypeOverride;
BoolQueryBuilder fq = boolQueryBuilder(structuredQuery, freeTextQuery);
Expand All @@ -791,13 +814,13 @@ private SearchResult<String> search(
.prepareSearch(getIndexName(docType))
.setQuery(fq)
.setTypes(docType)
.storedFields("_id")
.setFrom(start)
.setSize(size);

if (idOnly) {
srb.storedFields("_id");
}
addSortOptions(srb, sortOptions);

return mapSearchResult(srb.get());
return mapSearchResult(srb.get(), idOnly, clazz);
} catch (ParserException e) {
throw new TransientException(e.getMessage(), e);
}
Expand All @@ -820,10 +843,34 @@ private void addSortOptions(SearchRequestBuilder srb, List<String> sortOptions)
}
}

private SearchResult<String> mapSearchResult(SearchResponse response) {
List<String> result = new LinkedList<>();
response.getHits().forEach(hit -> result.add(hit.getId()));
long count = response.getHits().getTotalHits();
private <T> SearchResult<T> mapSearchResult(
SearchResponse response, boolean idOnly, Class<T> clazz) {
SearchHits searchHits = response.getHits();
long count = searchHits.getTotalHits();
List<T> result;
if (idOnly) {
result =
Arrays.stream(searchHits.getHits())
.map(hit -> clazz.cast(hit.getId()))
.collect(Collectors.toList());
} else {
result =
Arrays.stream(searchHits.getHits())
.map(
hit -> {
try {
return objectMapper.readValue(
hit.getSourceAsString(), clazz);
} catch (JsonProcessingException e) {
LOGGER.error(
"Failed to de-serialize elasticsearch from source: {}",
hit.getSourceAsString(),
e);
}
return null;
})
.collect(Collectors.toList());
}
return new SearchResult<>(count, result);
}

Expand Down
Loading

0 comments on commit 145ab53

Please sign in to comment.