mirror of
https://github.com/open-metadata/OpenMetadata.git
synced 2025-08-26 18:06:03 +00:00
Parallelize the search indexing process (#18445)
* Parallelize the indexing process * Parallelize indexing (cherry picked from commit 0114007a228936748b695cdc156f84fc5aa4f309)
This commit is contained in:
parent
669b09e6c5
commit
b7f3157986
@ -12,6 +12,11 @@ import java.util.HashMap;
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
|
import java.util.concurrent.Future;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
import lombok.Getter;
|
import lombok.Getter;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
import org.apache.commons.lang.exception.ExceptionUtils;
|
import org.apache.commons.lang.exception.ExceptionUtils;
|
||||||
@ -107,7 +112,9 @@ public class SearchIndexApp extends AbstractNativeApplication {
|
|||||||
private Sink searchIndexSink;
|
private Sink searchIndexSink;
|
||||||
|
|
||||||
@Getter EventPublisherJob jobData;
|
@Getter EventPublisherJob jobData;
|
||||||
|
private final Object jobDataLock = new Object(); // Dedicated final lock object
|
||||||
private volatile boolean stopped = false;
|
private volatile boolean stopped = false;
|
||||||
|
@Getter private volatile ExecutorService executorService;
|
||||||
|
|
||||||
public SearchIndexApp(CollectionDAO collectionDAO, SearchRepository searchRepository) {
|
public SearchIndexApp(CollectionDAO collectionDAO, SearchRepository searchRepository) {
|
||||||
super(collectionDAO, searchRepository);
|
super(collectionDAO, searchRepository);
|
||||||
@ -131,19 +138,12 @@ public class SearchIndexApp extends AbstractNativeApplication {
|
|||||||
try {
|
try {
|
||||||
initializeJob();
|
initializeJob();
|
||||||
LOG.info("Executing Reindexing Job with JobData : {}", jobData);
|
LOG.info("Executing Reindexing Job with JobData : {}", jobData);
|
||||||
// Update Job Status
|
|
||||||
jobData.setStatus(EventPublisherJob.Status.RUNNING);
|
jobData.setStatus(EventPublisherJob.Status.RUNNING);
|
||||||
|
|
||||||
// Make recreate as false for onDemand
|
|
||||||
String runType =
|
String runType =
|
||||||
(String) jobExecutionContext.getJobDetail().getJobDataMap().get("triggerType");
|
(String) jobExecutionContext.getJobDetail().getJobDataMap().get("triggerType");
|
||||||
|
|
||||||
// Schedule Run has re-create set to false
|
|
||||||
if (!runType.equals(ON_DEMAND_JOB)) {
|
if (!runType.equals(ON_DEMAND_JOB)) {
|
||||||
jobData.setRecreateIndex(false);
|
jobData.setRecreateIndex(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Run ReIndexing
|
|
||||||
performReindex(jobExecutionContext);
|
performReindex(jobExecutionContext);
|
||||||
} catch (Exception ex) {
|
} catch (Exception ex) {
|
||||||
IndexingError indexingError =
|
IndexingError indexingError =
|
||||||
@ -157,7 +157,6 @@ public class SearchIndexApp extends AbstractNativeApplication {
|
|||||||
jobData.setStatus(EventPublisherJob.Status.RUNNING);
|
jobData.setStatus(EventPublisherJob.Status.RUNNING);
|
||||||
jobData.setFailure(indexingError);
|
jobData.setFailure(indexingError);
|
||||||
} finally {
|
} finally {
|
||||||
// Send update
|
|
||||||
sendUpdates(jobExecutionContext);
|
sendUpdates(jobExecutionContext);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -176,7 +175,6 @@ public class SearchIndexApp extends AbstractNativeApplication {
|
|||||||
// Remove any Stale Jobs
|
// Remove any Stale Jobs
|
||||||
cleanUpStaleJobsFromRuns();
|
cleanUpStaleJobsFromRuns();
|
||||||
|
|
||||||
// Initialize New Job
|
|
||||||
int totalRecords = getTotalRequestToProcess(jobData.getEntities(), collectionDAO);
|
int totalRecords = getTotalRequestToProcess(jobData.getEntities(), collectionDAO);
|
||||||
this.jobData.setStats(
|
this.jobData.setStats(
|
||||||
new Stats()
|
new Stats()
|
||||||
@ -223,16 +221,11 @@ public class SearchIndexApp extends AbstractNativeApplication {
|
|||||||
public void updateRecordToDb(JobExecutionContext jobExecutionContext) {
|
public void updateRecordToDb(JobExecutionContext jobExecutionContext) {
|
||||||
AppRunRecord appRecord = getJobRecord(jobExecutionContext);
|
AppRunRecord appRecord = getJobRecord(jobExecutionContext);
|
||||||
|
|
||||||
// Update Run Record with Status
|
|
||||||
appRecord.setStatus(AppRunRecord.Status.fromValue(jobData.getStatus().value()));
|
appRecord.setStatus(AppRunRecord.Status.fromValue(jobData.getStatus().value()));
|
||||||
|
|
||||||
// Update Error
|
|
||||||
if (jobData.getFailure() != null) {
|
if (jobData.getFailure() != null) {
|
||||||
appRecord.setFailureContext(
|
appRecord.setFailureContext(
|
||||||
new FailureContext().withAdditionalProperty("failure", jobData.getFailure()));
|
new FailureContext().withAdditionalProperty("failure", jobData.getFailure()));
|
||||||
}
|
}
|
||||||
|
|
||||||
// Update Stats
|
|
||||||
if (jobData.getStats() != null) {
|
if (jobData.getStats() != null) {
|
||||||
appRecord.setSuccessContext(
|
appRecord.setSuccessContext(
|
||||||
new SuccessContext().withAdditionalProperty("stats", jobData.getStats()));
|
new SuccessContext().withAdditionalProperty("stats", jobData.getStats()));
|
||||||
@ -241,45 +234,110 @@ public class SearchIndexApp extends AbstractNativeApplication {
|
|||||||
pushAppStatusUpdates(jobExecutionContext, appRecord, true);
|
pushAppStatusUpdates(jobExecutionContext, appRecord, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void performReindex(JobExecutionContext jobExecutionContext) throws SearchIndexException {
|
private void performReindex(JobExecutionContext jobExecutionContext) {
|
||||||
Map<String, Object> contextData = new HashMap<>();
|
if (jobData.getStats() == null) {
|
||||||
|
jobData.setStats(new Stats());
|
||||||
|
}
|
||||||
|
|
||||||
|
if (executorService == null || executorService.isShutdown() || executorService.isTerminated()) {
|
||||||
|
int numThreads =
|
||||||
|
Math.min(paginatedSources.size(), Runtime.getRuntime().availableProcessors());
|
||||||
|
this.executorService = Executors.newFixedThreadPool(numThreads);
|
||||||
|
LOG.debug("Initialized new ExecutorService with {} threads.", numThreads);
|
||||||
|
}
|
||||||
|
List<Future<?>> futures = new ArrayList<>();
|
||||||
|
|
||||||
for (Source paginatedSource : paginatedSources) {
|
for (Source paginatedSource : paginatedSources) {
|
||||||
List<String> entityName = new ArrayList<>();
|
Future<?> future =
|
||||||
reCreateIndexes(paginatedSource.getEntityType());
|
executorService.submit(
|
||||||
contextData.put(ENTITY_TYPE_KEY, paginatedSource.getEntityType());
|
() -> {
|
||||||
Object resultList;
|
String entityType = paginatedSource.getEntityType();
|
||||||
while (!stopped && !paginatedSource.isDone()) {
|
Map<String, Object> contextData = new HashMap<>();
|
||||||
|
contextData.put(ENTITY_TYPE_KEY, entityType);
|
||||||
try {
|
try {
|
||||||
resultList = paginatedSource.readNext(null);
|
reCreateIndexes(entityType);
|
||||||
if (!TIME_SERIES_ENTITIES.contains(paginatedSource.getEntityType())) {
|
contextData.put(ENTITY_TYPE_KEY, entityType);
|
||||||
entityName =
|
|
||||||
|
while (!paginatedSource.isDone()) {
|
||||||
|
Object resultList = paginatedSource.readNext(null);
|
||||||
|
if (resultList == null) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!TIME_SERIES_ENTITIES.contains(entityType)) {
|
||||||
|
List<String> entityNames =
|
||||||
getEntityNameFromEntity(
|
getEntityNameFromEntity(
|
||||||
(ResultList<? extends EntityInterface>) resultList,
|
(ResultList<? extends EntityInterface>) resultList, entityType);
|
||||||
paginatedSource.getEntityType());
|
contextData.put(ENTITY_NAME_LIST_KEY, entityNames);
|
||||||
contextData.put(ENTITY_NAME_LIST_KEY, entityName);
|
|
||||||
processEntity(
|
processEntity(
|
||||||
(ResultList<? extends EntityInterface>) resultList, contextData, paginatedSource);
|
(ResultList<? extends EntityInterface>) resultList,
|
||||||
|
contextData,
|
||||||
|
paginatedSource);
|
||||||
} else {
|
} else {
|
||||||
entityName =
|
List<String> entityNames =
|
||||||
getEntityNameFromEntityTimeSeries(
|
getEntityNameFromEntityTimeSeries(
|
||||||
(ResultList<? extends EntityTimeSeriesInterface>) resultList,
|
(ResultList<? extends EntityTimeSeriesInterface>) resultList,
|
||||||
paginatedSource.getEntityType());
|
entityType);
|
||||||
contextData.put(ENTITY_NAME_LIST_KEY, entityName);
|
contextData.put(ENTITY_NAME_LIST_KEY, entityNames);
|
||||||
processEntityTimeSeries(
|
processEntityTimeSeries(
|
||||||
(ResultList<? extends EntityTimeSeriesInterface>) resultList,
|
(ResultList<? extends EntityTimeSeriesInterface>) resultList,
|
||||||
contextData,
|
contextData,
|
||||||
paginatedSource);
|
paginatedSource);
|
||||||
}
|
}
|
||||||
|
synchronized (jobDataLock) {
|
||||||
} catch (SearchIndexException rx) {
|
updateStats(entityType, paginatedSource.getStats());
|
||||||
jobData.setStatus(EventPublisherJob.Status.RUNNING);
|
}
|
||||||
jobData.setFailure(rx.getIndexingError());
|
|
||||||
paginatedSource.updateStats(
|
|
||||||
rx.getIndexingError().getSuccessCount(), rx.getIndexingError().getFailedCount());
|
|
||||||
} finally {
|
|
||||||
updateStats(paginatedSource.getEntityType(), paginatedSource.getStats());
|
|
||||||
sendUpdates(jobExecutionContext);
|
sendUpdates(jobExecutionContext);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
} catch (SearchIndexException e) {
|
||||||
|
synchronized (jobDataLock) {
|
||||||
|
jobData.setStatus(EventPublisherJob.Status.RUNNING);
|
||||||
|
jobData.setFailure(e.getIndexingError());
|
||||||
|
paginatedSource.updateStats(
|
||||||
|
e.getIndexingError().getSuccessCount(),
|
||||||
|
e.getIndexingError().getFailedCount());
|
||||||
|
updateStats(entityType, paginatedSource.getStats());
|
||||||
|
}
|
||||||
|
sendUpdates(jobExecutionContext);
|
||||||
|
} catch (Exception e) {
|
||||||
|
synchronized (jobDataLock) {
|
||||||
|
jobData.setStatus(EventPublisherJob.Status.FAILED);
|
||||||
|
jobData.setFailure(
|
||||||
|
new IndexingError()
|
||||||
|
.withErrorSource(IndexingError.ErrorSource.JOB)
|
||||||
|
.withMessage(e.getMessage()));
|
||||||
|
}
|
||||||
|
sendUpdates(jobExecutionContext);
|
||||||
|
LOG.error("Unexpected error during reindexing for entity {}", entityType, e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
futures.add(future);
|
||||||
|
}
|
||||||
|
|
||||||
|
executorService.shutdown();
|
||||||
|
|
||||||
|
try {
|
||||||
|
boolean allTasksCompleted = executorService.awaitTermination(1, TimeUnit.HOURS);
|
||||||
|
if (!allTasksCompleted) {
|
||||||
|
LOG.warn("Reindexing tasks did not complete within the expected time.");
|
||||||
|
executorService.shutdownNow();
|
||||||
|
}
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
LOG.error("Reindexing was interrupted", e);
|
||||||
|
executorService.shutdownNow();
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
}
|
||||||
|
|
||||||
|
for (Future<?> future : futures) {
|
||||||
|
try {
|
||||||
|
future.get();
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
LOG.error("Task was interrupted", e);
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
LOG.error("Exception in reindexing task", e.getCause());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -347,9 +405,7 @@ public class SearchIndexApp extends AbstractNativeApplication {
|
|||||||
|
|
||||||
private void sendUpdates(JobExecutionContext jobExecutionContext) {
|
private void sendUpdates(JobExecutionContext jobExecutionContext) {
|
||||||
try {
|
try {
|
||||||
// store job details in Database
|
|
||||||
jobExecutionContext.getJobDetail().getJobDataMap().put(APP_RUN_STATS, jobData.getStats());
|
jobExecutionContext.getJobDetail().getJobDataMap().put(APP_RUN_STATS, jobData.getStats());
|
||||||
// Update Record to db
|
|
||||||
updateRecordToDb(jobExecutionContext);
|
updateRecordToDb(jobExecutionContext);
|
||||||
if (WebSocketManager.getInstance() != null) {
|
if (WebSocketManager.getInstance() != null) {
|
||||||
WebSocketManager.getInstance()
|
WebSocketManager.getInstance()
|
||||||
@ -362,10 +418,8 @@ public class SearchIndexApp extends AbstractNativeApplication {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public void updateStats(String entityType, StepStats currentEntityStats) {
|
public void updateStats(String entityType, StepStats currentEntityStats) {
|
||||||
// Job Level Stats
|
|
||||||
Stats jobDataStats = jobData.getStats();
|
Stats jobDataStats = jobData.getStats();
|
||||||
|
|
||||||
// Update Entity Level Stats
|
|
||||||
StepStats entityLevelStats = jobDataStats.getEntityStats();
|
StepStats entityLevelStats = jobDataStats.getEntityStats();
|
||||||
if (entityLevelStats == null) {
|
if (entityLevelStats == null) {
|
||||||
entityLevelStats =
|
entityLevelStats =
|
||||||
@ -373,7 +427,6 @@ public class SearchIndexApp extends AbstractNativeApplication {
|
|||||||
}
|
}
|
||||||
entityLevelStats.withAdditionalProperty(entityType, currentEntityStats);
|
entityLevelStats.withAdditionalProperty(entityType, currentEntityStats);
|
||||||
|
|
||||||
// Total Stats
|
|
||||||
StepStats stats = jobData.getStats().getJobStats();
|
StepStats stats = jobData.getStats().getJobStats();
|
||||||
if (stats == null) {
|
if (stats == null) {
|
||||||
stats =
|
stats =
|
||||||
@ -392,7 +445,6 @@ public class SearchIndexApp extends AbstractNativeApplication {
|
|||||||
.mapToInt(StepStats::getFailedRecords)
|
.mapToInt(StepStats::getFailedRecords)
|
||||||
.sum());
|
.sum());
|
||||||
|
|
||||||
// Update for the Job
|
|
||||||
jobDataStats.setJobStats(stats);
|
jobDataStats.setJobStats(stats);
|
||||||
jobDataStats.setEntityStats(entityLevelStats);
|
jobDataStats.setEntityStats(entityLevelStats);
|
||||||
|
|
||||||
@ -405,13 +457,31 @@ public class SearchIndexApp extends AbstractNativeApplication {
|
|||||||
}
|
}
|
||||||
|
|
||||||
IndexMapping indexType = searchRepository.getIndexMapping(entityType);
|
IndexMapping indexType = searchRepository.getIndexMapping(entityType);
|
||||||
// Delete index
|
|
||||||
searchRepository.deleteIndex(indexType);
|
searchRepository.deleteIndex(indexType);
|
||||||
// Create index
|
|
||||||
searchRepository.createIndex(indexType);
|
searchRepository.createIndex(indexType);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void stopJob() {
|
public void stopJob() {
|
||||||
|
LOG.info("Stopping reindexing job.");
|
||||||
stopped = true;
|
stopped = true;
|
||||||
|
if (executorService != null && !executorService.isShutdown()) {
|
||||||
|
List<Runnable> awaitingTasks = executorService.shutdownNow();
|
||||||
|
LOG.info(
|
||||||
|
"ExecutorService has been shutdown. Awaiting termination. {} tasks were awaiting execution.",
|
||||||
|
awaitingTasks.size());
|
||||||
|
|
||||||
|
try {
|
||||||
|
if (!executorService.awaitTermination(60, TimeUnit.SECONDS)) {
|
||||||
|
LOG.warn("ExecutorService did not terminate within the specified timeout.");
|
||||||
|
}
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
LOG.error("Interrupted while waiting for ExecutorService to terminate.", e);
|
||||||
|
List<Runnable> stillAwaitingTasks = executorService.shutdownNow(); // Force shutdown
|
||||||
|
LOG.info(
|
||||||
|
"Forced shutdown initiated due to interruption. {} tasks were awaiting execution.",
|
||||||
|
stillAwaitingTasks.size());
|
||||||
|
Thread.currentThread().interrupt(); // Restore interrupt status
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user