From f157a29fef977fd5a47b9ae7f9ca46af53dbd4bd Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Mon, 18 Dec 2023 14:29:58 +0530 Subject: [PATCH 01/28] Remove LMAX Disruptor , use queue and Quartz --- .../native/1.3.0/mysql/schemaChanges.sql | 18 ++ .../native/1.3.0/postgres/schemaChanges.sql | 17 ++ .../service/OpenMetadataApplication.java | 4 +- .../changeEvent/AbstractEventConsumer.java | 234 ++++++++++++++++++ .../apps/bundles/changeEvent/Consumer.java | 30 +++ .../changeEvent/EventAlertProducer.java | 3 + .../changeEvent}/email/EmailMessage.java | 2 +- .../changeEvent}/email/EmailPublisher.java | 44 ++-- .../changeEvent}/gchat/GChatMessage.java | 2 +- .../changeEvent}/gchat/GChatPublisher.java | 51 ++-- .../generic/GenericPublisher.java | 63 ++--- .../msteams/MSTeamsPublisher.java | 52 ++-- .../changeEvent}/msteams/TeamsMessage.java | 2 +- .../bundles/changeEvent}/slack/Field.java | 2 +- .../changeEvent}/slack/SlackAttachment.java | 2 +- .../slack/SlackEventPublisher.java | 51 ++-- .../changeEvent}/slack/SlackMessage.java | 2 +- .../insights/DataInsightsReportApp.java | 4 +- .../service/apps/scheduler/AppScheduler.java | 2 + .../service/events/ChangeEventHandler.java | 1 - .../scheduled/EventSubscriptionScheduler.java | 190 ++++++++++++++ .../events/scheduled/ReportsHandler.java | 154 ------------ .../events/subscription/AlertUtil.java | 27 +- .../exception/DataInsightJobException.java | 11 - .../EventSubscriptionJobException.java | 11 + .../decorators/EmailMessageDecorator.java | 8 +- .../decorators/GChatMessageDecorator.java | 10 +- .../decorators/MSTeamsMessageDecorator.java | 11 +- .../decorators/MessageDecorator.java | 13 + .../decorators/SlackMessageDecorator.java | 13 +- .../formatter/entity/TestCaseFormatter.java | 3 +- .../service/jdbi3/CollectionDAO.java | 27 ++ .../jdbi3/EventSubscriptionRepository.java | 133 +--------- .../EventSubscriptionResource.java | 81 ++---- .../openmetadata/service/util/EmailUtil.java | 2 +- .../service/util/SubscriptionUtil.java | 15 +- .../DataInsightReportAlert.json | 17 -- .../json/schema/events/eventSubscription.json | 7 +- .../events/eventSubscriptionOffset.json | 20 ++ 39 files changed, 759 insertions(+), 580 deletions(-) create mode 100644 openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java create mode 100644 openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Consumer.java create mode 100644 openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/EventAlertProducer.java rename openmetadata-service/src/main/java/org/openmetadata/service/{events/subscription => apps/bundles/changeEvent}/email/EmailMessage.java (87%) rename openmetadata-service/src/main/java/org/openmetadata/service/{events/subscription => apps/bundles/changeEvent}/email/EmailPublisher.java (76%) rename openmetadata-service/src/main/java/org/openmetadata/service/{events/subscription => apps/bundles/changeEvent}/gchat/GChatMessage.java (93%) rename openmetadata-service/src/main/java/org/openmetadata/service/{events/subscription => apps/bundles/changeEvent}/gchat/GChatPublisher.java (71%) rename openmetadata-service/src/main/java/org/openmetadata/service/{events/subscription => apps/bundles/changeEvent}/generic/GenericPublisher.java (71%) rename openmetadata-service/src/main/java/org/openmetadata/service/{events/subscription => apps/bundles/changeEvent}/msteams/MSTeamsPublisher.java (68%) rename openmetadata-service/src/main/java/org/openmetadata/service/{events/subscription => apps/bundles/changeEvent}/msteams/TeamsMessage.java (91%) rename openmetadata-service/src/main/java/org/openmetadata/service/{events/subscription => apps/bundles/changeEvent}/slack/Field.java (87%) rename openmetadata-service/src/main/java/org/openmetadata/service/{events/subscription => apps/bundles/changeEvent}/slack/SlackAttachment.java (94%) rename openmetadata-service/src/main/java/org/openmetadata/service/{events/subscription => apps/bundles/changeEvent}/slack/SlackEventPublisher.java (70%) rename openmetadata-service/src/main/java/org/openmetadata/service/{events/subscription => apps/bundles/changeEvent}/slack/SlackMessage.java (91%) create mode 100644 openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java delete mode 100644 openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/ReportsHandler.java delete mode 100644 openmetadata-service/src/main/java/org/openmetadata/service/exception/DataInsightJobException.java create mode 100644 openmetadata-service/src/main/java/org/openmetadata/service/exception/EventSubscriptionJobException.java delete mode 100644 openmetadata-service/src/main/resources/json/data/eventsubscription/DataInsightReportAlert.json create mode 100644 openmetadata-spec/src/main/resources/json/schema/events/eventSubscriptionOffset.json diff --git a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql index 0b6e2c6dea76..da5512e823ec 100644 --- a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql +++ b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql @@ -32,6 +32,24 @@ set json = JSON_INSERT( ) where name = 'DataInsightsApplication'; +-- Update Change Event Table +ALTER TABLE change_event ADD COLUMN offset INT AUTO_INCREMENT PRIMARY KEY; + +-- Add new table for event subscription extensions +CREATE TABLE IF NOT EXISTS event_subscription_extension ( + id VARCHAR(36) NOT NULL, + extension VARCHAR(256) NOT NULL, + jsonSchema VARCHAR(256) NOT NULL, + json JSON NOT NULL, + timestamp BIGINT UNSIGNED GENERATED ALWAYS AS (json ->> '$.timestamp') NOT NULL, + UNIQUE(id, extension) +); + +DELETE FROM event_subscription_entity ese where name = 'DataInsightReport'; + +ALTER TABLE event_subscription_extension ADD COLUMN offset INT AUTO_INCREMENT; + + -- Rename NOOP Secret Manager to DB update metadata_service_entity set json = JSON_REPLACE(json, '$.connection.config.secretsManagerProvider', 'db') diff --git a/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql b/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql index f8b25af25d42..60ea7182ff5b 100644 --- a/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql +++ b/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql @@ -39,6 +39,23 @@ SET json = jsonb_set( ) where name = 'DataInsightsApplication'; +-- Update Change Event Table +ALTER TABLE change_event ADD COLUMN offset SERIAL PRIMARY KEY; + +-- Add new table for event subscription extensions +CREATE TABLE IF NOT EXISTS event_subscription_extension ( + id VARCHAR(36) NOT NULL, + extension VARCHAR(256) NOT NULL, + jsonSchema VARCHAR(256) NOT NULL, + json jsonb NOT NULL, + timestamp BIGINT GENERATED ALWAYS AS ((json ->> 'timestamp')::bigint) STORED NOT NULL, + UNIQUE(id, extension) +); + +DELETE FROM event_subscription_entity ese where name = 'DataInsightReport'; + +ALTER TABLE event_subscription_extension ADD COLUMN offset SERIAL; + -- Rename NOOP Secret Manager to DB update metadata_service_entity set json = jsonb_set( diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/OpenMetadataApplication.java b/openmetadata-service/src/main/java/org/openmetadata/service/OpenMetadataApplication.java index 02dc3b6c130e..b5103eab8dab 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/OpenMetadataApplication.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/OpenMetadataApplication.java @@ -71,8 +71,8 @@ import org.openmetadata.service.config.OMWebConfiguration; import org.openmetadata.service.events.EventFilter; import org.openmetadata.service.events.EventPubSub; +import org.openmetadata.service.events.scheduled.EventSubscriptionScheduler; import org.openmetadata.service.events.scheduled.PipelineServiceStatusJobHandler; -import org.openmetadata.service.events.scheduled.ReportsHandler; import org.openmetadata.service.exception.CatalogGenericExceptionMapper; import org.openmetadata.service.exception.ConstraintViolationExceptionMapper; import org.openmetadata.service.exception.JsonMappingExceptionMapper; @@ -480,8 +480,8 @@ public void stop() throws InterruptedException, SchedulerException { LOG.info("Cache with Id Stats {}", EntityRepository.CACHE_WITH_ID.stats()); LOG.info("Cache with name Stats {}", EntityRepository.CACHE_WITH_NAME.stats()); EventPubSub.shutdown(); - ReportsHandler.shutDown(); AppScheduler.shutDown(); + EventSubscriptionScheduler.shutDown(); LOG.info("Stopping the application"); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java new file mode 100644 index 000000000000..3b69a30caa78 --- /dev/null +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -0,0 +1,234 @@ +/* + * Copyright 2021 Collate + * Licensed 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.openmetadata.service.apps.bundles.changeEvent; + +import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.ACTIVE; +import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.AWAITING_RETRY; +import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.FAILED; + +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; +import lombok.Setter; +import lombok.extern.slf4j.Slf4j; +import org.openmetadata.schema.entity.events.EventSubscription; +import org.openmetadata.schema.entity.events.EventSubscriptionOffset; +import org.openmetadata.schema.entity.events.SubscriptionStatus; +import org.openmetadata.schema.type.ChangeEvent; +import org.openmetadata.service.Entity; +import org.openmetadata.service.events.errors.RetriableException; +import org.openmetadata.service.events.subscription.AlertUtil; +import org.openmetadata.service.util.JsonUtils; +import org.quartz.DisallowConcurrentExecution; +import org.quartz.Job; +import org.quartz.JobDetail; +import org.quartz.JobExecutionContext; +import org.quartz.JobExecutionException; + +@Slf4j +@DisallowConcurrentExecution +public abstract class AbstractEventConsumer implements Consumer, Job { + public static final String ALERT_OFFSET_KEY = "alertOffsetKey"; + public static final String ALERT_INFO_KEY = "alertInfoKey"; + private static final String OFFSET_EXTENSION = "eventSubscription.Offset"; + protected static final int BACKOFF_NORMAL = 0; + protected static final int BACKOFF_3_SECONDS = 3 * 1000; + protected static final int BACKOFF_30_SECONDS = 30 * 1000; + protected static final int BACKOFF_5_MINUTES = 5 * 60 * 1000; + protected static final int BACKOFF_1_HOUR = 60 * 60 * 1000; + protected static final int BACKOFF_24_HOUR = 24 * 60 * 60 * 1000; + + @Getter protected int currentBackoffTime = BACKOFF_NORMAL; + private int offset = -1; + + @Getter @Setter private JobDetail jobDetail; + protected EventSubscription eventSubscription; + + protected AbstractEventConsumer() {} + + private void init(JobExecutionContext context) { + EventSubscription sub = (EventSubscription) context.getJobDetail().getJobDataMap().get(ALERT_INFO_KEY); + this.jobDetail = context.getJobDetail(); + this.eventSubscription = sub; + this.offset = loadInitialOffset(); + this.doInit(context); + } + + protected abstract void doInit(JobExecutionContext context); + + protected void sendAlert(List list) { + /* This method needs to be over-ridden by specific Publisher for sending Alert */ + + } + + @Override + public void handleFailedEvents(List failedEvents) {} + + @Override + public void handleException(Exception e) {} + + private int loadInitialOffset() { + int eventSubscriptionOffset; + String json = + Entity.getCollectionDAO() + .eventSubscriptionDAO() + .getSubscriberOffset(eventSubscription.getId().toString(), OFFSET_EXTENSION); + if (json != null) { + EventSubscriptionOffset offsetFromDb = JsonUtils.readValue(json, EventSubscriptionOffset.class); + eventSubscriptionOffset = offsetFromDb.getOffset(); + } else { + eventSubscriptionOffset = Entity.getCollectionDAO().changeEventDAO().listCount(); + } + // Update the Job Data Map with the latest offset + return eventSubscriptionOffset; + } + + @Override + public boolean publishEvents(List events) throws InterruptedException { + // Publish to the given Alert Actions + // Evaluate Alert Trigger Config + + // Filter the Change Events based on Alert Trigger Config + List filteredEvents = new ArrayList<>(); + for (ChangeEvent event : events) { + boolean triggerChangeEvent = + AlertUtil.shouldTriggerAlert(event.getEntityType(), eventSubscription.getFilteringRules()); + + // Evaluate ChangeEvent Alert Filtering + if (eventSubscription.getFilteringRules() != null + && !AlertUtil.evaluateAlertConditions(event, eventSubscription.getFilteringRules().getRules())) { + triggerChangeEvent = false; + } + + if (triggerChangeEvent) { + // Ignore the event since change description is null + if (event.getChangeDescription() != null) { + filteredEvents.add(event); + } else { + LOG.info("Email Publisher Event Will be Ignored Since Change Description is null. Received Event: {}", event); + } + } + } + + try { + sendAlert(filteredEvents); + return true; + } catch (RetriableException ex) { + setNextBackOff(); + LOG.error( + "Failed to publish event in batch {} due to {}, will try again in {} ms", + filteredEvents, + ex, + currentBackoffTime); + Thread.sleep(currentBackoffTime); + } catch (Exception e) { + LOG.error("[AbstractAlertPublisher] error {}", e.getMessage(), e); + } + return false; + } + + @Override + public void commitOffset(JobExecutionContext jobExecutionContext, int offset) { + EventSubscriptionOffset eventSubscriptionOffset = + new EventSubscriptionOffset().withOffset(offset).withTimestamp(System.currentTimeMillis()); + Entity.getCollectionDAO() + .eventSubscriptionDAO() + .upsertSubscriberOffset( + eventSubscription.getId().toString(), + OFFSET_EXTENSION, + "eventSubscriptionOffset", + JsonUtils.pojoToJson(eventSubscriptionOffset)); + + // Update the Job Data Map with the latest offset + jobExecutionContext.getJobDetail().getJobDataMap().put(ALERT_OFFSET_KEY, offset); + } + + public synchronized void setErrorStatus(Long attemptTime, Integer statusCode, String reason) { + setStatus(FAILED, attemptTime, statusCode, reason, null); + } + + public synchronized void setAwaitingRetry(Long attemptTime, int statusCode, String reason) { + setStatus(AWAITING_RETRY, attemptTime, statusCode, reason, attemptTime + currentBackoffTime); + } + + public synchronized void setSuccessStatus(Long updateTime) { + SubscriptionStatus subStatus = + AlertUtil.buildSubscriptionStatus(ACTIVE, updateTime, null, null, null, updateTime, updateTime); + eventSubscription.setStatusDetails(subStatus); + } + + protected synchronized void setStatus( + SubscriptionStatus.Status status, Long attemptTime, Integer statusCode, String reason, Long timestamp) { + SubscriptionStatus subStatus = + AlertUtil.buildSubscriptionStatus(status, null, attemptTime, statusCode, reason, timestamp, attemptTime); + eventSubscription.setStatusDetails(subStatus); + } + + @Override + public List pollEvents(long offset, long batchSize) { + // Read from Change Event Table + List eventJson = Entity.getCollectionDAO().changeEventDAO().list(batchSize, offset); + + List changeEvents = new ArrayList<>(); + for (String json : eventJson) { + ChangeEvent event = JsonUtils.readValue(json, ChangeEvent.class); + changeEvents.add(event); + } + return changeEvents; + } + + @Override + public void execute(JobExecutionContext jobExecutionContext) throws JobExecutionException { + // Must Have , Before Execute the Init, Quartz Requires a Non-Arg Constructor + this.init(jobExecutionContext); + + try { + List batch = pollEvents(offset, 100); + if (!batch.isEmpty()) { + boolean success = publishEvents(batch); + if (success) { + offset += batch.size(); + } else { + handleFailedEvents(batch); + } + } + } catch (InterruptedException e) { + LOG.error("Interrupted while polling events", e); + Thread.currentThread().interrupt(); + } catch (Exception e) { + handleException(e); + } finally { + LOG.debug("Committing offset for eventSubscription {} {}", eventSubscription.getName(), offset); + commitOffset(jobExecutionContext, offset); + } + } + + public void setNextBackOff() { + if (currentBackoffTime == BACKOFF_NORMAL) { + currentBackoffTime = BACKOFF_3_SECONDS; + } else if (currentBackoffTime == BACKOFF_3_SECONDS) { + currentBackoffTime = BACKOFF_30_SECONDS; + } else if (currentBackoffTime == BACKOFF_30_SECONDS) { + currentBackoffTime = BACKOFF_5_MINUTES; + } else if (currentBackoffTime == BACKOFF_5_MINUTES) { + currentBackoffTime = BACKOFF_1_HOUR; + } else if (currentBackoffTime == BACKOFF_1_HOUR) { + currentBackoffTime = BACKOFF_24_HOUR; + } + } + + public EventSubscription getEventSubscription() { + return (EventSubscription) jobDetail.getJobDataMap().get(ALERT_INFO_KEY); + } +} diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Consumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Consumer.java new file mode 100644 index 000000000000..4173c961c04e --- /dev/null +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Consumer.java @@ -0,0 +1,30 @@ +/* + * Copyright 2021 Collate + * Licensed 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.openmetadata.service.apps.bundles.changeEvent; + +import io.dropwizard.lifecycle.Managed; +import java.util.List; +import org.quartz.JobExecutionContext; + +public interface Consumer extends Managed { + List pollEvents(long offset, long batchSize); + + boolean publishEvents(List events) throws InterruptedException; + + void handleFailedEvents(List failedEvents); + + void handleException(Exception e); + + void commitOffset(JobExecutionContext jobExecutionContext, int offset); +} diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/EventAlertProducer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/EventAlertProducer.java new file mode 100644 index 000000000000..874473064215 --- /dev/null +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/EventAlertProducer.java @@ -0,0 +1,3 @@ +package org.openmetadata.service.apps.bundles.changeEvent; + +public class EventAlertProducer {} diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/email/EmailMessage.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailMessage.java similarity index 87% rename from openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/email/EmailMessage.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailMessage.java index 3d20be5e490d..73f0782ff8ab 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/email/EmailMessage.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailMessage.java @@ -1,4 +1,4 @@ -package org.openmetadata.service.events.subscription.email; +package org.openmetadata.service.apps.bundles.changeEvent.email; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/email/EmailPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailPublisher.java similarity index 76% rename from openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/email/EmailPublisher.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailPublisher.java index 2cb87727b93f..e86fb4a42a16 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/email/EmailPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailPublisher.java @@ -11,58 +11,49 @@ * limitations under the License. */ -package org.openmetadata.service.events.subscription.email; +package org.openmetadata.service.apps.bundles.changeEvent.email; import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.EMAIL; import static org.openmetadata.service.events.subscription.AlertsRuleEvaluator.getEntity; import static org.openmetadata.service.util.SubscriptionUtil.buildReceiversListFromActions; import java.util.HashSet; +import java.util.List; import java.util.Set; import lombok.extern.slf4j.Slf4j; import org.openmetadata.schema.EntityInterface; import org.openmetadata.schema.alert.type.EmailAlertConfig; -import org.openmetadata.schema.entity.events.EventSubscription; import org.openmetadata.schema.type.ChangeEvent; +import org.openmetadata.service.Entity; +import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; import org.openmetadata.service.events.errors.EventPublisherException; -import org.openmetadata.service.events.subscription.SubscriptionPublisher; import org.openmetadata.service.exception.CatalogExceptionMessage; import org.openmetadata.service.formatter.decorators.EmailMessageDecorator; import org.openmetadata.service.formatter.decorators.MessageDecorator; import org.openmetadata.service.jdbi3.CollectionDAO; -import org.openmetadata.service.resources.events.EventResource; import org.openmetadata.service.util.EmailUtil; import org.openmetadata.service.util.JsonUtils; +import org.quartz.JobExecutionContext; @Slf4j -public class EmailPublisher extends SubscriptionPublisher { +public class EmailPublisher extends AbstractEventConsumer { private final MessageDecorator emailDecorator = new EmailMessageDecorator(); - private final EmailAlertConfig emailAlertConfig; - private final CollectionDAO daoCollection; + private EmailAlertConfig emailAlertConfig; + private CollectionDAO daoCollection; - public EmailPublisher(EventSubscription eventSub, CollectionDAO dao) { - super(eventSub); - if (eventSub.getSubscriptionType() == EMAIL) { - this.emailAlertConfig = JsonUtils.convertValue(eventSub.getSubscriptionConfig(), EmailAlertConfig.class); - this.daoCollection = dao; + @Override + protected void doInit(JobExecutionContext context) { + if (eventSubscription.getSubscriptionType() == EMAIL) { + this.emailAlertConfig = JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), EmailAlertConfig.class); + this.daoCollection = Entity.getCollectionDAO(); } else { throw new IllegalArgumentException("Email Alert Invoked with Illegal Type and Settings."); } } @Override - public void onStartDelegate() { - LOG.info("Email Publisher Started"); - } - - @Override - public void onShutdownDelegate() { - LOG.info("Email Publisher Stopped"); - } - - @Override - public void sendAlert(EventResource.EventList list) { - for (ChangeEvent event : list.getData()) { + public void sendAlert(List changeEvents) { + for (ChangeEvent event : changeEvents) { try { Set receivers = buildReceiversList(event); EmailMessage emailMessage = emailDecorator.buildMessage(event); @@ -88,4 +79,9 @@ private Set buildReceiversList(ChangeEvent changeEvent) { emailAlertConfig, EMAIL, daoCollection, entityInterface.getId(), changeEvent.getEntityType())); return receiverList; } + + @Override + public void stop() { + LOG.info("Email Publisher Stopped"); + } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/gchat/GChatMessage.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatMessage.java similarity index 93% rename from openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/gchat/GChatMessage.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatMessage.java index 77d9d7a17197..eac73fe20252 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/gchat/GChatMessage.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatMessage.java @@ -1,4 +1,4 @@ -package org.openmetadata.service.events.subscription.gchat; +package org.openmetadata.service.apps.bundles.changeEvent.gchat; import java.util.List; import lombok.AllArgsConstructor; diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/gchat/GChatPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java similarity index 71% rename from openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/gchat/GChatPublisher.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java index 13840413154e..8470004843ed 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/gchat/GChatPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java @@ -11,7 +11,7 @@ * limitations under the License. */ -package org.openmetadata.service.events.subscription.gchat; +package org.openmetadata.service.apps.bundles.changeEvent.gchat; import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.G_CHAT_WEBHOOK; import static org.openmetadata.service.util.SubscriptionUtil.getClient; @@ -23,34 +23,30 @@ import javax.ws.rs.client.Invocation; import lombok.extern.slf4j.Slf4j; import org.openmetadata.common.utils.CommonUtil; -import org.openmetadata.schema.entity.events.EventSubscription; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.schema.type.Webhook; +import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; import org.openmetadata.service.events.errors.EventPublisherException; -import org.openmetadata.service.events.subscription.SubscriptionPublisher; import org.openmetadata.service.exception.CatalogExceptionMessage; import org.openmetadata.service.formatter.decorators.GChatMessageDecorator; import org.openmetadata.service.formatter.decorators.MessageDecorator; -import org.openmetadata.service.jdbi3.CollectionDAO; -import org.openmetadata.service.resources.events.EventResource; import org.openmetadata.service.util.JsonUtils; +import org.quartz.JobExecutionContext; @Slf4j -public class GChatPublisher extends SubscriptionPublisher { +public class GChatPublisher extends AbstractEventConsumer { private final MessageDecorator gChatMessageMessageDecorator = new GChatMessageDecorator(); - private final Webhook webhook; + private Webhook webhook; private Invocation.Builder target; - private final Client client; - private final CollectionDAO daoCollection; + private Client client; - public GChatPublisher(EventSubscription eventSub, CollectionDAO dao) { - super(eventSub); - if (eventSub.getSubscriptionType() == G_CHAT_WEBHOOK) { - this.daoCollection = dao; - this.webhook = JsonUtils.convertValue(eventSub.getSubscriptionConfig(), Webhook.class); + @Override + protected void doInit(JobExecutionContext context) { + if (eventSubscription.getSubscriptionType() == G_CHAT_WEBHOOK) { + this.webhook = JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); // Build Client - client = getClient(eventSub.getTimeout(), eventSub.getReadTimeout()); + client = getClient(eventSubscription.getTimeout(), eventSubscription.getReadTimeout()); // Build Target if (webhook.getEndpoint() != null) { @@ -65,23 +61,11 @@ public GChatPublisher(EventSubscription eventSub, CollectionDAO dao) { } @Override - protected void onStartDelegate() { - LOG.info("GChat Webhook publisher started"); - } - - @Override - protected void onShutdownDelegate() { - if (null != client) { - client.close(); - } - } - - @Override - protected void sendAlert(EventResource.EventList list) { - for (ChangeEvent event : list.getData()) { + public void sendAlert(List changeEvents) { + for (ChangeEvent event : changeEvents) { try { GChatMessage gchatMessage = gChatMessageMessageDecorator.buildMessage(event); - List targets = getTargetsForWebhook(webhook, G_CHAT_WEBHOOK, client, daoCollection, event); + List targets = getTargetsForWebhook(webhook, G_CHAT_WEBHOOK, client, event); if (target != null) { targets.add(target); } @@ -95,4 +79,11 @@ protected void sendAlert(EventResource.EventList list) { } } } + + @Override + public void stop() { + if (null != client) { + client.close(); + } + } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/generic/GenericPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java similarity index 71% rename from openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/generic/GenericPublisher.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java index 216aeecbda2c..aec952450fc3 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/generic/GenericPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java @@ -11,7 +11,7 @@ * limitations under the License. */ -package org.openmetadata.service.events.subscription.generic; +package org.openmetadata.service.apps.bundles.changeEvent.generic; import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.GENERIC_WEBHOOK; import static org.openmetadata.service.util.SubscriptionUtil.getClient; @@ -25,59 +25,38 @@ import javax.ws.rs.client.Invocation; import lombok.extern.slf4j.Slf4j; import org.openmetadata.common.utils.CommonUtil; -import org.openmetadata.schema.entity.events.EventSubscription; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.schema.type.Webhook; +import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; import org.openmetadata.service.events.errors.EventPublisherException; -import org.openmetadata.service.events.subscription.SubscriptionPublisher; -import org.openmetadata.service.jdbi3.CollectionDAO; -import org.openmetadata.service.resources.events.EventResource; import org.openmetadata.service.security.SecurityUtil; import org.openmetadata.service.util.JsonUtils; import org.openmetadata.service.util.RestUtil; +import org.quartz.JobExecutionContext; @Slf4j -public class GenericPublisher extends SubscriptionPublisher { - private final Client client; - private final Webhook webhook; - private final CollectionDAO daoCollection; +public class GenericPublisher extends AbstractEventConsumer { + private Client client; + private Webhook webhook; - public GenericPublisher(EventSubscription eventSub, CollectionDAO dao) { - super(eventSub); - if (eventSub.getSubscriptionType() == GENERIC_WEBHOOK) { - this.daoCollection = dao; - this.webhook = JsonUtils.convertValue(eventSub.getSubscriptionConfig(), Webhook.class); + @Override + protected void doInit(JobExecutionContext context) { + if (eventSubscription.getSubscriptionType() == GENERIC_WEBHOOK) { + this.webhook = JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); // Build Client - client = getClient(eventSub.getTimeout(), eventSub.getReadTimeout()); + this.client = getClient(eventSubscription.getTimeout(), eventSubscription.getReadTimeout()); } else { throw new IllegalArgumentException("GenericWebhook Alert Invoked with Illegal Type and Settings."); } } @Override - public void onStartDelegate() { - LOG.info("Generic Webhook Publisher Started"); - } - - @Override - public void onShutdownDelegate() { - if (client != null) { - client.close(); - } - } - - private Invocation.Builder getTarget() { - Map authHeaders = SecurityUtil.authHeaders("admin@open-metadata.org"); - return SecurityUtil.addHeaders(client.target(webhook.getEndpoint()), authHeaders); - } - - @Override - public void sendAlert(EventResource.EventList list) throws EventPublisherException { + public void sendAlert(List changeEvents) throws EventPublisherException { long attemptTime = System.currentTimeMillis(); try { // Post Message to default - String json = JsonUtils.pojoToJson(list); + String json = JsonUtils.pojoToJson(changeEvents); if (webhook.getEndpoint() != null) { if (webhook.getSecretKey() != null && !webhook.getSecretKey().isEmpty()) { String hmac = "sha256=" + CommonUtil.calculateHMAC(webhook.getSecretKey(), json); @@ -88,9 +67,9 @@ public void sendAlert(EventResource.EventList list) throws EventPublisherExcepti } // Post to Generic Webhook with Actions - for (ChangeEvent event : list.getData()) { + for (ChangeEvent event : changeEvents) { String eventJson = JsonUtils.pojoToJson(event); - List targets = getTargetsForWebhook(webhook, GENERIC_WEBHOOK, client, daoCollection, event); + List targets = getTargetsForWebhook(webhook, GENERIC_WEBHOOK, client, event); for (Invocation.Builder actionTarget : targets) { postWebhookMessage(this, actionTarget, eventJson); } @@ -105,4 +84,16 @@ public void sendAlert(EventResource.EventList list) throws EventPublisherExcepti } } } + + private Invocation.Builder getTarget() { + Map authHeaders = SecurityUtil.authHeaders("admin@open-metadata.org"); + return SecurityUtil.addHeaders(client.target(webhook.getEndpoint()), authHeaders); + } + + @Override + public void stop() { + if (null != client) { + client.close(); + } + } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/msteams/MSTeamsPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java similarity index 68% rename from openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/msteams/MSTeamsPublisher.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java index db69cc9e892c..86f4d1387488 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/msteams/MSTeamsPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java @@ -11,7 +11,7 @@ * limitations under the License. */ -package org.openmetadata.service.events.subscription.msteams; +package org.openmetadata.service.apps.bundles.changeEvent.msteams; import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.MS_TEAMS_WEBHOOK; import static org.openmetadata.service.util.SubscriptionUtil.getClient; @@ -23,34 +23,30 @@ import javax.ws.rs.client.Invocation; import lombok.extern.slf4j.Slf4j; import org.openmetadata.common.utils.CommonUtil; -import org.openmetadata.schema.entity.events.EventSubscription; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.schema.type.Webhook; +import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; import org.openmetadata.service.events.errors.EventPublisherException; -import org.openmetadata.service.events.subscription.SubscriptionPublisher; import org.openmetadata.service.exception.CatalogExceptionMessage; import org.openmetadata.service.formatter.decorators.MSTeamsMessageDecorator; import org.openmetadata.service.formatter.decorators.MessageDecorator; -import org.openmetadata.service.jdbi3.CollectionDAO; -import org.openmetadata.service.resources.events.EventResource; import org.openmetadata.service.util.JsonUtils; +import org.quartz.JobExecutionContext; @Slf4j -public class MSTeamsPublisher extends SubscriptionPublisher { +public class MSTeamsPublisher extends AbstractEventConsumer { private final MessageDecorator teamsMessageFormatter = new MSTeamsMessageDecorator(); - private final Webhook webhook; + private Webhook webhook; private Invocation.Builder target; - private final Client client; - private final CollectionDAO daoCollection; + private Client client; - public MSTeamsPublisher(EventSubscription eventSub, CollectionDAO dao) { - super(eventSub); - if (eventSub.getSubscriptionType() == MS_TEAMS_WEBHOOK) { - this.daoCollection = dao; - this.webhook = JsonUtils.convertValue(eventSub.getSubscriptionConfig(), Webhook.class); + @Override + protected void doInit(JobExecutionContext context) { + if (eventSubscription.getSubscriptionType() == MS_TEAMS_WEBHOOK) { + this.webhook = JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); // Build Client - client = getClient(eventSub.getTimeout(), eventSub.getReadTimeout()); + client = getClient(eventSubscription.getTimeout(), eventSubscription.getReadTimeout()); // Build Target if (webhook.getEndpoint() != null) { @@ -65,24 +61,11 @@ public MSTeamsPublisher(EventSubscription eventSub, CollectionDAO dao) { } @Override - public void onStartDelegate() { - LOG.info("MsTeams Webhook Publisher Started"); - } - - @Override - public void onShutdownDelegate() { - if (client != null) { - client.close(); - } - } - - @Override - public void sendAlert(EventResource.EventList list) { - for (ChangeEvent event : list.getData()) { + public void sendAlert(List changeEvents) { + for (ChangeEvent event : changeEvents) { try { TeamsMessage teamsMessage = teamsMessageFormatter.buildMessage(event); - List targets = - getTargetsForWebhook(webhook, MS_TEAMS_WEBHOOK, client, daoCollection, event); + List targets = getTargetsForWebhook(webhook, MS_TEAMS_WEBHOOK, client, event); if (target != null) { targets.add(target); } @@ -96,4 +79,11 @@ public void sendAlert(EventResource.EventList list) { } } } + + @Override + public void stop() { + if (null != client) { + client.close(); + } + } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/msteams/TeamsMessage.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/TeamsMessage.java similarity index 91% rename from openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/msteams/TeamsMessage.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/TeamsMessage.java index 81c48790f5b8..8c0b019e5a2a 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/msteams/TeamsMessage.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/TeamsMessage.java @@ -1,4 +1,4 @@ -package org.openmetadata.service.events.subscription.msteams; +package org.openmetadata.service.apps.bundles.changeEvent.msteams; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/Field.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/Field.java similarity index 87% rename from openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/Field.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/Field.java index ff1afc5fd721..c3c566028a9d 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/Field.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/Field.java @@ -1,4 +1,4 @@ -package org.openmetadata.service.events.subscription.slack; +package org.openmetadata.service.apps.bundles.changeEvent.slack; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/SlackAttachment.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackAttachment.java similarity index 94% rename from openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/SlackAttachment.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackAttachment.java index 5d6404181eaf..7cffcb406498 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/SlackAttachment.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackAttachment.java @@ -1,4 +1,4 @@ -package org.openmetadata.service.events.subscription.slack; +package org.openmetadata.service.apps.bundles.changeEvent.slack; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/SlackEventPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java similarity index 70% rename from openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/SlackEventPublisher.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java index 4fd1105df795..9811c051e62a 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/SlackEventPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java @@ -11,7 +11,7 @@ * limitations under the License. */ -package org.openmetadata.service.events.subscription.slack; +package org.openmetadata.service.apps.bundles.changeEvent.slack; import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.SLACK_WEBHOOK; import static org.openmetadata.service.util.SubscriptionUtil.getClient; @@ -23,33 +23,29 @@ import javax.ws.rs.client.Invocation; import lombok.extern.slf4j.Slf4j; import org.openmetadata.common.utils.CommonUtil; -import org.openmetadata.schema.entity.events.EventSubscription; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.schema.type.Webhook; +import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; import org.openmetadata.service.events.errors.EventPublisherException; -import org.openmetadata.service.events.subscription.SubscriptionPublisher; import org.openmetadata.service.formatter.decorators.MessageDecorator; import org.openmetadata.service.formatter.decorators.SlackMessageDecorator; -import org.openmetadata.service.jdbi3.CollectionDAO; -import org.openmetadata.service.resources.events.EventResource; import org.openmetadata.service.util.JsonUtils; +import org.quartz.JobExecutionContext; @Slf4j -public class SlackEventPublisher extends SubscriptionPublisher { +public class SlackEventPublisher extends AbstractEventConsumer { private final MessageDecorator slackMessageFormatter = new SlackMessageDecorator(); - private final Webhook webhook; + private Webhook webhook; private Invocation.Builder target; - private final Client client; - private final CollectionDAO daoCollection; + private Client client; - public SlackEventPublisher(EventSubscription eventSub, CollectionDAO dao) { - super(eventSub); - if (eventSub.getSubscriptionType() == SLACK_WEBHOOK) { - this.daoCollection = dao; - this.webhook = JsonUtils.convertValue(eventSub.getSubscriptionConfig(), Webhook.class); + @Override + protected void doInit(JobExecutionContext context) { + if (eventSubscription.getSubscriptionType() == SLACK_WEBHOOK) { + this.webhook = JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); // Build Client - client = getClient(eventSub.getTimeout(), eventSub.getReadTimeout()); + client = getClient(eventSubscription.getTimeout(), eventSubscription.getReadTimeout()); // Build Target if (webhook.getEndpoint() != null) { @@ -64,23 +60,11 @@ public SlackEventPublisher(EventSubscription eventSub, CollectionDAO dao) { } @Override - public void onStartDelegate() { - LOG.info("Slack Webhook Publisher Started"); - } - - @Override - public void onShutdownDelegate() { - if (client != null) { - client.close(); - } - } - - @Override - public void sendAlert(EventResource.EventList list) { - for (ChangeEvent event : list.getData()) { + public void sendAlert(List changeEvents) { + for (ChangeEvent event : changeEvents) { try { SlackMessage slackMessage = slackMessageFormatter.buildMessage(event); - List targets = getTargetsForWebhook(webhook, SLACK_WEBHOOK, client, daoCollection, event); + List targets = getTargetsForWebhook(webhook, SLACK_WEBHOOK, client, event); if (target != null) { targets.add(target); } @@ -94,4 +78,11 @@ public void sendAlert(EventResource.EventList list) { } } } + + @Override + public void stop() { + if (null != client) { + client.close(); + } + } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/SlackMessage.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackMessage.java similarity index 91% rename from openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/SlackMessage.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackMessage.java index 1d8ba73c7f0a..92db6207202b 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/slack/SlackMessage.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackMessage.java @@ -1,4 +1,4 @@ -package org.openmetadata.service.events.subscription.slack; +package org.openmetadata.service.apps.bundles.changeEvent.slack; import com.fasterxml.jackson.annotation.JsonProperty; import lombok.Getter; diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/insights/DataInsightsReportApp.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/insights/DataInsightsReportApp.java index 8956a760efe7..d00a7df49760 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/insights/DataInsightsReportApp.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/insights/DataInsightsReportApp.java @@ -46,7 +46,7 @@ import org.openmetadata.service.apps.AbstractNativeApplication; import org.openmetadata.service.events.scheduled.template.DataInsightDescriptionAndOwnerTemplate; import org.openmetadata.service.events.scheduled.template.DataInsightTotalAssetTemplate; -import org.openmetadata.service.exception.DataInsightJobException; +import org.openmetadata.service.exception.EventSubscriptionJobException; import org.openmetadata.service.jdbi3.KpiRepository; import org.openmetadata.service.jdbi3.ListFilter; import org.openmetadata.service.search.SearchClient; @@ -89,7 +89,7 @@ public void execute(JobExecutionContext jobExecutionContext) { } } catch (Exception e) { LOG.error("[DIReport] Failed in sending report due to", e); - throw new DataInsightJobException(e); + throw new EventSubscriptionJobException(e); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/scheduler/AppScheduler.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/scheduler/AppScheduler.java index 4c4f103643a1..77cb96283e4f 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/scheduler/AppScheduler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/scheduler/AppScheduler.java @@ -23,6 +23,7 @@ import org.quartz.SchedulerException; import org.quartz.Trigger; import org.quartz.TriggerBuilder; +import org.quartz.TriggerKey; import org.quartz.impl.StdSchedulerFactory; @Slf4j @@ -88,6 +89,7 @@ public void deleteScheduledApplication(App app) throws SchedulerException { JobDetail jobDetail = getJobKey(app.getId()); if (jobDetail != null) { appScheduler.deleteJob(jobDetail.getKey()); + appScheduler.unscheduleJob(new TriggerKey(app.getId().toString(), APPS_TRIGGER_GROUP)); appJobsKeyMap.remove(app.getId()); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/ChangeEventHandler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/ChangeEventHandler.java index d7a33e5cebd1..14a6a274e8ba 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/ChangeEventHandler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/ChangeEventHandler.java @@ -72,7 +72,6 @@ public Void process(ContainerRequestContext requestContext, ContainerResponseCon changeEvent.getEntityId(), changeEvent.getEventType(), changeEvent.getEntityType()); - EventPubSub.publish(changeEvent); if (changeEvent.getEntity() != null) { Object entity = changeEvent.getEntity(); changeEvent = copyChangeEvent(changeEvent); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java new file mode 100644 index 000000000000..55ec758877d2 --- /dev/null +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java @@ -0,0 +1,190 @@ +/* + * Copyright 2021 Collate + * Licensed 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.openmetadata.service.events.scheduled; + +import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.ACTIVITY_FEED; +import static org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer.ALERT_INFO_KEY; + +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.jdbi.v3.sqlobject.transaction.Transaction; +import org.openmetadata.schema.api.events.CreateEventSubscription; +import org.openmetadata.schema.entity.events.EventSubscription; +import org.openmetadata.schema.entity.events.SubscriptionStatus; +import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; +import org.openmetadata.service.events.subscription.AlertUtil; +import org.quartz.JobBuilder; +import org.quartz.JobDataMap; +import org.quartz.JobDetail; +import org.quartz.Scheduler; +import org.quartz.SchedulerException; +import org.quartz.SimpleScheduleBuilder; +import org.quartz.Trigger; +import org.quartz.TriggerBuilder; +import org.quartz.TriggerKey; +import org.quartz.impl.StdSchedulerFactory; + +@Slf4j +public class EventSubscriptionScheduler { + public static final String ALERT_JOB_GROUP = "OMAlertJobGroup"; + public static final String ALERT_TRIGGER_GROUP = "OMAlertJobGroup"; + private static final String INVALID_ALERT = "Invalid Alert Type"; + private static EventSubscriptionScheduler instance; + private static volatile boolean initialized = false; + private final Scheduler alertsScheduler = new StdSchedulerFactory().getScheduler(); + private static final ConcurrentHashMap alertJobMap = new ConcurrentHashMap<>(); + + private EventSubscriptionScheduler() throws SchedulerException { + this.alertsScheduler.start(); + } + + @SneakyThrows + public static EventSubscriptionScheduler getInstance() { + if (!initialized) { + initialize(); + } + return instance; + } + + private static void initialize() throws SchedulerException { + if (!initialized) { + instance = new EventSubscriptionScheduler(); + initialized = true; + } else { + LOG.info("Event Subscription Scheduler is already initialized"); + } + } + + @Transaction + @SneakyThrows + public void addSubscriptionPublisher(EventSubscription eventSubscription) { + if (Objects.requireNonNull(eventSubscription.getAlertType()) == CreateEventSubscription.AlertType.CHANGE_EVENT) { + AbstractEventConsumer publisher = AlertUtil.getNotificationsPublisher(eventSubscription); + if (Boolean.FALSE.equals( + eventSubscription.getEnabled())) { // Only add webhook that is enabled for publishing events + eventSubscription.setStatusDetails(getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.DISABLED)); + } else { + eventSubscription.setStatusDetails(getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.ACTIVE)); + JobDetail jobDetail = + jobBuilder(publisher, eventSubscription, String.format("%s", eventSubscription.getId().toString())); + Trigger trigger = trigger(eventSubscription); + + // Schedule the Job + alertsScheduler.scheduleJob(jobDetail, trigger); + } + alertJobMap.put(eventSubscription.getId(), publisher); + LOG.info( + "Webhook publisher subscription started as {} : status {}", + eventSubscription.getName(), + eventSubscription.getStatusDetails().getStatus()); + } else { + throw new IllegalArgumentException(INVALID_ALERT); + } + } + + private JobDetail jobBuilder( + AbstractEventConsumer consumer, EventSubscription eventSubscription, String jobIdentity) { + JobDataMap dataMap = new JobDataMap(); + dataMap.put(ALERT_INFO_KEY, eventSubscription); + JobBuilder jobBuilder = + JobBuilder.newJob(consumer.getClass()).withIdentity(jobIdentity, ALERT_JOB_GROUP).usingJobData(dataMap); + return jobBuilder.build(); + } + + private Trigger trigger(EventSubscription eventSubscription) { + return TriggerBuilder.newTrigger() + .withIdentity(eventSubscription.getId().toString(), ALERT_TRIGGER_GROUP) + .withSchedule(SimpleScheduleBuilder.repeatSecondlyForever(10)) + .startNow() + .build(); + } + + private SubscriptionStatus getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status status) { + return new SubscriptionStatus().withStatus(status).withTimestamp(System.currentTimeMillis()); + } + + private AbstractEventConsumer getPublisher(UUID id) { + return alertJobMap.get(id); + } + + @Transaction + @SneakyThrows + public void updateEventSubscription(EventSubscription eventSubscription) { + if (Objects.requireNonNull(eventSubscription.getAlertType()) == CreateEventSubscription.AlertType.CHANGE_EVENT) { + if (Boolean.TRUE.equals(eventSubscription.getEnabled())) { // Only add webhook that is enabled for publishing + // If there was a previous webhook either in disabled state or stopped due + // to errors, update it and restart publishing + AbstractEventConsumer previousPublisher = getPublisher(eventSubscription.getId()); + if (previousPublisher == null) { + if (!ACTIVITY_FEED.equals(eventSubscription.getSubscriptionType())) { + addSubscriptionPublisher(eventSubscription); + } + return; + } + + // Update the existing publisher + deleteEventSubscriptionPublisher(eventSubscription); + addSubscriptionPublisher(eventSubscription); + } else { + // Remove the webhook publisher + removeProcessorForEventSubscription( + eventSubscription.getId(), getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.DISABLED)); + } + } else { + throw new IllegalArgumentException(INVALID_ALERT); + } + } + + @Transaction + @SneakyThrows + public void removeProcessorForEventSubscription(UUID id, SubscriptionStatus reasonForRemoval) { + AbstractEventConsumer publisher = alertJobMap.get(id); + if (publisher != null) { + alertsScheduler.deleteJob(publisher.getJobDetail().getKey()); + publisher.getEventSubscription().setStatusDetails(reasonForRemoval); + LOG.info("Alert publisher deleted for {}", publisher.getEventSubscription().getName()); + } + } + + @Transaction + public void deleteEventSubscriptionPublisher(EventSubscription deletedEntity) throws SchedulerException { + if (Objects.requireNonNull(deletedEntity.getAlertType()) == CreateEventSubscription.AlertType.CHANGE_EVENT) { + AbstractEventConsumer publisher = alertJobMap.remove(deletedEntity.getId()); + if (publisher != null) { + alertsScheduler.deleteJob(publisher.getJobDetail().getKey()); + alertsScheduler.unscheduleJob(new TriggerKey(deletedEntity.getId().toString(), ALERT_TRIGGER_GROUP)); + LOG.info("Alert publisher deleted for {}", publisher.getEventSubscription().getName()); + } + } else { + throw new IllegalArgumentException(INVALID_ALERT); + } + } + + public SubscriptionStatus getStatusForEventSubscription(UUID id) { + AbstractEventConsumer publisher = alertJobMap.get(id); + if (publisher != null) { + return publisher.getEventSubscription().getStatusDetails(); + } + return null; + } + + public static void shutDown() throws SchedulerException { + if (instance != null) { + instance.alertsScheduler.shutdown(); + } + } +} diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/ReportsHandler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/ReportsHandler.java deleted file mode 100644 index fb9bd454050f..000000000000 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/ReportsHandler.java +++ /dev/null @@ -1,154 +0,0 @@ -/* - * Copyright 2021 Collate - * Licensed 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.openmetadata.service.events.scheduled; - -import static org.openmetadata.schema.api.events.CreateEventSubscription.AlertType.DATA_INSIGHT_REPORT; -import static org.openmetadata.service.Entity.EVENT_SUBSCRIPTION; -import static org.openmetadata.service.util.SubscriptionUtil.getCronSchedule; - -import java.io.IOException; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import javax.ws.rs.BadRequestException; -import javax.ws.rs.core.Response; -import lombok.extern.slf4j.Slf4j; -import org.openmetadata.schema.entity.events.EventSubscription; -import org.openmetadata.schema.entity.events.TriggerConfig; -import org.openmetadata.service.Entity; -import org.openmetadata.service.apps.bundles.searchIndex.SearchIndexApp; -import org.openmetadata.service.exception.DataInsightJobException; -import org.openmetadata.service.jdbi3.DataInsightChartRepository; -import org.openmetadata.service.search.SearchRepository; -import org.quartz.JobBuilder; -import org.quartz.JobDataMap; -import org.quartz.JobDetail; -import org.quartz.Scheduler; -import org.quartz.SchedulerException; -import org.quartz.Trigger; -import org.quartz.TriggerBuilder; -import org.quartz.impl.StdSchedulerFactory; - -@Slf4j -public class ReportsHandler { - public static final String DATA_INSIGHT_EMAIL_JOB = "dataInsightEmailJob"; - public static final String EMAIL_REPORT = "emailReport"; - public static final String CRON_TRIGGER = "dataInsightEmailTrigger"; - public static final String JOB_CONTEXT_CHART_REPO = "dataInsightChartRepository"; - public static final String SEARCH_CLIENT = "searchClient"; - - private final SearchRepository searchRepository; - private final DataInsightChartRepository chartRepository; - private static ReportsHandler instance; - private static volatile boolean initialized = false; - private final Scheduler reportScheduler = new StdSchedulerFactory().getScheduler(); - private static final ConcurrentHashMap reportJobKeyMap = new ConcurrentHashMap<>(); - - private ReportsHandler() throws SchedulerException { - this.searchRepository = Entity.getSearchRepository(); - this.chartRepository = (DataInsightChartRepository) Entity.getEntityRepository(Entity.DATA_INSIGHT_CHART); - this.reportScheduler.start(); - } - - public static ReportsHandler getInstance() { - if (initialized) return instance; - throw new DataInsightJobException("Reports Job Handler is not Initialized"); - } - - public ConcurrentMap getReportMap() { - return reportJobKeyMap; - } - - public static void initialize() throws SchedulerException { - if (!initialized) { - instance = new ReportsHandler(); - initialized = true; - } else { - LOG.info("Reindexing Handler is already initialized"); - } - } - - public void addDataReportConfig(EventSubscription dataReport) { - try { - if (Boolean.TRUE.equals(dataReport.getEnabled())) { - JobDetail jobDetail = jobBuilder(dataReport); - Trigger trigger = trigger(dataReport.getTrigger()); - reportScheduler.scheduleJob(jobDetail, trigger); - reportJobKeyMap.put(dataReport.getId(), jobDetail); - } else { - LOG.info("[Data Insight Report Job] Job Not Scheduled since it is disabled"); - } - } catch (Exception ex) { - LOG.error("Failed in setting up job Scheduler for Data Reporting", ex); - } - } - - public void updateDataReportConfig(EventSubscription dataReport) throws SchedulerException { - deleteDataReportConfig(dataReport); - addDataReportConfig(dataReport); - } - - public void deleteDataReportConfig(EventSubscription dataReport) throws SchedulerException { - JobDetail jobDetail = getJobKey(dataReport.getId()); - if (jobDetail != null) { - reportScheduler.deleteJob(jobDetail.getKey()); - reportJobKeyMap.remove(dataReport.getId()); - } - } - - private JobDetail jobBuilder(EventSubscription subscription) throws IOException { - if (subscription.getAlertType() == DATA_INSIGHT_REPORT) { - JobDataMap dataMap = new JobDataMap(); - dataMap.put(JOB_CONTEXT_CHART_REPO, this.chartRepository); - dataMap.put(SEARCH_CLIENT, searchRepository); - dataMap.put(EVENT_SUBSCRIPTION, subscription); - JobBuilder jobBuilder = - JobBuilder.newJob(SearchIndexApp.class) - .withIdentity(DATA_INSIGHT_EMAIL_JOB, EMAIL_REPORT) - .usingJobData(dataMap); - return jobBuilder.build(); - } - throw new IOException("Invalid Report Type"); - } - - private Trigger trigger(TriggerConfig trigger) { - return TriggerBuilder.newTrigger() - .withIdentity(CRON_TRIGGER, EMAIL_REPORT) - .withSchedule(getCronSchedule(trigger)) - .build(); - } - - private JobDetail getJobKey(UUID id) { - return reportJobKeyMap.get(id); - } - - public static void shutDown() throws SchedulerException { - if (instance != null) { - instance.reportScheduler.shutdown(); - } - } - - public Response triggerExistingDataInsightJob(EventSubscription dataReport) throws SchedulerException { - JobDetail jobDetail = getJobKey(dataReport.getId()); - if (jobDetail != null) { - JobDataMap dataMap = new JobDataMap(); - dataMap.put(JOB_CONTEXT_CHART_REPO, this.chartRepository); - dataMap.put(SEARCH_CLIENT, searchRepository); - dataMap.put(EVENT_SUBSCRIPTION, dataReport); - reportScheduler.triggerJob(jobDetail.getKey(), dataMap); - return Response.status(Response.Status.OK).entity("Job Triggered Successfully.").build(); - } - throw new BadRequestException("Job with given Id does not exist"); - } -} diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java index 908dfb4485a8..488cc9893a18 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java @@ -39,13 +39,13 @@ import org.openmetadata.schema.type.ParamAdditionalContext; import org.openmetadata.schema.type.SubscriptionFilterOperation; import org.openmetadata.service.Entity; -import org.openmetadata.service.events.subscription.email.EmailPublisher; -import org.openmetadata.service.events.subscription.gchat.GChatPublisher; -import org.openmetadata.service.events.subscription.generic.GenericPublisher; -import org.openmetadata.service.events.subscription.msteams.MSTeamsPublisher; -import org.openmetadata.service.events.subscription.slack.SlackEventPublisher; +import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; +import org.openmetadata.service.apps.bundles.changeEvent.email.EmailPublisher; +import org.openmetadata.service.apps.bundles.changeEvent.gchat.GChatPublisher; +import org.openmetadata.service.apps.bundles.changeEvent.generic.GenericPublisher; +import org.openmetadata.service.apps.bundles.changeEvent.msteams.MSTeamsPublisher; +import org.openmetadata.service.apps.bundles.changeEvent.slack.SlackEventPublisher; import org.openmetadata.service.exception.CatalogExceptionMessage; -import org.openmetadata.service.jdbi3.CollectionDAO; import org.openmetadata.service.resources.CollectionRegistry; import org.openmetadata.service.search.models.IndexMapping; import org.springframework.expression.Expression; @@ -55,25 +55,24 @@ public final class AlertUtil { private AlertUtil() {} - public static SubscriptionPublisher getNotificationsPublisher( - EventSubscription subscription, CollectionDAO daoCollection) { + public static AbstractEventConsumer getNotificationsPublisher(EventSubscription subscription) { validateSubscriptionConfig(subscription); - SubscriptionPublisher publisher; + AbstractEventConsumer publisher; switch (subscription.getSubscriptionType()) { case SLACK_WEBHOOK: - publisher = new SlackEventPublisher(subscription, daoCollection); + publisher = new SlackEventPublisher(); break; case MS_TEAMS_WEBHOOK: - publisher = new MSTeamsPublisher(subscription, daoCollection); + publisher = new MSTeamsPublisher(); break; case G_CHAT_WEBHOOK: - publisher = new GChatPublisher(subscription, daoCollection); + publisher = new GChatPublisher(); break; case GENERIC_WEBHOOK: - publisher = new GenericPublisher(subscription, daoCollection); + publisher = new GenericPublisher(); break; case EMAIL: - publisher = new EmailPublisher(subscription, daoCollection); + publisher = new EmailPublisher(); break; case ACTIVITY_FEED: throw new IllegalArgumentException("Cannot create Activity Feed as Publisher."); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/exception/DataInsightJobException.java b/openmetadata-service/src/main/java/org/openmetadata/service/exception/DataInsightJobException.java deleted file mode 100644 index cc31cb87bb07..000000000000 --- a/openmetadata-service/src/main/java/org/openmetadata/service/exception/DataInsightJobException.java +++ /dev/null @@ -1,11 +0,0 @@ -package org.openmetadata.service.exception; - -public class DataInsightJobException extends RuntimeException { - public DataInsightJobException(String message) { - super(message); - } - - public DataInsightJobException(Throwable throwable) { - super(throwable); - } -} diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/exception/EventSubscriptionJobException.java b/openmetadata-service/src/main/java/org/openmetadata/service/exception/EventSubscriptionJobException.java new file mode 100644 index 000000000000..6944de845734 --- /dev/null +++ b/openmetadata-service/src/main/java/org/openmetadata/service/exception/EventSubscriptionJobException.java @@ -0,0 +1,11 @@ +package org.openmetadata.service.exception; + +public class EventSubscriptionJobException extends RuntimeException { + public EventSubscriptionJobException(String message) { + super(message); + } + + public EventSubscriptionJobException(Throwable throwable) { + super(throwable); + } +} diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/EmailMessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/EmailMessageDecorator.java index 75d21f116f0e..7e38dce5b7ef 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/EmailMessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/EmailMessageDecorator.java @@ -20,9 +20,10 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import org.openmetadata.schema.EntityInterface; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.service.Entity; -import org.openmetadata.service.events.subscription.email.EmailMessage; +import org.openmetadata.service.apps.bundles.changeEvent.email.EmailMessage; import org.openmetadata.service.resources.feeds.MessageParser; public class EmailMessageDecorator implements MessageDecorator { @@ -66,16 +67,17 @@ public String getEntityUrl(String entityType, String fqn) { public EmailMessage buildMessage(ChangeEvent event) { EmailMessage emailMessage = new EmailMessage(); emailMessage.setUserName(event.getUserName()); + EntityInterface entityInterface = getEntity(event); if (event.getEntity() != null) { emailMessage.setUpdatedBy(event.getUserName()); if (event.getEntityType().equals(Entity.QUERY)) { emailMessage.setEntityUrl(Entity.QUERY); } else { - emailMessage.setEntityUrl(this.getEntityUrl(event.getEntityType(), event.getEntityFullyQualifiedName())); + emailMessage.setEntityUrl(this.buildEntityUrl(event.getEntityType(), entityInterface)); } } Map messages = - getFormattedMessages(this, event.getChangeDescription(), getEntity(event)); + getFormattedMessages(this, event.getChangeDescription(), entityInterface); List changeMessage = new ArrayList<>(); for (Map.Entry entry : messages.entrySet()) { changeMessage.add(entry.getValue()); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java index 00a259e393fd..570deeadda1f 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java @@ -20,8 +20,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import org.openmetadata.schema.EntityInterface; import org.openmetadata.schema.type.ChangeEvent; -import org.openmetadata.service.events.subscription.gchat.GChatMessage; +import org.openmetadata.service.apps.bundles.changeEvent.gchat.GChatMessage; import org.openmetadata.service.resources.feeds.MessageParser; public class GChatMessageDecorator implements MessageDecorator { @@ -69,6 +70,7 @@ public GChatMessage buildMessage(ChangeEvent event) { GChatMessage.CardsV2 cardsV2 = new GChatMessage.CardsV2(); GChatMessage.Card card = new GChatMessage.Card(); GChatMessage.Section section = new GChatMessage.Section(); + EntityInterface entityInterface = getEntity(event); if (event.getEntity() != null) { String headerTemplate = "%s posted on %s %s"; String headerText = @@ -76,16 +78,16 @@ public GChatMessage buildMessage(ChangeEvent event) { headerTemplate, event.getUserName(), event.getEntityType(), - this.getEntityUrl(event.getEntityType(), event.getEntityFullyQualifiedName())); + this.buildEntityUrl(event.getEntityType(), entityInterface)); gChatMessage.setText(headerText); GChatMessage.CardHeader cardHeader = new GChatMessage.CardHeader(); String cardHeaderText = - String.format(headerTemplate, event.getUserName(), event.getEntityType(), (getEntity(event)).getName()); + String.format(headerTemplate, event.getUserName(), event.getEntityType(), entityInterface.getName()); cardHeader.setTitle(cardHeaderText); card.setHeader(cardHeader); } Map messages = - getFormattedMessages(this, event.getChangeDescription(), getEntity(event)); + getFormattedMessages(this, event.getChangeDescription(), entityInterface); List widgets = new ArrayList<>(); for (Map.Entry entry : messages.entrySet()) { GChatMessage.Widget widget = new GChatMessage.Widget(); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java index 4e82a502d4d5..a23e6b6a1ce9 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java @@ -20,8 +20,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import org.openmetadata.schema.EntityInterface; import org.openmetadata.schema.type.ChangeEvent; -import org.openmetadata.service.events.subscription.msteams.TeamsMessage; +import org.openmetadata.service.apps.bundles.changeEvent.msteams.TeamsMessage; import org.openmetadata.service.resources.feeds.MessageParser; public class MSTeamsMessageDecorator implements MessageDecorator { @@ -66,17 +67,15 @@ public TeamsMessage buildMessage(ChangeEvent event) { TeamsMessage teamsMessage = new TeamsMessage(); teamsMessage.setSummary("Change Event From OMD"); TeamsMessage.Section teamsSections = new TeamsMessage.Section(); + EntityInterface entityInterface = getEntity(event); if (event.getEntity() != null) { String headerTxt = "%s posted on " + event.getEntityType() + " %s"; String headerText = - String.format( - headerTxt, - event.getUserName(), - this.getEntityUrl(event.getEntityType(), event.getEntityFullyQualifiedName())); + String.format(headerTxt, event.getUserName(), this.buildEntityUrl(event.getEntityType(), entityInterface)); teamsSections.setActivityTitle(headerText); } Map messages = - getFormattedMessages(this, event.getChangeDescription(), getEntity(event)); + getFormattedMessages(this, event.getChangeDescription(), entityInterface); List attachmentList = new ArrayList<>(); for (Map.Entry entry : messages.entrySet()) { TeamsMessage.Section section = new TeamsMessage.Section(); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MessageDecorator.java index fd4b4ca26e06..5a5c6404456d 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MessageDecorator.java @@ -16,7 +16,11 @@ import java.util.LinkedList; import org.apache.commons.lang3.StringUtils; import org.bitbucket.cowwoc.diffmatchpatch.DiffMatchPatch; +import org.openmetadata.common.utils.CommonUtil; +import org.openmetadata.schema.EntityInterface; import org.openmetadata.schema.type.ChangeEvent; +import org.openmetadata.schema.type.Include; +import org.openmetadata.service.Entity; public interface MessageDecorator { String getBold(); @@ -33,6 +37,15 @@ public interface MessageDecorator { String getEntityUrl(String entityType, String fqn); + default String buildEntityUrl(String entityType, EntityInterface entityInterface) { + String fqn = entityInterface.getFullyQualifiedName(); + if (CommonUtil.nullOrEmpty(fqn)) { + EntityInterface result = Entity.getEntity(entityType, entityInterface.getId(), "id", Include.NON_DELETED); + fqn = result.getFullyQualifiedName(); + } + return getEntityUrl(entityType, fqn); + } + default String httpAddMarker() { return ""; } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java index 9c9f3fe7e631..d2823e9f0d39 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java @@ -20,11 +20,12 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import org.openmetadata.schema.EntityInterface; import org.openmetadata.schema.tests.TestCase; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.service.Entity; -import org.openmetadata.service.events.subscription.slack.SlackAttachment; -import org.openmetadata.service.events.subscription.slack.SlackMessage; +import org.openmetadata.service.apps.bundles.changeEvent.slack.SlackAttachment; +import org.openmetadata.service.apps.bundles.changeEvent.slack.SlackMessage; import org.openmetadata.service.resources.feeds.MessageParser; public class SlackMessageDecorator implements MessageDecorator { @@ -69,6 +70,7 @@ public String getEntityUrl(String entityType, String fqn) { public SlackMessage buildMessage(ChangeEvent event) { SlackMessage slackMessage = new SlackMessage(); slackMessage.setUsername(event.getUserName()); + EntityInterface entityInterface = getEntity(event); if (event.getEntity() != null) { String eventType; if (event.getEntity() instanceof TestCase) { @@ -84,15 +86,12 @@ public SlackMessage buildMessage(ChangeEvent event) { } else { headerTxt = "%s posted on " + eventType + " %s"; headerText = - String.format( - headerTxt, - event.getUserName(), - this.getEntityUrl(event.getEntityType(), event.getEntityFullyQualifiedName())); + String.format(headerTxt, event.getUserName(), this.buildEntityUrl(event.getEntityType(), entityInterface)); } slackMessage.setText(headerText); } Map messages = - getFormattedMessages(this, event.getChangeDescription(), getEntity(event)); + getFormattedMessages(this, event.getChangeDescription(), entityInterface); List attachmentList = new ArrayList<>(); for (Map.Entry entry : messages.entrySet()) { SlackAttachment attachment = new SlackAttachment(); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/entity/TestCaseFormatter.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/entity/TestCaseFormatter.java index 35d6681e324a..843bc5042860 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/entity/TestCaseFormatter.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/entity/TestCaseFormatter.java @@ -23,6 +23,7 @@ import org.openmetadata.service.formatter.decorators.MessageDecorator; import org.openmetadata.service.formatter.util.FormatterUtil; import org.openmetadata.service.resources.feeds.MessageParser; +import org.openmetadata.service.util.JsonUtils; public class TestCaseFormatter implements EntityFormatter { private static final String TEST_RESULT_FIELD = "testCaseResult"; @@ -42,7 +43,7 @@ public String format( private String transformTestCaseResult( MessageDecorator messageFormatter, FieldChange fieldChange, EntityInterface entity) { String testCaseName = entity.getName(); - TestCaseResult result = (TestCaseResult) fieldChange.getNewValue(); + TestCaseResult result = JsonUtils.convertValue(fieldChange.getNewValue(), TestCaseResult.class); TestCase testCaseEntity = (TestCase) entity; if (result != null) { String format = diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java index 6d32ea3ea79d..c54c8ce2755a 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java @@ -1525,6 +1525,27 @@ default String getNameHashColumn() { default boolean supportsSoftDelete() { return false; } + + @SqlQuery("SELECT json FROM event_subscription_extension where id = :id AND extension = :extension") + String getSubscriberOffset(@Bind("id") String id, @Bind("extension") String extension); + + @ConnectionAwareSqlUpdate( + value = + "INSERT INTO event_subscription_extension(id, extension, jsonSchema, json) " + + "VALUES (:id, :extension, :jsonSchema, :json)" + + "ON DUPLICATE KEY UPDATE json = :json, jsonSchema = :jsonSchema", + connectionType = MYSQL) + @ConnectionAwareSqlUpdate( + value = + "INSERT INTO event_subscription_extension(id, extension, jsonSchema, json) " + + "VALUES (:id, :extension, :jsonSchema, (:json :: jsonb)) " + + "DO UPDATE SET json = EXCLUDED.json, jsonSchema = EXCLUDED.jsonSchema", + connectionType = POSTGRES) + void upsertSubscriberOffset( + @Bind("id") String id, + @Bind("extension") String extension, + @Bind("jsonSchema") String jsonSchema, + @Bind("json") String json); } interface ChartDAO extends EntityDAO { @@ -3066,6 +3087,12 @@ List listWithEntityFilter( + "eventType = :eventType AND eventTime >= :timestamp " + "ORDER BY eventTime ASC") List listWithoutEntityFilter(@Bind("eventType") String eventType, @Bind("timestamp") long timestamp); + + @SqlQuery("SELECT json FROM change_event ORDER BY eventTime ASC LIMIT :limit OFFSET :offset") + List list(@Bind("limit") long limit, @Bind("offset") long offset); + + @SqlQuery("SELECT count(*) FROM change_event") + int listCount(); } interface TypeEntityDAO extends EntityDAO { diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EventSubscriptionRepository.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EventSubscriptionRepository.java index 0001522b9433..063748e86594 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EventSubscriptionRepository.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EventSubscriptionRepository.java @@ -13,33 +13,19 @@ package org.openmetadata.service.jdbi3; -import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.ACTIVITY_FEED; - -import com.lmax.disruptor.BatchEventProcessor; import java.util.Comparator; import java.util.List; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; -import org.jdbi.v3.sqlobject.transaction.Transaction; import org.openmetadata.schema.entity.events.EventFilterRule; import org.openmetadata.schema.entity.events.EventSubscription; -import org.openmetadata.schema.entity.events.SubscriptionStatus; import org.openmetadata.service.Entity; -import org.openmetadata.service.events.EventPubSub; -import org.openmetadata.service.events.scheduled.ReportsHandler; +import org.openmetadata.service.events.scheduled.EventSubscriptionScheduler; import org.openmetadata.service.events.subscription.AlertUtil; -import org.openmetadata.service.events.subscription.SubscriptionPublisher; import org.openmetadata.service.resources.events.subscription.EventSubscriptionResource; import org.openmetadata.service.util.EntityUtil.Fields; -import org.quartz.SchedulerException; @Slf4j public class EventSubscriptionRepository extends EntityRepository { - private static final String INVALID_ALERT = "Invalid Alert Type"; - private static final ConcurrentHashMap subscriptionPublisherMap = - new ConcurrentHashMap<>(); static final String ALERT_PATCH_FIELDS = "trigger,enabled,batchSize,timeout"; static final String ALERT_UPDATE_FIELDS = "trigger,enabled,batchSize,timeout,filteringRules"; @@ -56,7 +42,10 @@ public EventSubscriptionRepository() { @Override public void setFields(EventSubscription entity, Fields fields) { if (entity.getStatusDetails() == null) { - entity.withStatusDetails(fields.contains("statusDetails") ? getStatusForEventSubscription(entity.getId()) : null); + entity.withStatusDetails( + fields.contains("statusDetails") + ? EventSubscriptionScheduler.getInstance().getStatusForEventSubscription(entity.getId()) + : null); } } @@ -92,118 +81,6 @@ public void storeRelationships(EventSubscription entity) { // No relationships to store beyond what is stored in the super class } - private SubscriptionPublisher getPublisher(UUID id) { - return subscriptionPublisherMap.get(id); - } - - @Transaction - public void addSubscriptionPublisher(EventSubscription eventSubscription) { - switch (eventSubscription.getAlertType()) { - case CHANGE_EVENT: - SubscriptionPublisher publisher = AlertUtil.getNotificationsPublisher(eventSubscription, daoCollection); - if (Boolean.FALSE.equals( - eventSubscription.getEnabled())) { // Only add webhook that is enabled for publishing events - eventSubscription.setStatusDetails(getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.DISABLED)); - } else { - eventSubscription.setStatusDetails(getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.ACTIVE)); - BatchEventProcessor processor = EventPubSub.addEventHandler(publisher); - publisher.setProcessor(processor); - } - subscriptionPublisherMap.put(eventSubscription.getId(), publisher); - LOG.info( - "Webhook publisher subscription started as {} : status {}", - eventSubscription.getName(), - eventSubscription.getStatusDetails().getStatus()); - break; - case DATA_INSIGHT_REPORT: - if (Boolean.TRUE.equals(eventSubscription.getEnabled())) { - ReportsHandler.getInstance().addDataReportConfig(eventSubscription); - } - break; - default: - throw new IllegalArgumentException(INVALID_ALERT); - } - } - - private SubscriptionStatus getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status status) { - return new SubscriptionStatus().withStatus(status).withTimestamp(System.currentTimeMillis()); - } - - @Transaction - @SneakyThrows - public void updateEventSubscription(EventSubscription eventSubscription) { - switch (eventSubscription.getAlertType()) { - case CHANGE_EVENT: - if (Boolean.TRUE.equals(eventSubscription.getEnabled())) { // Only add webhook that is enabled for publishing - // If there was a previous webhook either in disabled state or stopped due - // to errors, update it and restart publishing - SubscriptionPublisher previousPublisher = getPublisher(eventSubscription.getId()); - if (previousPublisher == null) { - if (!ACTIVITY_FEED.equals(eventSubscription.getSubscriptionType())) { - addSubscriptionPublisher(eventSubscription); - } - return; - } - - // Update the existing publisher - deleteEventSubscriptionPublisher(eventSubscription); - addSubscriptionPublisher(eventSubscription); - } else { - // Remove the webhook publisher - removeProcessorForEventSubscription( - eventSubscription.getId(), getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.DISABLED)); - } - break; - case DATA_INSIGHT_REPORT: - ReportsHandler.getInstance().updateDataReportConfig(eventSubscription); - break; - default: - throw new IllegalArgumentException(INVALID_ALERT); - } - } - - @Transaction - public void removeProcessorForEventSubscription(UUID id, SubscriptionStatus reasonForRemoval) - throws InterruptedException { - SubscriptionPublisher publisher = subscriptionPublisherMap.get(id); - if (publisher != null) { - publisher.getProcessor().halt(); - publisher.awaitShutdown(); - EventPubSub.removeProcessor(publisher.getProcessor()); - publisher.getEventSubscription().setStatusDetails(reasonForRemoval); - LOG.info("Webhook publisher deleted for {}", publisher.getEventSubscription().getName()); - } - } - - @Transaction - public void deleteEventSubscriptionPublisher(EventSubscription deletedEntity) - throws InterruptedException, SchedulerException { - switch (deletedEntity.getAlertType()) { - case CHANGE_EVENT: - SubscriptionPublisher publisher = subscriptionPublisherMap.remove(deletedEntity.getId()); - if (publisher != null && publisher.getProcessor() != null) { - publisher.getProcessor().halt(); - publisher.awaitShutdown(); - EventPubSub.removeProcessor(publisher.getProcessor()); - LOG.info("Webhook publisher deleted for {}", publisher.getEventSubscription().getName()); - } - break; - case DATA_INSIGHT_REPORT: - ReportsHandler.getInstance().deleteDataReportConfig(deletedEntity); - break; - default: - throw new IllegalArgumentException(INVALID_ALERT); - } - } - - public SubscriptionStatus getStatusForEventSubscription(UUID id) { - SubscriptionPublisher publisher = subscriptionPublisherMap.get(id); - if (publisher != null) { - return publisher.getEventSubscription().getStatusDetails(); - } - return null; - } - @Override public EventSubscriptionUpdater getUpdater( EventSubscription original, EventSubscription updated, Operation operation) { diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java b/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java index 47323e7ba849..3be89140718a 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java @@ -34,7 +34,6 @@ import javax.validation.Valid; import javax.validation.constraints.Max; import javax.validation.constraints.Min; -import javax.ws.rs.BadRequestException; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.DefaultValue; @@ -58,15 +57,13 @@ import org.openmetadata.schema.entity.events.SubscriptionStatus; import org.openmetadata.schema.type.EntityHistory; import org.openmetadata.schema.type.Function; -import org.openmetadata.schema.type.Include; import org.openmetadata.schema.type.MetadataOperation; import org.openmetadata.schema.type.SubscriptionResourceDescriptor; import org.openmetadata.service.Entity; import org.openmetadata.service.OpenMetadataApplicationConfig; -import org.openmetadata.service.events.scheduled.ReportsHandler; +import org.openmetadata.service.events.scheduled.EventSubscriptionScheduler; import org.openmetadata.service.events.subscription.AlertUtil; import org.openmetadata.service.events.subscription.EventsSubscriptionRegistry; -import org.openmetadata.service.exception.EntityNotFoundException; import org.openmetadata.service.jdbi3.CollectionDAO; import org.openmetadata.service.jdbi3.EventSubscriptionRepository; import org.openmetadata.service.jdbi3.ListFilter; @@ -114,7 +111,6 @@ public void initialize(OpenMetadataApplicationConfig config) { try { repository.initSeedDataFromResources(); EventsSubscriptionRegistry.initialize(listOrEmpty(EventSubscriptionResource.getDescriptors())); - ReportsHandler.initialize(); initializeEventSubscriptions(); } catch (Exception ex) { // Starting application should not fail @@ -130,12 +126,11 @@ private void initializeEventSubscriptions() { .eventSubscriptionDAO() .listAllEventsSubscriptions(daoCollection.eventSubscriptionDAO().getTableName()); List eventSubList = JsonUtils.readObjects(listAllEventsSubscriptions, EventSubscription.class); - eventSubList.forEach( - subscription -> { - if (subscription.getSubscriptionType() != ACTIVITY_FEED) { - repository.addSubscriptionPublisher(subscription); - } - }); + for (EventSubscription subscription : eventSubList) { + if (subscription.getSubscriptionType() != ACTIVITY_FEED) { + EventSubscriptionScheduler.getInstance().addSubscriptionPublisher(subscription); + } + } } catch (Exception ex) { // Starting application should not fail LOG.warn("Exception during initializeEventSubscriptions", ex); @@ -261,13 +256,8 @@ public Response createEventSubscription( @Context UriInfo uriInfo, @Context SecurityContext securityContext, @Valid CreateEventSubscription request) { EventSubscription eventSub = getEventSubscription(request, securityContext.getUserPrincipal().getName()); // Only one Creation is allowed - if (eventSub.getAlertType() == CreateEventSubscription.AlertType.DATA_INSIGHT_REPORT - && ReportsHandler.getInstance() != null - && ReportsHandler.getInstance().getReportMap().size() > 0) { - throw new BadRequestException("Data Insight Report Alert already exists."); - } Response response = create(uriInfo, securityContext, eventSub); - repository.addSubscriptionPublisher(eventSub); + EventSubscriptionScheduler.getInstance().addSubscriptionPublisher(eventSub); return response; } @@ -288,43 +278,12 @@ public Response createEventSubscription( }) public Response createOrUpdateEventSubscription( @Context UriInfo uriInfo, @Context SecurityContext securityContext, @Valid CreateEventSubscription create) { - // Only one Creation is allowed for Data Insight - if (create.getAlertType() == CreateEventSubscription.AlertType.DATA_INSIGHT_REPORT) { - try { - repository.getByName(null, create.getName(), repository.getFields("id")); - } catch (EntityNotFoundException ex) { - if (ReportsHandler.getInstance() != null && ReportsHandler.getInstance().getReportMap().size() > 0) { - throw new BadRequestException("Data Insight Report Alert already exists."); - } - } - } EventSubscription eventSub = getEventSubscription(create, securityContext.getUserPrincipal().getName()); Response response = createOrUpdate(uriInfo, securityContext, eventSub); - repository.updateEventSubscription((EventSubscription) response.getEntity()); + EventSubscriptionScheduler.getInstance().updateEventSubscription((EventSubscription) response.getEntity()); return response; } - @PUT - @Path("/trigger/{id}") - @Operation( - operationId = "triggerDataInsightJob", - summary = "Trigger a existing Data Insight Report Job to run", - description = "Trigger a existing Data Insight Report Job to run", - responses = { - @ApiResponse(responseCode = "200", description = "Trigger a Data Insight Job"), - @ApiResponse(responseCode = "400", description = "Bad request") - }) - public Response triggerDataInsightJob( - @Context UriInfo uriInfo, - @Context SecurityContext securityContext, - @Parameter(description = "Id of the event Subscription", schema = @Schema(type = "UUID")) @PathParam("id") - UUID id) - throws SchedulerException { - authorizer.authorizeAdmin(securityContext); - EventSubscription eventSub = repository.find(id, Include.NON_DELETED); - return ReportsHandler.getInstance().triggerExistingDataInsightJob(eventSub); - } - @PATCH @Path("/{id}") @Operation( @@ -346,7 +305,7 @@ public Response patchEventSubscription( examples = {@ExampleObject("[{op:remove, path:/a},{op:add, path: /b, value: val}]")})) JsonPatch patch) { Response response = patchInternal(uriInfo, securityContext, id, patch); - repository.updateEventSubscription((EventSubscription) response.getEntity()); + EventSubscriptionScheduler.getInstance().updateEventSubscription((EventSubscription) response.getEntity()); return response; } @@ -419,11 +378,10 @@ public Response deleteEventSubscription( @Context SecurityContext securityContext, @Parameter(description = "Id of the Event Subscription", schema = @Schema(type = "UUID")) @PathParam("id") UUID id) - throws InterruptedException, SchedulerException { - Response response = delete(uriInfo, securityContext, id, true, true); - EventSubscription deletedEntity = (EventSubscription) response.getEntity(); - repository.deleteEventSubscriptionPublisher(deletedEntity); - return response; + throws SchedulerException { + EventSubscription eventSubscription = repository.get(null, id, repository.getFields("id")); + EventSubscriptionScheduler.getInstance().deleteEventSubscriptionPublisher(eventSubscription); + return delete(uriInfo, securityContext, id, true, true); } @DELETE @@ -441,11 +399,10 @@ public Response deleteEventSubscriptionByName( @Context SecurityContext securityContext, @Parameter(description = "Name of the Event Subscription", schema = @Schema(type = "string")) @PathParam("name") String name) - throws InterruptedException, SchedulerException { - Response response = deleteByName(uriInfo, securityContext, name, true, true); - EventSubscription deletedEntity = (EventSubscription) response.getEntity(); - repository.deleteEventSubscriptionPublisher(deletedEntity); - return response; + throws SchedulerException { + EventSubscription eventSubscription = repository.getByName(null, name, repository.getFields("id")); + EventSubscriptionScheduler.getInstance().deleteEventSubscriptionPublisher(eventSubscription); + return deleteByName(uriInfo, securityContext, name, true, true); } @GET @@ -470,7 +427,7 @@ public SubscriptionStatus getEventSubscriptionStatusByName( @PathParam("eventSubscriptionName") String name) { EventSubscription sub = repository.getByName(null, name, repository.getFields("name")); - return repository.getStatusForEventSubscription(sub.getId()); + return EventSubscriptionScheduler.getInstance().getStatusForEventSubscription(sub.getId()); } @GET @@ -494,7 +451,7 @@ public SubscriptionStatus getEventSubscriptionStatusById( @Parameter(description = "Name of the Event Subscription", schema = @Schema(type = "UUID")) @PathParam("eventSubscriptionId") UUID id) { - return repository.getStatusForEventSubscription(id); + return EventSubscriptionScheduler.getInstance().getStatusForEventSubscription(id); } @GET diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/util/EmailUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/util/EmailUtil.java index e6e0404152ca..b9cecba5a8c1 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/util/EmailUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/util/EmailUtil.java @@ -34,9 +34,9 @@ import org.openmetadata.schema.entity.feed.Thread; import org.openmetadata.schema.entity.teams.User; import org.openmetadata.schema.settings.SettingsType; +import org.openmetadata.service.apps.bundles.changeEvent.email.EmailMessage; import org.openmetadata.service.events.scheduled.template.DataInsightDescriptionAndOwnerTemplate; import org.openmetadata.service.events.scheduled.template.DataInsightTotalAssetTemplate; -import org.openmetadata.service.events.subscription.email.EmailMessage; import org.openmetadata.service.resources.settings.SettingsCache; import org.simplejavamail.api.email.Email; import org.simplejavamail.api.email.EmailPopulatingBuilder; diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java index a5a96f1abbda..3e8b31593e36 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java @@ -42,7 +42,7 @@ import org.openmetadata.schema.type.Relationship; import org.openmetadata.schema.type.Webhook; import org.openmetadata.service.Entity; -import org.openmetadata.service.events.subscription.SubscriptionPublisher; +import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; import org.openmetadata.service.jdbi3.CollectionDAO; import org.openmetadata.service.jdbi3.ListFilter; import org.openmetadata.service.jdbi3.UserRepository; @@ -192,22 +192,19 @@ public static Set buildReceiversListFromActions( } public static List getTargetsForWebhook( - SubscriptionAction action, - CreateEventSubscription.SubscriptionType type, - Client client, - CollectionDAO daoCollection, - ChangeEvent event) { + SubscriptionAction action, CreateEventSubscription.SubscriptionType type, Client client, ChangeEvent event) { EntityInterface entityInterface = getEntity(event); List targets = new ArrayList<>(); Set receiversUrls = - buildReceiversListFromActions(action, type, daoCollection, entityInterface.getId(), event.getEntityType()); + buildReceiversListFromActions( + action, type, Entity.getCollectionDAO(), entityInterface.getId(), event.getEntityType()); for (String url : receiversUrls) { targets.add(client.target(url).request()); } return targets; } - public static void postWebhookMessage(SubscriptionPublisher publisher, Invocation.Builder target, Object message) + public static void postWebhookMessage(AbstractEventConsumer publisher, Invocation.Builder target, Object message) throws InterruptedException { long attemptTime = System.currentTimeMillis(); Response response = target.post(javax.ws.rs.client.Entity.entity(message, MediaType.APPLICATION_JSON_TYPE)); @@ -223,7 +220,7 @@ public static void postWebhookMessage(SubscriptionPublisher publisher, Invocatio // 4xx, 5xx response retry delivering events after timeout publisher.setNextBackOff(); publisher.setAwaitingRetry(attemptTime, response.getStatus(), response.getStatusInfo().getReasonPhrase()); - Thread.sleep(publisher.getCurrentBackOff()); + Thread.sleep(publisher.getCurrentBackoffTime()); } else if (response.getStatus() == 200) { publisher.setSuccessStatus(System.currentTimeMillis()); } diff --git a/openmetadata-service/src/main/resources/json/data/eventsubscription/DataInsightReportAlert.json b/openmetadata-service/src/main/resources/json/data/eventsubscription/DataInsightReportAlert.json deleted file mode 100644 index 66f8b66b779a..000000000000 --- a/openmetadata-service/src/main/resources/json/data/eventsubscription/DataInsightReportAlert.json +++ /dev/null @@ -1,17 +0,0 @@ -{ - "name": "DataInsightReport", - "fullyQualifiedName": "DataInsightReport", - "description": "Data Insight Report send to the admin (organization level) and teams (team level) at given interval.", - "alertType": "DataInsightReport", - "trigger": { - "triggerType": "Scheduled", - "scheduleInfo": "Weekly" - }, - "subscriptionType": "DataInsight", - "subscriptionConfig": { - "sendToAdmins": true, - "sendToTeams": true - }, - "provider" : "system", - "enabled" : true -} \ No newline at end of file diff --git a/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json b/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json index b645823be8fc..5f73146b1e93 100644 --- a/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json +++ b/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json @@ -16,7 +16,7 @@ "alertType": { "description": "Type of Alerts supported.", "type": "string", - "enum": ["ChangeEvent", "DataInsightReport", "Task/Conversation/Announcement"], + "enum": ["ChangeEvent", "Task/Conversation/Announcement"], "default": "ChangeEvent" }, "subscriptionType": { @@ -243,6 +243,11 @@ }, "provider" : { "$ref": "../type/basic.json#/definitions/providerType" + }, + "retries": { + "description": "Number of times to retry callback on failure. (Default 3).", + "type": "integer", + "default": 3 } }, "required": ["id", "name", "alertType", "subscriptionType"], diff --git a/openmetadata-spec/src/main/resources/json/schema/events/eventSubscriptionOffset.json b/openmetadata-spec/src/main/resources/json/schema/events/eventSubscriptionOffset.json new file mode 100644 index 000000000000..464cab327345 --- /dev/null +++ b/openmetadata-spec/src/main/resources/json/schema/events/eventSubscriptionOffset.json @@ -0,0 +1,20 @@ +{ + "$id": "https://open-metadata.org/schema/events/eventSubscriptionOffset.json", + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "EventSubscriptionOffset", + "description": "Event Subscription Offset", + "type": "object", + "javaType": "org.openmetadata.schema.entity.events.EventSubscriptionOffset", + "properties": { + "offset": { + "description": "Name of this Event Filter.", + "type": "integer" + }, + "timestamp": { + "description": "Update time of the job status.", + "$ref": "../type/basic.json#/definitions/timestamp" + } + }, + "required": ["offset"], + "additionalProperties": false +} From 72a8a5f0a874ba94531ee0d173d5e01107415193 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Tue, 19 Dec 2023 10:26:07 +0530 Subject: [PATCH 02/28] Spotless Fix --- .../changeEvent/AbstractEventConsumer.java | 31 ++++++++++---- .../changeEvent/email/EmailPublisher.java | 3 +- .../changeEvent/gchat/GChatPublisher.java | 9 ++-- .../changeEvent/generic/GenericPublisher.java | 6 ++- .../changeEvent/msteams/MSTeamsPublisher.java | 9 ++-- .../slack/SlackEventPublisher.java | 6 ++- .../scheduled/EventSubscriptionScheduler.java | 42 +++++++++++++------ .../events/scheduled/ReportsHandler.java | 1 + .../decorators/GChatMessageDecorator.java | 6 ++- .../decorators/MSTeamsMessageDecorator.java | 5 ++- .../decorators/MessageDecorator.java | 3 +- .../decorators/SlackMessageDecorator.java | 5 ++- .../service/jdbi3/CollectionDAO.java | 5 ++- .../jdbi3/EventSubscriptionRepository.java | 3 +- .../EventSubscriptionResource.java | 22 ++++++---- .../service/util/SubscriptionUtil.java | 17 ++++++-- 16 files changed, 122 insertions(+), 51 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index 3b69a30caa78..1f0ac10b42b9 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -58,7 +58,8 @@ public abstract class AbstractEventConsumer implements Consumer, Jo protected AbstractEventConsumer() {} private void init(JobExecutionContext context) { - EventSubscription sub = (EventSubscription) context.getJobDetail().getJobDataMap().get(ALERT_INFO_KEY); + EventSubscription sub = + (EventSubscription) context.getJobDetail().getJobDataMap().get(ALERT_INFO_KEY); this.jobDetail = context.getJobDetail(); this.eventSubscription = sub; this.offset = loadInitialOffset(); @@ -85,7 +86,8 @@ private int loadInitialOffset() { .eventSubscriptionDAO() .getSubscriberOffset(eventSubscription.getId().toString(), OFFSET_EXTENSION); if (json != null) { - EventSubscriptionOffset offsetFromDb = JsonUtils.readValue(json, EventSubscriptionOffset.class); + EventSubscriptionOffset offsetFromDb = + JsonUtils.readValue(json, EventSubscriptionOffset.class); eventSubscriptionOffset = offsetFromDb.getOffset(); } else { eventSubscriptionOffset = Entity.getCollectionDAO().changeEventDAO().listCount(); @@ -103,11 +105,13 @@ public boolean publishEvents(List events) throws InterruptedExcepti List filteredEvents = new ArrayList<>(); for (ChangeEvent event : events) { boolean triggerChangeEvent = - AlertUtil.shouldTriggerAlert(event.getEntityType(), eventSubscription.getFilteringRules()); + AlertUtil.shouldTriggerAlert( + event.getEntityType(), eventSubscription.getFilteringRules()); // Evaluate ChangeEvent Alert Filtering if (eventSubscription.getFilteringRules() != null - && !AlertUtil.evaluateAlertConditions(event, eventSubscription.getFilteringRules().getRules())) { + && !AlertUtil.evaluateAlertConditions( + event, eventSubscription.getFilteringRules().getRules())) { triggerChangeEvent = false; } @@ -116,7 +120,9 @@ public boolean publishEvents(List events) throws InterruptedExcepti if (event.getChangeDescription() != null) { filteredEvents.add(event); } else { - LOG.info("Email Publisher Event Will be Ignored Since Change Description is null. Received Event: {}", event); + LOG.info( + "Email Publisher Event Will be Ignored Since Change Description is null. Received Event: {}", + event); } } } @@ -164,14 +170,20 @@ public synchronized void setAwaitingRetry(Long attemptTime, int statusCode, Stri public synchronized void setSuccessStatus(Long updateTime) { SubscriptionStatus subStatus = - AlertUtil.buildSubscriptionStatus(ACTIVE, updateTime, null, null, null, updateTime, updateTime); + AlertUtil.buildSubscriptionStatus( + ACTIVE, updateTime, null, null, null, updateTime, updateTime); eventSubscription.setStatusDetails(subStatus); } protected synchronized void setStatus( - SubscriptionStatus.Status status, Long attemptTime, Integer statusCode, String reason, Long timestamp) { + SubscriptionStatus.Status status, + Long attemptTime, + Integer statusCode, + String reason, + Long timestamp) { SubscriptionStatus subStatus = - AlertUtil.buildSubscriptionStatus(status, null, attemptTime, statusCode, reason, timestamp, attemptTime); + AlertUtil.buildSubscriptionStatus( + status, null, attemptTime, statusCode, reason, timestamp, attemptTime); eventSubscription.setStatusDetails(subStatus); } @@ -209,7 +221,8 @@ public void execute(JobExecutionContext jobExecutionContext) throws JobExecution } catch (Exception e) { handleException(e); } finally { - LOG.debug("Committing offset for eventSubscription {} {}", eventSubscription.getName(), offset); + LOG.debug( + "Committing offset for eventSubscription {} {}", eventSubscription.getName(), offset); commitOffset(jobExecutionContext, offset); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailPublisher.java index a40ee12e609a..864ef0635ddc 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailPublisher.java @@ -44,7 +44,8 @@ public class EmailPublisher extends AbstractEventConsumer { @Override protected void doInit(JobExecutionContext context) { if (eventSubscription.getSubscriptionType() == EMAIL) { - this.emailAlertConfig = JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), EmailAlertConfig.class); + this.emailAlertConfig = + JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), EmailAlertConfig.class); this.daoCollection = Entity.getCollectionDAO(); } else { throw new IllegalArgumentException("Email Alert Invoked with Illegal Type and Settings."); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java index dd65e7c60f3c..8bf82b8bae63 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java @@ -35,7 +35,8 @@ @Slf4j public class GChatPublisher extends AbstractEventConsumer { - private final MessageDecorator gChatMessageMessageDecorator = new GChatMessageDecorator(); + private final MessageDecorator gChatMessageMessageDecorator = + new GChatMessageDecorator(); private Webhook webhook; private Invocation.Builder target; private Client client; @@ -43,7 +44,8 @@ public class GChatPublisher extends AbstractEventConsumer { @Override protected void doInit(JobExecutionContext context) { if (eventSubscription.getSubscriptionType() == G_CHAT_WEBHOOK) { - this.webhook = JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); + this.webhook = + JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); // Build Client client = getClient(eventSubscription.getTimeout(), eventSubscription.getReadTimeout()); @@ -65,7 +67,8 @@ public void sendAlert(List changeEvents) { for (ChangeEvent event : changeEvents) { try { GChatMessage gchatMessage = gChatMessageMessageDecorator.buildMessage(event); - List targets = getTargetsForWebhook(webhook, G_CHAT_WEBHOOK, client, event); + List targets = + getTargetsForWebhook(webhook, G_CHAT_WEBHOOK, client, event); if (target != null) { targets.add(target); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java index 840f0ac24eea..e780e378d5e4 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java @@ -42,7 +42,8 @@ public class GenericPublisher extends AbstractEventConsumer { @Override protected void doInit(JobExecutionContext context) { if (eventSubscription.getSubscriptionType() == GENERIC_WEBHOOK) { - this.webhook = JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); + this.webhook = + JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); // Build Client this.client = getClient(eventSubscription.getTimeout(), eventSubscription.getReadTimeout()); @@ -70,7 +71,8 @@ public void sendAlert(List changeEvents) throws EventPublisherExcep // Post to Generic Webhook with Actions for (ChangeEvent event : changeEvents) { String eventJson = JsonUtils.pojoToJson(event); - List targets = getTargetsForWebhook(webhook, GENERIC_WEBHOOK, client, event); + List targets = + getTargetsForWebhook(webhook, GENERIC_WEBHOOK, client, event); for (Invocation.Builder actionTarget : targets) { postWebhookMessage(this, actionTarget, eventJson); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java index 658b16e70342..175767943975 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java @@ -35,7 +35,8 @@ @Slf4j public class MSTeamsPublisher extends AbstractEventConsumer { - private final MessageDecorator teamsMessageFormatter = new MSTeamsMessageDecorator(); + private final MessageDecorator teamsMessageFormatter = + new MSTeamsMessageDecorator(); private Webhook webhook; private Invocation.Builder target; private Client client; @@ -43,7 +44,8 @@ public class MSTeamsPublisher extends AbstractEventConsumer { @Override protected void doInit(JobExecutionContext context) { if (eventSubscription.getSubscriptionType() == MS_TEAMS_WEBHOOK) { - this.webhook = JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); + this.webhook = + JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); // Build Client client = getClient(eventSubscription.getTimeout(), eventSubscription.getReadTimeout()); @@ -65,7 +67,8 @@ public void sendAlert(List changeEvents) { for (ChangeEvent event : changeEvents) { try { TeamsMessage teamsMessage = teamsMessageFormatter.buildMessage(event); - List targets = getTargetsForWebhook(webhook, MS_TEAMS_WEBHOOK, client, event); + List targets = + getTargetsForWebhook(webhook, MS_TEAMS_WEBHOOK, client, event); if (target != null) { targets.add(target); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java index 9811c051e62a..adf1d0fecd5d 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java @@ -42,7 +42,8 @@ public class SlackEventPublisher extends AbstractEventConsumer { @Override protected void doInit(JobExecutionContext context) { if (eventSubscription.getSubscriptionType() == SLACK_WEBHOOK) { - this.webhook = JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); + this.webhook = + JsonUtils.convertValue(eventSubscription.getSubscriptionConfig(), Webhook.class); // Build Client client = getClient(eventSubscription.getTimeout(), eventSubscription.getReadTimeout()); @@ -64,7 +65,8 @@ public void sendAlert(List changeEvents) { for (ChangeEvent event : changeEvents) { try { SlackMessage slackMessage = slackMessageFormatter.buildMessage(event); - List targets = getTargetsForWebhook(webhook, SLACK_WEBHOOK, client, event); + List targets = + getTargetsForWebhook(webhook, SLACK_WEBHOOK, client, event); if (target != null) { targets.add(target); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java index 55ec758877d2..18197ee6d211 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java @@ -46,7 +46,8 @@ public class EventSubscriptionScheduler { private static EventSubscriptionScheduler instance; private static volatile boolean initialized = false; private final Scheduler alertsScheduler = new StdSchedulerFactory().getScheduler(); - private static final ConcurrentHashMap alertJobMap = new ConcurrentHashMap<>(); + private static final ConcurrentHashMap alertJobMap = + new ConcurrentHashMap<>(); private EventSubscriptionScheduler() throws SchedulerException { this.alertsScheduler.start(); @@ -72,15 +73,22 @@ private static void initialize() throws SchedulerException { @Transaction @SneakyThrows public void addSubscriptionPublisher(EventSubscription eventSubscription) { - if (Objects.requireNonNull(eventSubscription.getAlertType()) == CreateEventSubscription.AlertType.CHANGE_EVENT) { + if (Objects.requireNonNull(eventSubscription.getAlertType()) + == CreateEventSubscription.AlertType.CHANGE_EVENT) { AbstractEventConsumer publisher = AlertUtil.getNotificationsPublisher(eventSubscription); if (Boolean.FALSE.equals( - eventSubscription.getEnabled())) { // Only add webhook that is enabled for publishing events - eventSubscription.setStatusDetails(getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.DISABLED)); + eventSubscription + .getEnabled())) { // Only add webhook that is enabled for publishing events + eventSubscription.setStatusDetails( + getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.DISABLED)); } else { - eventSubscription.setStatusDetails(getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.ACTIVE)); + eventSubscription.setStatusDetails( + getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.ACTIVE)); JobDetail jobDetail = - jobBuilder(publisher, eventSubscription, String.format("%s", eventSubscription.getId().toString())); + jobBuilder( + publisher, + eventSubscription, + String.format("%s", eventSubscription.getId().toString())); Trigger trigger = trigger(eventSubscription); // Schedule the Job @@ -101,7 +109,9 @@ private JobDetail jobBuilder( JobDataMap dataMap = new JobDataMap(); dataMap.put(ALERT_INFO_KEY, eventSubscription); JobBuilder jobBuilder = - JobBuilder.newJob(consumer.getClass()).withIdentity(jobIdentity, ALERT_JOB_GROUP).usingJobData(dataMap); + JobBuilder.newJob(consumer.getClass()) + .withIdentity(jobIdentity, ALERT_JOB_GROUP) + .usingJobData(dataMap); return jobBuilder.build(); } @@ -124,8 +134,10 @@ private AbstractEventConsumer getPublisher(UUID id) { @Transaction @SneakyThrows public void updateEventSubscription(EventSubscription eventSubscription) { - if (Objects.requireNonNull(eventSubscription.getAlertType()) == CreateEventSubscription.AlertType.CHANGE_EVENT) { - if (Boolean.TRUE.equals(eventSubscription.getEnabled())) { // Only add webhook that is enabled for publishing + if (Objects.requireNonNull(eventSubscription.getAlertType()) + == CreateEventSubscription.AlertType.CHANGE_EVENT) { + if (Boolean.TRUE.equals( + eventSubscription.getEnabled())) { // Only add webhook that is enabled for publishing // If there was a previous webhook either in disabled state or stopped due // to errors, update it and restart publishing AbstractEventConsumer previousPublisher = getPublisher(eventSubscription.getId()); @@ -142,7 +154,8 @@ public void updateEventSubscription(EventSubscription eventSubscription) { } else { // Remove the webhook publisher removeProcessorForEventSubscription( - eventSubscription.getId(), getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.DISABLED)); + eventSubscription.getId(), + getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.DISABLED)); } } else { throw new IllegalArgumentException(INVALID_ALERT); @@ -161,12 +174,15 @@ public void removeProcessorForEventSubscription(UUID id, SubscriptionStatus reas } @Transaction - public void deleteEventSubscriptionPublisher(EventSubscription deletedEntity) throws SchedulerException { - if (Objects.requireNonNull(deletedEntity.getAlertType()) == CreateEventSubscription.AlertType.CHANGE_EVENT) { + public void deleteEventSubscriptionPublisher(EventSubscription deletedEntity) + throws SchedulerException { + if (Objects.requireNonNull(deletedEntity.getAlertType()) + == CreateEventSubscription.AlertType.CHANGE_EVENT) { AbstractEventConsumer publisher = alertJobMap.remove(deletedEntity.getId()); if (publisher != null) { alertsScheduler.deleteJob(publisher.getJobDetail().getKey()); - alertsScheduler.unscheduleJob(new TriggerKey(deletedEntity.getId().toString(), ALERT_TRIGGER_GROUP)); + alertsScheduler.unscheduleJob( + new TriggerKey(deletedEntity.getId().toString(), ALERT_TRIGGER_GROUP)); LOG.info("Alert publisher deleted for {}", publisher.getEventSubscription().getName()); } } else { diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/ReportsHandler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/ReportsHandler.java index e69de29bb2d1..8b137891791f 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/ReportsHandler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/ReportsHandler.java @@ -0,0 +1 @@ + diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java index d1fe6de6fe53..75fb6047e44d 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java @@ -85,7 +85,11 @@ public GChatMessage buildMessage(ChangeEvent event) { gChatMessage.setText(headerText); GChatMessage.CardHeader cardHeader = new GChatMessage.CardHeader(); String cardHeaderText = - String.format(headerTemplate, event.getUserName(), event.getEntityType(), entityInterface.getName()); + String.format( + headerTemplate, + event.getUserName(), + event.getEntityType(), + entityInterface.getName()); cardHeader.setTitle(cardHeaderText); card.setHeader(cardHeader); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java index f241b52791ea..4e8c3d292f4f 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java @@ -72,7 +72,10 @@ public TeamsMessage buildMessage(ChangeEvent event) { if (event.getEntity() != null) { String headerTxt = "%s posted on " + event.getEntityType() + " %s"; String headerText = - String.format(headerTxt, event.getUserName(), this.buildEntityUrl(event.getEntityType(), entityInterface)); + String.format( + headerTxt, + event.getUserName(), + this.buildEntityUrl(event.getEntityType(), entityInterface)); teamsSections.setActivityTitle(headerText); } Map messages = diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MessageDecorator.java index bb373796da8e..149bb2f8aa69 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MessageDecorator.java @@ -40,7 +40,8 @@ public interface MessageDecorator { default String buildEntityUrl(String entityType, EntityInterface entityInterface) { String fqn = entityInterface.getFullyQualifiedName(); if (CommonUtil.nullOrEmpty(fqn)) { - EntityInterface result = Entity.getEntity(entityType, entityInterface.getId(), "id", Include.NON_DELETED); + EntityInterface result = + Entity.getEntity(entityType, entityInterface.getId(), "id", Include.NON_DELETED); fqn = result.getFullyQualifiedName(); } return getEntityUrl(entityType, fqn); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java index 0ae0281b5e20..381b43105c3b 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java @@ -89,7 +89,10 @@ public SlackMessage buildMessage(ChangeEvent event) { } else { headerTxt = "%s posted on " + eventType + " %s"; headerText = - String.format(headerTxt, event.getUserName(), this.buildEntityUrl(event.getEntityType(), entityInterface)); + String.format( + headerTxt, + event.getUserName(), + this.buildEntityUrl(event.getEntityType(), entityInterface)); } slackMessage.setText(headerText); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java index 055b6be15f9e..44388e82641f 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java @@ -1551,7 +1551,8 @@ default boolean supportsSoftDelete() { return false; } - @SqlQuery("SELECT json FROM event_subscription_extension where id = :id AND extension = :extension") + @SqlQuery( + "SELECT json FROM event_subscription_extension where id = :id AND extension = :extension") String getSubscriberOffset(@Bind("id") String id, @Bind("extension") String extension); @ConnectionAwareSqlUpdate( @@ -3181,7 +3182,7 @@ List listWithEntityFilter( + "ORDER BY eventTime ASC") List listWithoutEntityFilter( @Bind("eventType") String eventType, @Bind("timestamp") long timestamp); - + @SqlQuery("SELECT json FROM change_event ORDER BY eventTime ASC LIMIT :limit OFFSET :offset") List list(@Bind("limit") long limit, @Bind("offset") long offset); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EventSubscriptionRepository.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EventSubscriptionRepository.java index d8901d6a0e61..3909b3aceba3 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EventSubscriptionRepository.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EventSubscriptionRepository.java @@ -44,7 +44,8 @@ public void setFields(EventSubscription entity, Fields fields) { if (entity.getStatusDetails() == null) { entity.withStatusDetails( fields.contains("statusDetails") - ? EventSubscriptionScheduler.getInstance().getStatusForEventSubscription(entity.getId()) + ? EventSubscriptionScheduler.getInstance() + .getStatusForEventSubscription(entity.getId()) : null); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java b/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java index dce4fc00019f..ad1a8646fe48 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java @@ -112,7 +112,8 @@ public static class EventSubResourceDescriptorList public void initialize(OpenMetadataApplicationConfig config) { try { repository.initSeedDataFromResources(); - EventsSubscriptionRegistry.initialize(listOrEmpty(EventSubscriptionResource.getDescriptors())); + EventsSubscriptionRegistry.initialize( + listOrEmpty(EventSubscriptionResource.getDescriptors())); initializeEventSubscriptions(); } catch (Exception ex) { // Starting application should not fail @@ -127,7 +128,8 @@ private void initializeEventSubscriptions() { daoCollection .eventSubscriptionDAO() .listAllEventsSubscriptions(daoCollection.eventSubscriptionDAO().getTableName()); - List eventSubList = JsonUtils.readObjects(listAllEventsSubscriptions, EventSubscription.class); + List eventSubList = + JsonUtils.readObjects(listAllEventsSubscriptions, EventSubscription.class); for (EventSubscription subscription : eventSubList) { if (subscription.getSubscriptionType() != ACTIVITY_FEED) { EventSubscriptionScheduler.getInstance().addSubscriptionPublisher(subscription); @@ -292,10 +294,14 @@ public Response createEventSubscription( @ApiResponse(responseCode = "400", description = "Bad request") }) public Response createOrUpdateEventSubscription( - @Context UriInfo uriInfo, @Context SecurityContext securityContext, @Valid CreateEventSubscription create) { - EventSubscription eventSub = getEventSubscription(create, securityContext.getUserPrincipal().getName()); + @Context UriInfo uriInfo, + @Context SecurityContext securityContext, + @Valid CreateEventSubscription create) { + EventSubscription eventSub = + getEventSubscription(create, securityContext.getUserPrincipal().getName()); Response response = createOrUpdate(uriInfo, securityContext, eventSub); - EventSubscriptionScheduler.getInstance().updateEventSubscription((EventSubscription) response.getEntity()); + EventSubscriptionScheduler.getInstance() + .updateEventSubscription((EventSubscription) response.getEntity()); return response; } @@ -326,7 +332,8 @@ public Response patchEventSubscription( })) JsonPatch patch) { Response response = patchInternal(uriInfo, securityContext, id, patch); - EventSubscriptionScheduler.getInstance().updateEventSubscription((EventSubscription) response.getEntity()); + EventSubscriptionScheduler.getInstance() + .updateEventSubscription((EventSubscription) response.getEntity()); return response; } @@ -432,7 +439,8 @@ public Response deleteEventSubscriptionByName( @PathParam("name") String name) throws SchedulerException { - EventSubscription eventSubscription = repository.getByName(null, name, repository.getFields("id")); + EventSubscription eventSubscription = + repository.getByName(null, name, repository.getFields("id")); EventSubscriptionScheduler.getInstance().deleteEventSubscriptionPublisher(eventSubscription); return deleteByName(uriInfo, securityContext, name, true, true); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java index 28807853c5f3..24137240bcbd 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java @@ -190,19 +190,27 @@ public static Set buildReceiversListFromActions( } public static List getTargetsForWebhook( - SubscriptionAction action, CreateEventSubscription.SubscriptionType type, Client client, ChangeEvent event) { + SubscriptionAction action, + CreateEventSubscription.SubscriptionType type, + Client client, + ChangeEvent event) { EntityInterface entityInterface = getEntity(event); List targets = new ArrayList<>(); Set receiversUrls = buildReceiversListFromActions( - action, type, Entity.getCollectionDAO(), entityInterface.getId(), event.getEntityType()); + action, + type, + Entity.getCollectionDAO(), + entityInterface.getId(), + event.getEntityType()); for (String url : receiversUrls) { targets.add(client.target(url).request()); } return targets; } - public static void postWebhookMessage(AbstractEventConsumer publisher, Invocation.Builder target, Object message) + public static void postWebhookMessage( + AbstractEventConsumer publisher, Invocation.Builder target, Object message) throws InterruptedException { long attemptTime = System.currentTimeMillis(); Response response = @@ -219,7 +227,8 @@ public static void postWebhookMessage(AbstractEventConsumer publisher, Invocatio } else if (response.getStatus() >= 400 && response.getStatus() < 600) { // 4xx, 5xx response retry delivering events after timeout publisher.setNextBackOff(); - publisher.setAwaitingRetry(attemptTime, response.getStatus(), response.getStatusInfo().getReasonPhrase()); + publisher.setAwaitingRetry( + attemptTime, response.getStatus(), response.getStatusInfo().getReasonPhrase()); Thread.sleep(publisher.getCurrentBackoffTime()); } else if (response.getStatus() == 200) { publisher.setSuccessStatus(System.currentTimeMillis()); From 22cefe028fb2cc7799d36d9b1d1caf5101c62dd5 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Wed, 27 Dec 2023 18:26:08 +0530 Subject: [PATCH 03/28] Spotless formatting --- .../org/openmetadata/service/apps/scheduler/AppScheduler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/scheduler/AppScheduler.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/scheduler/AppScheduler.java index f8ad8e3a7e43..6a8134c82f9e 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/scheduler/AppScheduler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/scheduler/AppScheduler.java @@ -93,8 +93,8 @@ public void addApplicationSchedule(App application) { public void deleteScheduledApplication(App app) throws SchedulerException { JobDetail jobDetail = getJobKey(app.getId()); if (jobDetail != null) { - scheduler.deleteJob(jobDetail.getKey()); - scheduler.unscheduleJob(new TriggerKey(app.getId().toString(), APPS_TRIGGER_GROUP)); + scheduler.deleteJob(jobDetail.getKey()); + scheduler.unscheduleJob(new TriggerKey(app.getId().toString(), APPS_TRIGGER_GROUP)); appJobsKeyMap.remove(app.getId()); } } From edcb5adf0a448bc531d6e24d6d3e0f9d52ef43f9 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Sun, 31 Dec 2023 12:08:01 +0530 Subject: [PATCH 04/28] Review Comments --- .../sql/migrations/native/1.3.0/mysql/schemaChanges.sql | 4 ++-- .../sql/migrations/native/1.3.0/postgres/schemaChanges.sql | 4 ++-- .../apps/bundles/changeEvent/AbstractEventConsumer.java | 2 +- .../java/org/openmetadata/service/jdbi3/CollectionDAO.java | 7 +++---- 4 files changed, 8 insertions(+), 9 deletions(-) diff --git a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql index abbda7047eaa..f9edd580cc76 100644 --- a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql +++ b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql @@ -51,7 +51,7 @@ TRUNCATE automations_workflow; ALTER TABLE change_event ADD COLUMN offset INT AUTO_INCREMENT PRIMARY KEY; -- Add new table for event subscription extensions -CREATE TABLE IF NOT EXISTS event_subscription_extension ( +CREATE TABLE IF NOT EXISTS change_event_consumers ( id VARCHAR(36) NOT NULL, extension VARCHAR(256) NOT NULL, jsonSchema VARCHAR(256) NOT NULL, @@ -62,5 +62,5 @@ CREATE TABLE IF NOT EXISTS event_subscription_extension ( DELETE FROM event_subscription_entity ese where name = 'DataInsightReport'; -ALTER TABLE event_subscription_extension ADD COLUMN offset INT AUTO_INCREMENT; +ALTER TABLE change_event_consumers ADD COLUMN offset INT AUTO_INCREMENT; diff --git a/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql b/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql index 66ed938e61a7..0f1f14690dfb 100644 --- a/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql +++ b/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql @@ -67,7 +67,7 @@ TRUNCATE automations_workflow; ALTER TABLE change_event ADD COLUMN offset SERIAL PRIMARY KEY; -- Add new table for event subscription extensions -CREATE TABLE IF NOT EXISTS event_subscription_extension ( +CREATE TABLE IF NOT EXISTS change_event_consumers ( id VARCHAR(36) NOT NULL, extension VARCHAR(256) NOT NULL, jsonSchema VARCHAR(256) NOT NULL, @@ -78,4 +78,4 @@ CREATE TABLE IF NOT EXISTS event_subscription_extension ( DELETE FROM event_subscription_entity ese where name = 'DataInsightReport'; -ALTER TABLE event_subscription_extension ADD COLUMN offset SERIAL; +ALTER TABLE change_event_consumers ADD COLUMN offset SERIAL; diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index 1f0ac10b42b9..bf7d085f24ac 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -120,7 +120,7 @@ public boolean publishEvents(List events) throws InterruptedExcepti if (event.getChangeDescription() != null) { filteredEvents.add(event); } else { - LOG.info( + LOG.debug( "Email Publisher Event Will be Ignored Since Change Description is null. Received Event: {}", event); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java index ab9a06ab1ee5..1d1c37298dd1 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java @@ -1552,19 +1552,18 @@ default boolean supportsSoftDelete() { return false; } - @SqlQuery( - "SELECT json FROM event_subscription_extension where id = :id AND extension = :extension") + @SqlQuery("SELECT json FROM change_event_consumers where id = :id AND extension = :extension") String getSubscriberOffset(@Bind("id") String id, @Bind("extension") String extension); @ConnectionAwareSqlUpdate( value = - "INSERT INTO event_subscription_extension(id, extension, jsonSchema, json) " + "INSERT INTO change_event_consumers(id, extension, jsonSchema, json) " + "VALUES (:id, :extension, :jsonSchema, :json)" + "ON DUPLICATE KEY UPDATE json = :json, jsonSchema = :jsonSchema", connectionType = MYSQL) @ConnectionAwareSqlUpdate( value = - "INSERT INTO event_subscription_extension(id, extension, jsonSchema, json) " + "INSERT INTO change_event_consumers(id, extension, jsonSchema, json) " + "VALUES (:id, :extension, :jsonSchema, (:json :: jsonb)) " + "DO UPDATE SET json = EXCLUDED.json, jsonSchema = EXCLUDED.jsonSchema", connectionType = POSTGRES) From 0a279bbd33dacd703295f8eb66684f9800287e40 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Mon, 1 Jan 2024 15:23:51 +0530 Subject: [PATCH 05/28] Improvements --- .../native/1.3.0/mysql/schemaChanges.sql | 6 + .../native/1.3.0/postgres/schemaChanges.sql | 7 + .../changeEvent/AbstractEventConsumer.java | 189 ++++++++++-------- .../apps/bundles/changeEvent/Consumer.java | 9 +- .../changeEvent/email/EmailPublisher.java | 29 ++- .../changeEvent/gchat/GChatPublisher.java | 36 ++-- .../changeEvent/generic/GenericPublisher.java | 18 +- .../changeEvent/msteams/MSTeamsPublisher.java | 36 ++-- .../slack/SlackEventPublisher.java | 32 ++- .../service/events/EventPubSub.java | 26 +-- .../errors/EventPublisherException.java | 16 +- .../events/subscription/AlertUtil.java | 52 +++++ .../service/jdbi3/CollectionDAO.java | 4 +- .../service/util/SubscriptionUtil.java | 4 +- .../json/schema/events/alertMetrics.json | 27 +++ 15 files changed, 292 insertions(+), 199 deletions(-) create mode 100644 openmetadata-spec/src/main/resources/json/schema/events/alertMetrics.json diff --git a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql index f9edd580cc76..5d69acb421d1 100644 --- a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql +++ b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql @@ -64,3 +64,9 @@ DELETE FROM event_subscription_entity ese where name = 'DataInsightReport'; ALTER TABLE change_event_consumers ADD COLUMN offset INT AUTO_INCREMENT; +CREATE TABLE IF NOT EXISTS consumers_dlq ( + id VARCHAR(36) NOT NULL, + extension VARCHAR(256) NOT NULL, + json JSON NOT NULL, + timestamp BIGINT UNSIGNED GENERATED ALWAYS AS (json ->> '$.timestamp') NOT NULL +); diff --git a/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql b/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql index 0f1f14690dfb..839f8d61460d 100644 --- a/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql +++ b/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql @@ -79,3 +79,10 @@ CREATE TABLE IF NOT EXISTS change_event_consumers ( DELETE FROM event_subscription_entity ese where name = 'DataInsightReport'; ALTER TABLE change_event_consumers ADD COLUMN offset SERIAL; + +CREATE TABLE IF NOT EXISTS consumers_dlq ( + id VARCHAR(36) NOT NULL, + extension VARCHAR(256) NOT NULL, + json jsonb NOT NULL, + timestamp BIGINT GENERATED ALWAYS AS ((json ->> 'timestamp')::bigint) STORED NOT NULL +); \ No newline at end of file diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index bf7d085f24ac..3907fc1e34c3 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -16,18 +16,20 @@ import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.ACTIVE; import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.AWAITING_RETRY; import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.FAILED; +import static org.openmetadata.service.events.subscription.AlertUtil.getFilteredEvent; import java.util.ArrayList; import java.util.List; import lombok.Getter; import lombok.Setter; import lombok.extern.slf4j.Slf4j; +import org.openmetadata.schema.entity.events.AlertMetrics; import org.openmetadata.schema.entity.events.EventSubscription; import org.openmetadata.schema.entity.events.EventSubscriptionOffset; import org.openmetadata.schema.entity.events.SubscriptionStatus; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.service.Entity; -import org.openmetadata.service.events.errors.RetriableException; +import org.openmetadata.service.events.errors.EventPublisherException; import org.openmetadata.service.events.subscription.AlertUtil; import org.openmetadata.service.util.JsonUtils; import org.quartz.DisallowConcurrentExecution; @@ -35,13 +37,17 @@ import org.quartz.JobDetail; import org.quartz.JobExecutionContext; import org.quartz.JobExecutionException; +import org.quartz.PersistJobDataAfterExecution; @Slf4j @DisallowConcurrentExecution +@PersistJobDataAfterExecution public abstract class AbstractEventConsumer implements Consumer, Job { public static final String ALERT_OFFSET_KEY = "alertOffsetKey"; public static final String ALERT_INFO_KEY = "alertInfoKey"; private static final String OFFSET_EXTENSION = "eventSubscription.Offset"; + private static final String METRICS_EXTENSION = "eventSubscription.metrics"; + private static final String FAILED_EVENT_EXTENSION = "eventSubscription.failedEvent"; protected static final int BACKOFF_NORMAL = 0; protected static final int BACKOFF_3_SECONDS = 3 * 1000; protected static final int BACKOFF_30_SECONDS = 30 * 1000; @@ -51,6 +57,7 @@ public abstract class AbstractEventConsumer implements Consumer, Jo @Getter protected int currentBackoffTime = BACKOFF_NORMAL; private int offset = -1; + private AlertMetrics alertMetrics; @Getter @Setter private JobDetail jobDetail; protected EventSubscription eventSubscription; @@ -63,101 +70,132 @@ private void init(JobExecutionContext context) { this.jobDetail = context.getJobDetail(); this.eventSubscription = sub; this.offset = loadInitialOffset(); + this.alertMetrics = loadInitialMetrics(); this.doInit(context); } protected abstract void doInit(JobExecutionContext context); - protected void sendAlert(List list) { + protected void sendAlert(ChangeEvent event) throws EventPublisherException { /* This method needs to be over-ridden by specific Publisher for sending Alert */ } @Override - public void handleFailedEvents(List failedEvents) {} - - @Override - public void handleException(Exception e) {} + public void handleFailedEvent(EventPublisherException ex) { + LOG.debug( + "Failed in for Publisher : {} , Change Event : {} ", + eventSubscription.getName(), + ex.getChangeEvent()); + // Failed Event to be stored in Database + Entity.getCollectionDAO() + .eventSubscriptionDAO() + .upsertSubscriberExtension( + eventSubscription.getId().toString(), + FAILED_EVENT_EXTENSION, + "failedChangeEvent", + JsonUtils.pojoToJson(ex.getChangeEvent())); + } private int loadInitialOffset() { - int eventSubscriptionOffset; - String json = - Entity.getCollectionDAO() - .eventSubscriptionDAO() - .getSubscriberOffset(eventSubscription.getId().toString(), OFFSET_EXTENSION); - if (json != null) { - EventSubscriptionOffset offsetFromDb = - JsonUtils.readValue(json, EventSubscriptionOffset.class); - eventSubscriptionOffset = offsetFromDb.getOffset(); + EventSubscriptionOffset jobStoredOffset = + (EventSubscriptionOffset) jobDetail.getJobDataMap().get(ALERT_OFFSET_KEY); + // If the Job Data Map has the latest offset, use it + if (jobStoredOffset != null) { + return jobStoredOffset.getOffset(); } else { - eventSubscriptionOffset = Entity.getCollectionDAO().changeEventDAO().listCount(); + int eventSubscriptionOffset; + String json = + Entity.getCollectionDAO() + .eventSubscriptionDAO() + .getSubscriberExtension(eventSubscription.getId().toString(), OFFSET_EXTENSION); + if (json != null) { + EventSubscriptionOffset offsetFromDb = + JsonUtils.readValue(json, EventSubscriptionOffset.class); + eventSubscriptionOffset = offsetFromDb.getOffset(); + } else { + eventSubscriptionOffset = Entity.getCollectionDAO().changeEventDAO().listCount(); + } + // Update the Job Data Map with the latest offset + return eventSubscriptionOffset; + } + } + + private AlertMetrics loadInitialMetrics() { + AlertMetrics metrics = (AlertMetrics) jobDetail.getJobDataMap().get(METRICS_EXTENSION); + if (metrics != null) { + return metrics; + } else { + String json = + Entity.getCollectionDAO() + .eventSubscriptionDAO() + .getSubscriberExtension(eventSubscription.getId().toString(), METRICS_EXTENSION); + if (json != null) { + return JsonUtils.readValue(json, AlertMetrics.class); + } + // Update the Job Data Map with the latest offset + return new AlertMetrics().withTotalEvents(0).withFailedEvents(0).withSuccessEvents(0); } - // Update the Job Data Map with the latest offset - return eventSubscriptionOffset; } @Override - public boolean publishEvents(List events) throws InterruptedException { - // Publish to the given Alert Actions - // Evaluate Alert Trigger Config + public void publishEvents(List events) { + // If no events return + if (events.isEmpty()) { + return; + } // Filter the Change Events based on Alert Trigger Config - List filteredEvents = new ArrayList<>(); - for (ChangeEvent event : events) { - boolean triggerChangeEvent = - AlertUtil.shouldTriggerAlert( - event.getEntityType(), eventSubscription.getFilteringRules()); - - // Evaluate ChangeEvent Alert Filtering - if (eventSubscription.getFilteringRules() != null - && !AlertUtil.evaluateAlertConditions( - event, eventSubscription.getFilteringRules().getRules())) { - triggerChangeEvent = false; - } + List filteredEvents = + getFilteredEvent(events, eventSubscription.getFilteringRules()); - if (triggerChangeEvent) { - // Ignore the event since change description is null - if (event.getChangeDescription() != null) { - filteredEvents.add(event); - } else { - LOG.debug( - "Email Publisher Event Will be Ignored Since Change Description is null. Received Event: {}", - event); - } + for (ChangeEvent event : filteredEvents) { + try { + sendAlert(event); + } catch (EventPublisherException e) { + handleFailedEvent(e); } } - - try { - sendAlert(filteredEvents); - return true; - } catch (RetriableException ex) { - setNextBackOff(); - LOG.error( - "Failed to publish event in batch {} due to {}, will try again in {} ms", - filteredEvents, - ex, - currentBackoffTime); - Thread.sleep(currentBackoffTime); - } catch (Exception e) { - LOG.error("[AbstractAlertPublisher] error {}", e.getMessage(), e); - } - return false; } @Override - public void commitOffset(JobExecutionContext jobExecutionContext, int offset) { + public void commit(JobExecutionContext jobExecutionContext) { + long currentTime = System.currentTimeMillis(); EventSubscriptionOffset eventSubscriptionOffset = - new EventSubscriptionOffset().withOffset(offset).withTimestamp(System.currentTimeMillis()); + new EventSubscriptionOffset().withOffset(offset).withTimestamp(currentTime); + + // Upsert Offset to Database Entity.getCollectionDAO() .eventSubscriptionDAO() - .upsertSubscriberOffset( + .upsertSubscriberExtension( eventSubscription.getId().toString(), OFFSET_EXTENSION, "eventSubscriptionOffset", JsonUtils.pojoToJson(eventSubscriptionOffset)); // Update the Job Data Map with the latest offset - jobExecutionContext.getJobDetail().getJobDataMap().put(ALERT_OFFSET_KEY, offset); + jobExecutionContext + .getJobDetail() + .getJobDataMap() + .put(ALERT_OFFSET_KEY, eventSubscriptionOffset); + + // Upsert Metrics to Database + AlertMetrics metrics = + new AlertMetrics() + .withTotalEvents(alertMetrics.getTotalEvents()) + .withFailedEvents(alertMetrics.getFailedEvents()) + .withSuccessEvents(alertMetrics.getSuccessEvents()) + .withTimestamp(currentTime); + Entity.getCollectionDAO() + .eventSubscriptionDAO() + .upsertSubscriberExtension( + eventSubscription.getId().toString(), + METRICS_EXTENSION, + "alertMetrics", + JsonUtils.pojoToJson(metrics)); + + // Update the Job Data Map with latest Metrics + jobExecutionContext.getJobDetail().getJobDataMap().put(METRICS_EXTENSION, alertMetrics); } public synchronized void setErrorStatus(Long attemptTime, Integer statusCode, String reason) { @@ -205,26 +243,13 @@ public void execute(JobExecutionContext jobExecutionContext) throws JobExecution // Must Have , Before Execute the Init, Quartz Requires a Non-Arg Constructor this.init(jobExecutionContext); - try { - List batch = pollEvents(offset, 100); - if (!batch.isEmpty()) { - boolean success = publishEvents(batch); - if (success) { - offset += batch.size(); - } else { - handleFailedEvents(batch); - } - } - } catch (InterruptedException e) { - LOG.error("Interrupted while polling events", e); - Thread.currentThread().interrupt(); - } catch (Exception e) { - handleException(e); - } finally { - LOG.debug( - "Committing offset for eventSubscription {} {}", eventSubscription.getName(), offset); - commitOffset(jobExecutionContext, offset); - } + // Poll Events from Change Event Table + List batch = pollEvents(offset, 100); + publishEvents(batch); + + // Commit the Offset + offset += batch.size(); + commit(jobExecutionContext); } public void setNextBackOff() { diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Consumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Consumer.java index 4173c961c04e..d485814fcce2 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Consumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Consumer.java @@ -15,16 +15,15 @@ import io.dropwizard.lifecycle.Managed; import java.util.List; +import org.openmetadata.service.events.errors.EventPublisherException; import org.quartz.JobExecutionContext; public interface Consumer extends Managed { List pollEvents(long offset, long batchSize); - boolean publishEvents(List events) throws InterruptedException; + void publishEvents(List events); - void handleFailedEvents(List failedEvents); + void handleFailedEvent(EventPublisherException e); - void handleException(Exception e); - - void commitOffset(JobExecutionContext jobExecutionContext, int offset); + void commit(JobExecutionContext jobExecutionContext); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailPublisher.java index 864ef0635ddc..82b3df8ce8ef 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/email/EmailPublisher.java @@ -18,7 +18,6 @@ import static org.openmetadata.service.util.SubscriptionUtil.buildReceiversListFromActions; import java.util.HashSet; -import java.util.List; import java.util.Set; import lombok.extern.slf4j.Slf4j; import org.openmetadata.schema.EntityInterface; @@ -53,22 +52,20 @@ protected void doInit(JobExecutionContext context) { } @Override - public void sendAlert(List changeEvents) { - for (ChangeEvent event : changeEvents) { - try { - Set receivers = buildReceiversList(event); - EmailMessage emailMessage = emailDecorator.buildMessage(event); - for (String email : receivers) { - EmailUtil.sendChangeEventMail(email, emailMessage); - } - setSuccessStatus(System.currentTimeMillis()); - } catch (Exception e) { - setErrorStatus(System.currentTimeMillis(), 500, e.getMessage()); - String message = - CatalogExceptionMessage.eventPublisherFailedToPublish(EMAIL, event, e.getMessage()); - LOG.error(message); - throw new EventPublisherException(message); + public void sendAlert(ChangeEvent event) throws EventPublisherException { + try { + Set receivers = buildReceiversList(event); + EmailMessage emailMessage = emailDecorator.buildMessage(event); + for (String email : receivers) { + EmailUtil.sendChangeEventMail(email, emailMessage); } + setSuccessStatus(System.currentTimeMillis()); + } catch (Exception e) { + setErrorStatus(System.currentTimeMillis(), 500, e.getMessage()); + String message = + CatalogExceptionMessage.eventPublisherFailedToPublish(EMAIL, event, e.getMessage()); + LOG.error(message); + throw new EventPublisherException(message, event); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java index 9c4c595bb9fe..feb7086e4baf 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java @@ -63,26 +63,24 @@ protected void doInit(JobExecutionContext context) { } @Override - public void sendAlert(List changeEvents) { - for (ChangeEvent event : changeEvents) { - try { - GChatMessage gchatMessage = gChatMessageMessageDecorator.buildMessage(event); - List targets = - getTargetsForWebhook(webhook, G_CHAT_WEBHOOK, client, event); - if (target != null) { - targets.add(target); - } - for (Invocation.Builder actionTarget : targets) { - postWebhookMessage(this, actionTarget, gchatMessage); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - String message = - CatalogExceptionMessage.eventPublisherFailedToPublish( - G_CHAT_WEBHOOK, event, e.getMessage()); - LOG.error(message); - throw new EventPublisherException(message); + public void sendAlert(ChangeEvent event) throws EventPublisherException { + try { + GChatMessage gchatMessage = gChatMessageMessageDecorator.buildMessage(event); + List targets = + getTargetsForWebhook(webhook, G_CHAT_WEBHOOK, client, event); + if (target != null) { + targets.add(target); + } + for (Invocation.Builder actionTarget : targets) { + postWebhookMessage(this, actionTarget, gchatMessage); } + } catch (Exception e) { + Thread.currentThread().interrupt(); + String message = + CatalogExceptionMessage.eventPublisherFailedToPublish( + G_CHAT_WEBHOOK, event, e.getMessage()); + LOG.error(message); + throw new EventPublisherException(message, event); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java index e780e378d5e4..973d270f66ca 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java @@ -28,7 +28,6 @@ import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.schema.type.Webhook; import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; -import org.openmetadata.service.events.errors.EventPublisherException; import org.openmetadata.service.security.SecurityUtil; import org.openmetadata.service.util.JsonUtils; import org.openmetadata.service.util.RestUtil; @@ -54,11 +53,11 @@ protected void doInit(JobExecutionContext context) { } @Override - public void sendAlert(List changeEvents) throws EventPublisherException { + public void sendAlert(ChangeEvent event) { long attemptTime = System.currentTimeMillis(); try { // Post Message to default - String json = JsonUtils.pojoToJson(changeEvents); + String json = JsonUtils.pojoToJson(event); if (webhook.getEndpoint() != null) { if (webhook.getSecretKey() != null && !webhook.getSecretKey().isEmpty()) { String hmac = "sha256=" + CommonUtil.calculateHMAC(webhook.getSecretKey(), json); @@ -69,13 +68,11 @@ public void sendAlert(List changeEvents) throws EventPublisherExcep } // Post to Generic Webhook with Actions - for (ChangeEvent event : changeEvents) { - String eventJson = JsonUtils.pojoToJson(event); - List targets = - getTargetsForWebhook(webhook, GENERIC_WEBHOOK, client, event); - for (Invocation.Builder actionTarget : targets) { - postWebhookMessage(this, actionTarget, eventJson); - } + String eventJson = JsonUtils.pojoToJson(event); + List targets = + getTargetsForWebhook(webhook, GENERIC_WEBHOOK, client, event); + for (Invocation.Builder actionTarget : targets) { + postWebhookMessage(this, actionTarget, eventJson); } } catch (Exception ex) { Throwable cause = ex.getCause(); @@ -83,6 +80,7 @@ public void sendAlert(List changeEvents) throws EventPublisherExcep LOG.warn( "Invalid webhook {} endpoint {}", eventSubscription.getName(), webhook.getEndpoint()); setErrorStatus(attemptTime, 400, "UnknownHostException"); + } else { LOG.debug("Exception occurred while publishing webhook", ex); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java index c7886f45f9e7..ed813fa7c7cb 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java @@ -63,26 +63,24 @@ protected void doInit(JobExecutionContext context) { } @Override - public void sendAlert(List changeEvents) { - for (ChangeEvent event : changeEvents) { - try { - TeamsMessage teamsMessage = teamsMessageFormatter.buildMessage(event); - List targets = - getTargetsForWebhook(webhook, MS_TEAMS_WEBHOOK, client, event); - if (target != null) { - targets.add(target); - } - for (Invocation.Builder actionTarget : targets) { - postWebhookMessage(this, actionTarget, teamsMessage); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - String message = - CatalogExceptionMessage.eventPublisherFailedToPublish( - MS_TEAMS_WEBHOOK, event, e.getMessage()); - LOG.error(message); - throw new EventPublisherException(message); + public void sendAlert(ChangeEvent event) throws EventPublisherException { + try { + TeamsMessage teamsMessage = teamsMessageFormatter.buildMessage(event); + List targets = + getTargetsForWebhook(webhook, MS_TEAMS_WEBHOOK, client, event); + if (target != null) { + targets.add(target); + } + for (Invocation.Builder actionTarget : targets) { + postWebhookMessage(this, actionTarget, teamsMessage); } + } catch (Exception e) { + Thread.currentThread().interrupt(); + String message = + CatalogExceptionMessage.eventPublisherFailedToPublish( + MS_TEAMS_WEBHOOK, event, e.getMessage()); + LOG.error(message); + throw new EventPublisherException(message, event); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java index 7290f8eaa6af..ba9c502c9433 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java @@ -61,24 +61,22 @@ protected void doInit(JobExecutionContext context) { } @Override - public void sendAlert(List changeEvents) { - for (ChangeEvent event : changeEvents) { - try { - SlackMessage slackMessage = slackMessageFormatter.buildMessage(event); - List targets = - getTargetsForWebhook(webhook, SLACK_WEBHOOK, client, event); - if (target != null) { - targets.add(target); - } - for (Invocation.Builder actionTarget : targets) { - postWebhookMessage(this, actionTarget, slackMessage); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOG.error("Failed to publish event {} to slack due to {} ", event, e.getMessage()); - throw new EventPublisherException( - String.format("Failed to publish event %s to slack due to %s ", event, e.getMessage())); + public void sendAlert(ChangeEvent event) throws EventPublisherException { + try { + SlackMessage slackMessage = slackMessageFormatter.buildMessage(event); + List targets = + getTargetsForWebhook(webhook, SLACK_WEBHOOK, client, event); + if (target != null) { + targets.add(target); + } + for (Invocation.Builder actionTarget : targets) { + postWebhookMessage(this, actionTarget, slackMessage); } + } catch (Exception e) { + LOG.error("Failed to publish event {} to slack due to {} ", event, e.getMessage()); + throw new EventPublisherException( + String.format("Failed to publish event %s to slack due to %s ", event, e.getMessage()), + event); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/EventPubSub.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/EventPubSub.java index 3864ffa01505..e914590db63f 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/EventPubSub.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/EventPubSub.java @@ -16,7 +16,6 @@ import com.lmax.disruptor.BatchEventProcessor; import com.lmax.disruptor.EventFactory; import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.ExceptionHandler; import com.lmax.disruptor.RingBuffer; import com.lmax.disruptor.dsl.Disruptor; import com.lmax.disruptor.util.DaemonThreadFactory; @@ -28,7 +27,6 @@ import lombok.extern.slf4j.Slf4j; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.service.events.EventPubSub.ChangeEventHolder; -import org.openmetadata.service.events.errors.EventPublisherException; /** Change event PubSub built based on LMAX Disruptor. */ @Slf4j @@ -41,7 +39,7 @@ public class EventPubSub { public static void start() { if (!started) { disruptor = new Disruptor<>(ChangeEventHolder::new, 1024, DaemonThreadFactory.INSTANCE); - disruptor.setDefaultExceptionHandler(new DefaultExceptionHandler()); + // disruptor.setDefaultExceptionHandler(new DefaultExceptionHandler()); executor = Executors.newCachedThreadPool(DaemonThreadFactory.INSTANCE); ringBuffer = disruptor.start(); LOG.info("Disruptor started"); @@ -85,7 +83,7 @@ public static BatchEventProcessor addEventHandler( EventHandler eventHandler) { BatchEventProcessor processor = new BatchEventProcessor<>(ringBuffer, ringBuffer.newBarrier(), eventHandler); - processor.setExceptionHandler(new DefaultExceptionHandler()); + // processor.setExceptionHandler(new DefaultExceptionHandler()); ringBuffer.addGatingSequences(processor.getSequence()); executor.execute(processor); LOG.info("Processor added for {}", processor); @@ -100,24 +98,4 @@ public static void removeProcessor(BatchEventProcessor proces public void close() { /* Nothing to clean up */ } - - public static class DefaultExceptionHandler implements ExceptionHandler { - @Override - public void handleEventException( - Throwable throwable, long l, ChangeEventHolder changeEventHolder) { - LOG.warn("Disruptor error in onEvent {}", throwable.getMessage()); - throw new EventPublisherException( - throwable.getMessage()); // Throw runtime exception to stop the event handler thread - } - - @Override - public void handleOnStartException(Throwable throwable) { - LOG.warn("Disruptor error in onStart {}", throwable.getMessage()); - } - - @Override - public void handleOnShutdownException(Throwable throwable) { - LOG.warn("Disruptor error on onShutdown {}", throwable.getMessage()); - } - } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/errors/EventPublisherException.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/errors/EventPublisherException.java index b3d04a41e33d..825b29a6bdac 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/errors/EventPublisherException.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/errors/EventPublisherException.java @@ -1,7 +1,14 @@ package org.openmetadata.service.events.errors; -public class EventPublisherException extends RuntimeException { - private static final long serialVersionUID = 1L; +import java.io.Serial; +import lombok.Getter; +import org.openmetadata.schema.type.ChangeEvent; + +@Getter +public class EventPublisherException extends Exception { + @Serial private static final long serialVersionUID = 1L; + + private ChangeEvent changeEvent; public EventPublisherException(String message, Throwable cause) { super(message, cause); @@ -11,6 +18,11 @@ public EventPublisherException(String message) { super(message); } + public EventPublisherException(String message, ChangeEvent event) { + super(message); + this.changeEvent = event; + } + public EventPublisherException(Throwable cause) { super(cause); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java index 2c23ecb2bd2e..3c8cdbf8b61a 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java @@ -18,11 +18,13 @@ import static org.openmetadata.service.Entity.USER; import static org.openmetadata.service.security.policyevaluator.CompiledRule.parseExpression; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.UnaryOperator; import java.util.stream.Stream; import javax.ws.rs.BadRequestException; import lombok.extern.slf4j.Slf4j; @@ -48,6 +50,7 @@ import org.openmetadata.service.exception.CatalogExceptionMessage; import org.openmetadata.service.resources.CollectionRegistry; import org.openmetadata.service.search.models.IndexMapping; +import org.quartz.JobDataMap; import org.springframework.expression.Expression; @Slf4j @@ -230,4 +233,53 @@ public static SubscriptionStatus buildSubscriptionStatus( .withNextAttempt(nextAttempt) .withTimestamp(timeStamp); } + + public static List getFilteredEvent( + List events, FilteringRules filteringRules) { + List filteredEvents = new ArrayList<>(); + for (ChangeEvent event : events) { + boolean triggerChangeEvent = + AlertUtil.shouldTriggerAlert(event.getEntityType(), filteringRules); + + // Evaluate ChangeEvent Alert Filtering + if (filteringRules != null + && !AlertUtil.evaluateAlertConditions(event, filteringRules.getRules())) { + triggerChangeEvent = false; + } + + if (triggerChangeEvent) { + // Ignore the event since change description is null + if (event.getChangeDescription() != null) { + filteredEvents.add(event); + } else { + LOG.debug( + "Email Publisher Event Will be Ignored Since Change Description is null. Received Event: {}", + event); + } + } + } + return filteredEvents; + } + + public static T getValueFromQtzJobMap(JobDataMap dataMap, String key, Class tClass) { + Object value = dataMap.get(key); + if (value == null) { + return null; + } + return tClass.cast(value); + } + + public static T getOrDefaultFromQtzJobMap( + JobDataMap dataMap, String key, Class tClass, UnaryOperator defaultSupplier) { + if (dataMap.containsKey(key)) { + return tClass.cast(dataMap.get(key)); + } else { + return tClass.cast(defaultSupplier.apply(key)); + } + } + + public static JobDataMap putValueInQtzJobMap(JobDataMap dataMap, String key, Object obj) { + dataMap.put(key, obj); + return dataMap; + } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java index 1d1c37298dd1..5d04e1fff30a 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java @@ -1553,7 +1553,7 @@ default boolean supportsSoftDelete() { } @SqlQuery("SELECT json FROM change_event_consumers where id = :id AND extension = :extension") - String getSubscriberOffset(@Bind("id") String id, @Bind("extension") String extension); + String getSubscriberExtension(@Bind("id") String id, @Bind("extension") String extension); @ConnectionAwareSqlUpdate( value = @@ -1567,7 +1567,7 @@ default boolean supportsSoftDelete() { + "VALUES (:id, :extension, :jsonSchema, (:json :: jsonb)) " + "DO UPDATE SET json = EXCLUDED.json, jsonSchema = EXCLUDED.jsonSchema", connectionType = POSTGRES) - void upsertSubscriberOffset( + void upsertSubscriberExtension( @Bind("id") String id, @Bind("extension") String extension, @Bind("jsonSchema") String jsonSchema, diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java index 43aac2022643..ec105214ae09 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java @@ -214,8 +214,7 @@ public static List getTargetsForWebhook( } public static void postWebhookMessage( - AbstractEventConsumer publisher, Invocation.Builder target, Object message) - throws InterruptedException { + AbstractEventConsumer publisher, Invocation.Builder target, Object message) { long attemptTime = System.currentTimeMillis(); Response response = target.post(javax.ws.rs.client.Entity.entity(message, MediaType.APPLICATION_JSON_TYPE)); @@ -233,7 +232,6 @@ public static void postWebhookMessage( publisher.setNextBackOff(); publisher.setAwaitingRetry( attemptTime, response.getStatus(), response.getStatusInfo().getReasonPhrase()); - Thread.sleep(publisher.getCurrentBackoffTime()); } else if (response.getStatus() == 200) { publisher.setSuccessStatus(System.currentTimeMillis()); } diff --git a/openmetadata-spec/src/main/resources/json/schema/events/alertMetrics.json b/openmetadata-spec/src/main/resources/json/schema/events/alertMetrics.json new file mode 100644 index 000000000000..b378e980cebd --- /dev/null +++ b/openmetadata-spec/src/main/resources/json/schema/events/alertMetrics.json @@ -0,0 +1,27 @@ +{ + "$id": "https://open-metadata.org/schema/events/alertMetrics.json", + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "AlertMetrics", + "description": "Event Subscription Offset", + "type": "object", + "javaType": "org.openmetadata.schema.entity.events.AlertMetrics", + "properties": { + "totalEvents": { + "description": "Total number of events.", + "type": "integer" + }, + "successEvents": { + "description": "Number of events that were successfully processed.", + "type": "integer" + }, + "failedEvents": { + "description": "Number of events that failed to be processed.", + "type": "integer" + }, + "timestamp": { + "description": "Update time of the job status.", + "$ref": "../type/basic.json#/definitions/timestamp" + } + }, + "additionalProperties": false +} From c028984422ea8c3a6fdfb503d4fc684d87746d01 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Mon, 1 Jan 2024 23:40:06 +0530 Subject: [PATCH 06/28] Improvements to add Retry and DLQ --- .../changeEvent/AbstractEventConsumer.java | 75 ++++++++++++++++--- .../changeEvent/msteams/MSTeamsPublisher.java | 1 - .../slack/SlackEventPublisher.java | 10 ++- .../events/subscription/AlertUtil.java | 15 ---- .../service/jdbi3/CollectionDAO.java | 15 ++++ .../json/schema/events/alertMetrics.json | 2 +- .../json/schema/events/failedEvent.json | 27 +++++++ 7 files changed, 115 insertions(+), 30 deletions(-) create mode 100644 openmetadata-spec/src/main/resources/json/schema/events/failedEvent.json diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index 3907fc1e34c3..29b194cbaf31 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -18,14 +18,19 @@ import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.FAILED; import static org.openmetadata.service.events.subscription.AlertUtil.getFilteredEvent; +import com.fasterxml.jackson.core.type.TypeReference; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.function.BiPredicate; import lombok.Getter; import lombok.Setter; import lombok.extern.slf4j.Slf4j; import org.openmetadata.schema.entity.events.AlertMetrics; import org.openmetadata.schema.entity.events.EventSubscription; import org.openmetadata.schema.entity.events.EventSubscriptionOffset; +import org.openmetadata.schema.entity.events.FailedEvent; import org.openmetadata.schema.entity.events.SubscriptionStatus; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.service.Entity; @@ -43,6 +48,8 @@ @DisallowConcurrentExecution @PersistJobDataAfterExecution public abstract class AbstractEventConsumer implements Consumer, Job { + public static final BiPredicate eventMatch = + (f1, f2) -> f1.getChangeEvent().equals(f2.getChangeEvent()); public static final String ALERT_OFFSET_KEY = "alertOffsetKey"; public static final String ALERT_INFO_KEY = "alertInfoKey"; private static final String OFFSET_EXTENSION = "eventSubscription.Offset"; @@ -87,14 +94,49 @@ public void handleFailedEvent(EventPublisherException ex) { "Failed in for Publisher : {} , Change Event : {} ", eventSubscription.getName(), ex.getChangeEvent()); - // Failed Event to be stored in Database - Entity.getCollectionDAO() - .eventSubscriptionDAO() - .upsertSubscriberExtension( - eventSubscription.getId().toString(), - FAILED_EVENT_EXTENSION, - "failedChangeEvent", - JsonUtils.pojoToJson(ex.getChangeEvent())); + + FailedEvent failedEvent = + new FailedEvent() + .withChangeEvent(ex.getChangeEvent()) + .withReason(ex.getMessage()) + .withRetriesLeft(eventSubscription.getRetries()) + .withTimestamp(System.currentTimeMillis()); + + // Check in Qtz Map + Set failedEventsList = + JsonUtils.convertValue( + jobDetail.getJobDataMap().get(FAILED_EVENT_EXTENSION), new TypeReference<>() {}); + if (failedEventsList == null) { + failedEventsList = new HashSet<>(); + } + + // Test If the Failing Event is present in the List + boolean removeChangeEvent = + failedEventsList.removeIf( + (failedEvent1) -> { + boolean matched = eventMatch.test(failedEvent1, failedEvent); + if (matched) { + failedEvent.withRetriesLeft(failedEvent1.getRetriesLeft() - 1); + } + return matched; + }); + + // Check if the event was removed + if (removeChangeEvent) { + if (failedEvent.getRetriesLeft() == 0) { + // If the Retries are exhausted, then remove the Event from the List to DLQ + Entity.getCollectionDAO() + .eventSubscriptionDAO() + .upsertFailedEvent( + eventSubscription.getId().toString(), + FAILED_EVENT_EXTENSION, + JsonUtils.pojoToJson(failedEvent.withRetriesLeft(0))); + } else { + failedEvent.withRetriesLeft(failedEvent.getRetriesLeft() - 1); + } + } + failedEventsList.add(failedEvent); + jobDetail.getJobDataMap().put(FAILED_EVENT_EXTENSION, failedEventsList); } private int loadInitialOffset() { @@ -245,10 +287,25 @@ public void execute(JobExecutionContext jobExecutionContext) throws JobExecution // Poll Events from Change Event Table List batch = pollEvents(offset, 100); + int batchSize = batch.size(); + + // Retry Failed Events + Set failedEventsList = + (HashSet) jobDetail.getJobDataMap().get(FAILED_EVENT_EXTENSION); + if (failedEventsList != null) { + List failedChangeEvents = + failedEventsList.stream() + .filter(failedEvent -> failedEvent.getRetriesLeft() > 0) + .map(FailedEvent::getChangeEvent) + .toList(); + batch.addAll(failedChangeEvents); + } + + // Publish Events publishEvents(batch); // Commit the Offset - offset += batch.size(); + offset += batchSize; commit(jobExecutionContext); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java index ed813fa7c7cb..a7f911576241 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/msteams/MSTeamsPublisher.java @@ -75,7 +75,6 @@ public void sendAlert(ChangeEvent event) throws EventPublisherException { postWebhookMessage(this, actionTarget, teamsMessage); } } catch (Exception e) { - Thread.currentThread().interrupt(); String message = CatalogExceptionMessage.eventPublisherFailedToPublish( MS_TEAMS_WEBHOOK, event, e.getMessage()); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java index ba9c502c9433..d467373927a3 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/slack/SlackEventPublisher.java @@ -27,6 +27,7 @@ import org.openmetadata.schema.type.Webhook; import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; import org.openmetadata.service.events.errors.EventPublisherException; +import org.openmetadata.service.exception.CatalogExceptionMessage; import org.openmetadata.service.formatter.decorators.MessageDecorator; import org.openmetadata.service.formatter.decorators.SlackMessageDecorator; import org.openmetadata.service.util.JsonUtils; @@ -73,10 +74,11 @@ public void sendAlert(ChangeEvent event) throws EventPublisherException { postWebhookMessage(this, actionTarget, slackMessage); } } catch (Exception e) { - LOG.error("Failed to publish event {} to slack due to {} ", event, e.getMessage()); - throw new EventPublisherException( - String.format("Failed to publish event %s to slack due to %s ", event, e.getMessage()), - event); + String message = + CatalogExceptionMessage.eventPublisherFailedToPublish( + SLACK_WEBHOOK, event, e.getMessage()); + LOG.error(message); + throw new EventPublisherException(message, event); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java index 3c8cdbf8b61a..c25ec40135a0 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java @@ -24,7 +24,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.function.UnaryOperator; import java.util.stream.Stream; import javax.ws.rs.BadRequestException; import lombok.extern.slf4j.Slf4j; @@ -268,18 +267,4 @@ public static T getValueFromQtzJobMap(JobDataMap dataMap, String key, Class< } return tClass.cast(value); } - - public static T getOrDefaultFromQtzJobMap( - JobDataMap dataMap, String key, Class tClass, UnaryOperator defaultSupplier) { - if (dataMap.containsKey(key)) { - return tClass.cast(dataMap.get(key)); - } else { - return tClass.cast(defaultSupplier.apply(key)); - } - } - - public static JobDataMap putValueInQtzJobMap(JobDataMap dataMap, String key, Object obj) { - dataMap.put(key, obj); - return dataMap; - } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java index 5d04e1fff30a..dcb84819f4c2 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java @@ -1572,6 +1572,21 @@ void upsertSubscriberExtension( @Bind("extension") String extension, @Bind("jsonSchema") String jsonSchema, @Bind("json") String json); + + @ConnectionAwareSqlUpdate( + value = + "INSERT INTO consumers_dlq(id, extension, json) " + + "VALUES (:id, :extension, :json)" + + "ON DUPLICATE KEY UPDATE json = :json", + connectionType = MYSQL) + @ConnectionAwareSqlUpdate( + value = + "INSERT INTO consumers_dlq(id, extension, json) " + + "VALUES (:id, :extension, (:json :: jsonb)) " + + "DO UPDATE SET json = EXCLUDED.json", + connectionType = POSTGRES) + void upsertFailedEvent( + @Bind("id") String id, @Bind("extension") String extension, @Bind("json") String json); } interface ChartDAO extends EntityDAO { diff --git a/openmetadata-spec/src/main/resources/json/schema/events/alertMetrics.json b/openmetadata-spec/src/main/resources/json/schema/events/alertMetrics.json index b378e980cebd..6b5677b8ce26 100644 --- a/openmetadata-spec/src/main/resources/json/schema/events/alertMetrics.json +++ b/openmetadata-spec/src/main/resources/json/schema/events/alertMetrics.json @@ -2,7 +2,7 @@ "$id": "https://open-metadata.org/schema/events/alertMetrics.json", "$schema": "http://json-schema.org/draft-07/schema#", "title": "AlertMetrics", - "description": "Event Subscription Offset", + "description": "Alert Metrics Schema", "type": "object", "javaType": "org.openmetadata.schema.entity.events.AlertMetrics", "properties": { diff --git a/openmetadata-spec/src/main/resources/json/schema/events/failedEvent.json b/openmetadata-spec/src/main/resources/json/schema/events/failedEvent.json new file mode 100644 index 000000000000..ad18d4dab5ec --- /dev/null +++ b/openmetadata-spec/src/main/resources/json/schema/events/failedEvent.json @@ -0,0 +1,27 @@ +{ + "$id": "https://open-metadata.org/schema/events/failedEvents.json", + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "FailedEvents", + "description": "Failed Events Schema", + "type": "object", + "javaType": "org.openmetadata.schema.entity.events.FailedEvent", + "properties": { + "changeEvent": { + "description": "Change Event that failed", + "$ref": "../type/changeEvent.json" + }, + "reason": { + "description": "Reason for failure", + "type": "string" + }, + "retriesLeft": { + "description": "Retries Left for the event", + "type": "integer" + }, + "timestamp": { + "description": "Time of Failure", + "$ref": "../type/basic.json#/definitions/timestamp" + } + }, + "additionalProperties": false +} From 76b9076ed659c5e2375687282ac5cbb32ce7b1a9 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Mon, 1 Jan 2024 23:41:35 +0530 Subject: [PATCH 07/28] Spotless fix --- .../apps/bundles/changeEvent/AbstractEventConsumer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index 29b194cbaf31..f6cacf2894b8 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -291,7 +291,8 @@ public void execute(JobExecutionContext jobExecutionContext) throws JobExecution // Retry Failed Events Set failedEventsList = - (HashSet) jobDetail.getJobDataMap().get(FAILED_EVENT_EXTENSION); + JsonUtils.convertValue( + jobDetail.getJobDataMap().get(FAILED_EVENT_EXTENSION), new TypeReference<>() {}); if (failedEventsList != null) { List failedChangeEvents = failedEventsList.stream() From 74d11ea4b6d7fffe4896848a60fe399cc854a2f9 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Tue, 2 Jan 2024 12:13:35 +0530 Subject: [PATCH 08/28] Update Stats --- .../apps/bundles/changeEvent/AbstractEventConsumer.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index f6cacf2894b8..470da398664e 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -194,7 +194,9 @@ public void publishEvents(List events) { for (ChangeEvent event : filteredEvents) { try { sendAlert(event); + alertMetrics.withSuccessEvents(alertMetrics.getSuccessEvents() + 1); } catch (EventPublisherException e) { + alertMetrics.withFailedEvents(alertMetrics.getFailedEvents() + 1); handleFailedEvent(e); } } @@ -303,6 +305,7 @@ public void execute(JobExecutionContext jobExecutionContext) throws JobExecution } // Publish Events + alertMetrics.withTotalEvents(batch.size()); publishEvents(batch); // Commit the Offset From a44cfbda911f04dd0fc2f63ed5f798b423c5a763 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Tue, 2 Jan 2024 12:30:32 +0530 Subject: [PATCH 09/28] Fix stats --- .../service/apps/bundles/changeEvent/AbstractEventConsumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index 470da398664e..42e34e4dca6b 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -305,7 +305,7 @@ public void execute(JobExecutionContext jobExecutionContext) throws JobExecution } // Publish Events - alertMetrics.withTotalEvents(batch.size()); + alertMetrics.withTotalEvents(alertMetrics.getTotalEvents() + batch.size()); publishEvents(batch); // Commit the Offset From e7ae4c2603db89a0db6bb06df9aacf8c663a763c Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Tue, 2 Jan 2024 16:48:13 +0530 Subject: [PATCH 10/28] Fix delete Failures --- .../service/events/scheduled/EventSubscriptionScheduler.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java index 18197ee6d211..a1deb4fe6c0f 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java @@ -30,6 +30,7 @@ import org.quartz.JobBuilder; import org.quartz.JobDataMap; import org.quartz.JobDetail; +import org.quartz.JobKey; import org.quartz.Scheduler; import org.quartz.SchedulerException; import org.quartz.SimpleScheduleBuilder; @@ -180,10 +181,10 @@ public void deleteEventSubscriptionPublisher(EventSubscription deletedEntity) == CreateEventSubscription.AlertType.CHANGE_EVENT) { AbstractEventConsumer publisher = alertJobMap.remove(deletedEntity.getId()); if (publisher != null) { - alertsScheduler.deleteJob(publisher.getJobDetail().getKey()); + alertsScheduler.deleteJob(new JobKey(deletedEntity.getId().toString(), ALERT_JOB_GROUP)); alertsScheduler.unscheduleJob( new TriggerKey(deletedEntity.getId().toString(), ALERT_TRIGGER_GROUP)); - LOG.info("Alert publisher deleted for {}", publisher.getEventSubscription().getName()); + LOG.info("Alert publisher deleted for {}", deletedEntity.getName()); } } else { throw new IllegalArgumentException(INVALID_ALERT); From baf3281c5cde43a4e10ae37a073276d7ca10bd42 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Wed, 3 Jan 2024 00:59:40 +0530 Subject: [PATCH 11/28] Fix Failing Tests --- .../native/1.3.0/mysql/schemaChanges.sql | 3 +- .../native/1.3.0/postgres/schemaChanges.sql | 3 +- .../changeEvent/AbstractEventConsumer.java | 109 ++++++++---------- .../changeEvent/gchat/GChatPublisher.java | 1 - .../service/events/ChangeEventHandler.java | 1 + .../scheduled/EventSubscriptionScheduler.java | 108 +++++++++-------- .../events/subscription/AlertUtil.java | 9 +- .../service/formatter/util/FormatterUtil.java | 2 + .../service/jdbi3/EntityRepository.java | 3 + .../jdbi3/IngestionPipelineRepository.java | 1 + .../service/jdbi3/KpiRepository.java | 2 + .../service/jdbi3/QueryRepository.java | 1 + .../service/jdbi3/TestCaseRepository.java | 1 + .../service/jdbi3/UsageRepository.java | 1 + .../EventSubscriptionResource.java | 7 +- .../service/util/SubscriptionUtil.java | 1 - .../CloudWatchEventMonitorTest.java | 1 + .../events/EventSubscriptionResourceTest.java | 10 +- .../events/WebhookCallbackResource.java | 42 +++---- .../events/api/createEventSubscription.json | 10 ++ .../json/schema/events/eventSubscription.json | 5 + .../json/schema/type/changeEvent.json | 6 +- 22 files changed, 176 insertions(+), 151 deletions(-) diff --git a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql index 5d69acb421d1..d268815ef40e 100644 --- a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql +++ b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql @@ -68,5 +68,6 @@ CREATE TABLE IF NOT EXISTS consumers_dlq ( id VARCHAR(36) NOT NULL, extension VARCHAR(256) NOT NULL, json JSON NOT NULL, - timestamp BIGINT UNSIGNED GENERATED ALWAYS AS (json ->> '$.timestamp') NOT NULL + timestamp BIGINT UNSIGNED GENERATED ALWAYS AS (json ->> '$.timestamp') NOT NULL, + UNIQUE(id, extension) ); diff --git a/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql b/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql index 839f8d61460d..b559e9bfdf75 100644 --- a/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql +++ b/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql @@ -84,5 +84,6 @@ CREATE TABLE IF NOT EXISTS consumers_dlq ( id VARCHAR(36) NOT NULL, extension VARCHAR(256) NOT NULL, json jsonb NOT NULL, - timestamp BIGINT GENERATED ALWAYS AS ((json ->> 'timestamp')::bigint) STORED NOT NULL + timestamp BIGINT GENERATED ALWAYS AS ((json ->> 'timestamp')::bigint) STORED NOT NULL, + UNIQUE(id, extension) ); \ No newline at end of file diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index 42e34e4dca6b..37aec3b49f54 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -23,7 +23,6 @@ import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.function.BiPredicate; import lombok.Getter; import lombok.Setter; import lombok.extern.slf4j.Slf4j; @@ -48,21 +47,11 @@ @DisallowConcurrentExecution @PersistJobDataAfterExecution public abstract class AbstractEventConsumer implements Consumer, Job { - public static final BiPredicate eventMatch = - (f1, f2) -> f1.getChangeEvent().equals(f2.getChangeEvent()); public static final String ALERT_OFFSET_KEY = "alertOffsetKey"; public static final String ALERT_INFO_KEY = "alertInfoKey"; private static final String OFFSET_EXTENSION = "eventSubscription.Offset"; private static final String METRICS_EXTENSION = "eventSubscription.metrics"; - private static final String FAILED_EVENT_EXTENSION = "eventSubscription.failedEvent"; - protected static final int BACKOFF_NORMAL = 0; - protected static final int BACKOFF_3_SECONDS = 3 * 1000; - protected static final int BACKOFF_30_SECONDS = 30 * 1000; - protected static final int BACKOFF_5_MINUTES = 5 * 60 * 1000; - protected static final int BACKOFF_1_HOUR = 60 * 60 * 1000; - protected static final int BACKOFF_24_HOUR = 24 * 60 * 60 * 1000; - - @Getter protected int currentBackoffTime = BACKOFF_NORMAL; + public static final String FAILED_EVENT_EXTENSION = "eventSubscription.failedEvent"; private int offset = -1; private AlertMetrics alertMetrics; @@ -95,48 +84,62 @@ public void handleFailedEvent(EventPublisherException ex) { eventSubscription.getName(), ex.getChangeEvent()); + ChangeEvent event = ex.getChangeEvent(); + + // Update Failed Event with details FailedEvent failedEvent = new FailedEvent() .withChangeEvent(ex.getChangeEvent()) - .withReason(ex.getMessage()) .withRetriesLeft(eventSubscription.getRetries()) .withTimestamp(System.currentTimeMillis()); - // Check in Qtz Map - Set failedEventsList = - JsonUtils.convertValue( - jobDetail.getJobDataMap().get(FAILED_EVENT_EXTENSION), new TypeReference<>() {}); - if (failedEventsList == null) { - failedEventsList = new HashSet<>(); - } - - // Test If the Failing Event is present in the List - boolean removeChangeEvent = - failedEventsList.removeIf( - (failedEvent1) -> { - boolean matched = eventMatch.test(failedEvent1, failedEvent); - if (matched) { - failedEvent.withRetriesLeft(failedEvent1.getRetriesLeft() - 1); - } - return matched; - }); - - // Check if the event was removed - if (removeChangeEvent) { - if (failedEvent.getRetriesLeft() == 0) { - // If the Retries are exhausted, then remove the Event from the List to DLQ - Entity.getCollectionDAO() - .eventSubscriptionDAO() - .upsertFailedEvent( - eventSubscription.getId().toString(), - FAILED_EVENT_EXTENSION, - JsonUtils.pojoToJson(failedEvent.withRetriesLeft(0))); + if (eventSubscription.getRetries() == 0) { + Entity.getCollectionDAO() + .eventSubscriptionDAO() + .upsertFailedEvent( + eventSubscription.getId().toString(), + String.format("%s-%s", FAILED_EVENT_EXTENSION, event.getId()), + JsonUtils.pojoToJson(failedEvent)); + } else { + // Check in Qtz Map + Set failedEventsList = + JsonUtils.convertValue( + jobDetail.getJobDataMap().get(FAILED_EVENT_EXTENSION), new TypeReference<>() {}); + if (failedEventsList == null) { + failedEventsList = new HashSet<>(); } else { - failedEvent.withRetriesLeft(failedEvent.getRetriesLeft() - 1); + // Remove exising change event + boolean removeChangeEvent = + failedEventsList.removeIf( + failedEvent1 -> { + if (failedEvent1 + .getChangeEvent() + .getId() + .equals(failedEvent.getChangeEvent().getId())) { + failedEvent.withRetriesLeft(failedEvent1.getRetriesLeft()); + return true; + } + return false; + }); + + if (removeChangeEvent) { + if (failedEvent.getRetriesLeft() == 0) { + // If the Retries are exhausted, then remove the Event from the List to DLQ + failedEvent.withRetriesLeft(0); + } else { + failedEvent.withRetriesLeft(failedEvent.getRetriesLeft() - 1); + } + } } + failedEventsList.add(failedEvent); + jobDetail.getJobDataMap().put(FAILED_EVENT_EXTENSION, failedEventsList); + Entity.getCollectionDAO() + .eventSubscriptionDAO() + .upsertFailedEvent( + eventSubscription.getId().toString(), + String.format("%s-%s", FAILED_EVENT_EXTENSION, event.getId()), + JsonUtils.pojoToJson(failedEvent)); } - failedEventsList.add(failedEvent); - jobDetail.getJobDataMap().put(FAILED_EVENT_EXTENSION, failedEventsList); } private int loadInitialOffset() { @@ -247,7 +250,7 @@ public synchronized void setErrorStatus(Long attemptTime, Integer statusCode, St } public synchronized void setAwaitingRetry(Long attemptTime, int statusCode, String reason) { - setStatus(AWAITING_RETRY, attemptTime, statusCode, reason, attemptTime + currentBackoffTime); + setStatus(AWAITING_RETRY, attemptTime, statusCode, reason, attemptTime + 10); } public synchronized void setSuccessStatus(Long updateTime) { @@ -313,20 +316,6 @@ public void execute(JobExecutionContext jobExecutionContext) throws JobExecution commit(jobExecutionContext); } - public void setNextBackOff() { - if (currentBackoffTime == BACKOFF_NORMAL) { - currentBackoffTime = BACKOFF_3_SECONDS; - } else if (currentBackoffTime == BACKOFF_3_SECONDS) { - currentBackoffTime = BACKOFF_30_SECONDS; - } else if (currentBackoffTime == BACKOFF_30_SECONDS) { - currentBackoffTime = BACKOFF_5_MINUTES; - } else if (currentBackoffTime == BACKOFF_5_MINUTES) { - currentBackoffTime = BACKOFF_1_HOUR; - } else if (currentBackoffTime == BACKOFF_1_HOUR) { - currentBackoffTime = BACKOFF_24_HOUR; - } - } - public EventSubscription getEventSubscription() { return (EventSubscription) jobDetail.getJobDataMap().get(ALERT_INFO_KEY); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java index feb7086e4baf..6ef4e871ad8c 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/gchat/GChatPublisher.java @@ -75,7 +75,6 @@ public void sendAlert(ChangeEvent event) throws EventPublisherException { postWebhookMessage(this, actionTarget, gchatMessage); } } catch (Exception e) { - Thread.currentThread().interrupt(); String message = CatalogExceptionMessage.eventPublisherFailedToPublish( G_CHAT_WEBHOOK, event, e.getMessage()); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/ChangeEventHandler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/ChangeEventHandler.java index 4def5fa440e2..5e580f7429e5 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/ChangeEventHandler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/ChangeEventHandler.java @@ -110,6 +110,7 @@ public Void process( private static ChangeEvent copyChangeEvent(ChangeEvent changeEvent) { return new ChangeEvent() + .withId(changeEvent.getId()) .withEventType(changeEvent.getEventType()) .withEntityId(changeEvent.getEntityId()) .withEntityType(changeEvent.getEntityType()) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java index a1deb4fe6c0f..9403f3e33736 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java @@ -18,15 +18,18 @@ import java.util.Objects; import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.jdbi.v3.sqlobject.transaction.Transaction; +import org.openmetadata.schema.EntityInterface; import org.openmetadata.schema.api.events.CreateEventSubscription; import org.openmetadata.schema.entity.events.EventSubscription; import org.openmetadata.schema.entity.events.SubscriptionStatus; +import org.openmetadata.service.Entity; import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; import org.openmetadata.service.events.subscription.AlertUtil; +import org.openmetadata.service.exception.UnhandledServerException; +import org.openmetadata.service.jdbi3.EntityRepository; import org.quartz.JobBuilder; import org.quartz.JobDataMap; import org.quartz.JobDetail; @@ -47,8 +50,6 @@ public class EventSubscriptionScheduler { private static EventSubscriptionScheduler instance; private static volatile boolean initialized = false; private final Scheduler alertsScheduler = new StdSchedulerFactory().getScheduler(); - private static final ConcurrentHashMap alertJobMap = - new ConcurrentHashMap<>(); private EventSubscriptionScheduler() throws SchedulerException { this.alertsScheduler.start(); @@ -72,8 +73,8 @@ private static void initialize() throws SchedulerException { } @Transaction - @SneakyThrows - public void addSubscriptionPublisher(EventSubscription eventSubscription) { + public void addSubscriptionPublisher(EventSubscription eventSubscription) + throws SchedulerException { if (Objects.requireNonNull(eventSubscription.getAlertType()) == CreateEventSubscription.AlertType.CHANGE_EVENT) { AbstractEventConsumer publisher = AlertUtil.getNotificationsPublisher(eventSubscription); @@ -95,7 +96,6 @@ public void addSubscriptionPublisher(EventSubscription eventSubscription) { // Schedule the Job alertsScheduler.scheduleJob(jobDetail, trigger); } - alertJobMap.put(eventSubscription.getId(), publisher); LOG.info( "Webhook publisher subscription started as {} : status {}", eventSubscription.getName(), @@ -119,7 +119,7 @@ private JobDetail jobBuilder( private Trigger trigger(EventSubscription eventSubscription) { return TriggerBuilder.newTrigger() .withIdentity(eventSubscription.getId().toString(), ALERT_TRIGGER_GROUP) - .withSchedule(SimpleScheduleBuilder.repeatSecondlyForever(10)) + .withSchedule(SimpleScheduleBuilder.repeatSecondlyForever(3)) .startNow() .build(); } @@ -128,77 +128,83 @@ private SubscriptionStatus getSubscriptionStatusAtCurrentTime(SubscriptionStatus return new SubscriptionStatus().withStatus(status).withTimestamp(System.currentTimeMillis()); } - private AbstractEventConsumer getPublisher(UUID id) { - return alertJobMap.get(id); - } - @Transaction @SneakyThrows public void updateEventSubscription(EventSubscription eventSubscription) { if (Objects.requireNonNull(eventSubscription.getAlertType()) == CreateEventSubscription.AlertType.CHANGE_EVENT) { - if (Boolean.TRUE.equals( - eventSubscription.getEnabled())) { // Only add webhook that is enabled for publishing - // If there was a previous webhook either in disabled state or stopped due - // to errors, update it and restart publishing - AbstractEventConsumer previousPublisher = getPublisher(eventSubscription.getId()); - if (previousPublisher == null) { - if (!ACTIVITY_FEED.equals(eventSubscription.getSubscriptionType())) { - addSubscriptionPublisher(eventSubscription); - } - return; - } - - // Update the existing publisher - deleteEventSubscriptionPublisher(eventSubscription); + // Remove Existing Subscription Publisher + deleteEventSubscriptionPublisher(eventSubscription); + if (Boolean.TRUE.equals(eventSubscription.getEnabled()) + && (!eventSubscription.getSubscriptionType().equals(ACTIVITY_FEED))) { addSubscriptionPublisher(eventSubscription); - } else { - // Remove the webhook publisher - removeProcessorForEventSubscription( - eventSubscription.getId(), - getSubscriptionStatusAtCurrentTime(SubscriptionStatus.Status.DISABLED)); } } else { throw new IllegalArgumentException(INVALID_ALERT); } } - @Transaction - @SneakyThrows - public void removeProcessorForEventSubscription(UUID id, SubscriptionStatus reasonForRemoval) { - AbstractEventConsumer publisher = alertJobMap.get(id); - if (publisher != null) { - alertsScheduler.deleteJob(publisher.getJobDetail().getKey()); - publisher.getEventSubscription().setStatusDetails(reasonForRemoval); - LOG.info("Alert publisher deleted for {}", publisher.getEventSubscription().getName()); - } - } - @Transaction public void deleteEventSubscriptionPublisher(EventSubscription deletedEntity) throws SchedulerException { if (Objects.requireNonNull(deletedEntity.getAlertType()) == CreateEventSubscription.AlertType.CHANGE_EVENT) { - AbstractEventConsumer publisher = alertJobMap.remove(deletedEntity.getId()); - if (publisher != null) { - alertsScheduler.deleteJob(new JobKey(deletedEntity.getId().toString(), ALERT_JOB_GROUP)); - alertsScheduler.unscheduleJob( - new TriggerKey(deletedEntity.getId().toString(), ALERT_TRIGGER_GROUP)); - LOG.info("Alert publisher deleted for {}", deletedEntity.getName()); - } + alertsScheduler.deleteJob(new JobKey(deletedEntity.getId().toString(), ALERT_JOB_GROUP)); + alertsScheduler.unscheduleJob( + new TriggerKey(deletedEntity.getId().toString(), ALERT_TRIGGER_GROUP)); + LOG.info("Alert publisher deleted for {}", deletedEntity.getName()); } else { throw new IllegalArgumentException(INVALID_ALERT); } } public SubscriptionStatus getStatusForEventSubscription(UUID id) { - AbstractEventConsumer publisher = alertJobMap.get(id); - if (publisher != null) { - return publisher.getEventSubscription().getStatusDetails(); + EventSubscription eventSubscription = getEventSubscriptionFromScheduledJob(id); + if (eventSubscription == null) { + EntityRepository subscriptionRepository = + Entity.getEntityRepository(Entity.EVENT_SUBSCRIPTION); + EventSubscription subscription = + (EventSubscription) + subscriptionRepository.get(null, id, subscriptionRepository.getFields("id")); + if (subscription != null && (Boolean.FALSE.equals(subscription.getEnabled()))) { + return new SubscriptionStatus().withStatus(SubscriptionStatus.Status.DISABLED); + } + } else { + return eventSubscription.getStatusDetails(); } return null; } + public EventSubscription getEventSubscriptionFromScheduledJob(UUID id) { + try { + JobDetail jobDetail = + alertsScheduler.getJobDetail(new JobKey(id.toString(), ALERT_JOB_GROUP)); + if (jobDetail != null) { + return ((EventSubscription) jobDetail.getJobDataMap().get(ALERT_INFO_KEY)); + } + } catch (SchedulerException ex) { + LOG.error("Failed to get Event Subscription from Job, Subscription Id : {}", id); + } + return null; + } + + public EventSubscription putInJobDataMap(UUID subscriptionID, String key, Object obj) { + try { + JobDetail jobDetail = + alertsScheduler.getJobDetail(new JobKey(subscriptionID.toString(), ALERT_JOB_GROUP)); + if (jobDetail != null) { + jobDetail.getJobDataMap().put(key, obj); + return ((EventSubscription) jobDetail.getJobDataMap().get(ALERT_INFO_KEY)); + } + } catch (Exception ex) { + LOG.error( + "Failed to get Event Subscription from Job, Subscription Id : {}, Exception: ", + subscriptionID.toString(), + ex); + } + throw new UnhandledServerException("Cannot find Job Data Map for give Subscription"); + } + public static void shutDown() throws SchedulerException { if (instance != null) { instance.alertsScheduler.shutdown(); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java index c25ec40135a0..d9d25f9dc84c 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java @@ -247,14 +247,7 @@ public static List getFilteredEvent( } if (triggerChangeEvent) { - // Ignore the event since change description is null - if (event.getChangeDescription() != null) { - filteredEvents.add(event); - } else { - LOG.debug( - "Email Publisher Event Will be Ignored Since Change Description is null. Received Event: {}", - event); - } + filteredEvents.add(event); } } return filteredEvents; diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/util/FormatterUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/util/FormatterUtil.java index 44d354b7fc3c..0c7c581d99bd 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/util/FormatterUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/util/FormatterUtil.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.UUID; import java.util.stream.Collectors; import javax.json.JsonArray; import javax.json.JsonObject; @@ -337,6 +338,7 @@ public static ChangeEvent getChangeEventFromResponseContext( private static ChangeEvent getChangeEvent( String updateBy, EventType eventType, String entityType, EntityInterface entityInterface) { return new ChangeEvent() + .withId(UUID.randomUUID()) .withEventType(eventType) .withEntityId(entityInterface.getId()) .withEntityType(entityType) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EntityRepository.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EntityRepository.java index 48a8ea7ee4da..58da38df2255 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EntityRepository.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/EntityRepository.java @@ -865,6 +865,7 @@ public final PutResponse addFollower(String updatedBy, UUID entityId, UUID us ChangeEvent changeEvent = new ChangeEvent() + .withId(UUID.randomUUID()) .withEntity(entity) .withChangeDescription(change) .withEventType(EventType.ENTITY_UPDATED) @@ -912,6 +913,7 @@ public final PutResponse updateVote(String updatedBy, UUID entityId, VoteRequ setFieldsInternal(originalEntity, new Fields(allowedFields, "votes")); ChangeEvent changeEvent = new ChangeEvent() + .withId(UUID.randomUUID()) .withEntity(originalEntity) .withChangeDescription(change) .withEventType(EventType.ENTITY_UPDATED) @@ -1099,6 +1101,7 @@ public final PutResponse deleteFollower(String updatedBy, UUID entityId, UUID ChangeEvent changeEvent = new ChangeEvent() + .withId(UUID.randomUUID()) .withEntity(entity) .withChangeDescription(change) .withEventType(EventType.ENTITY_UPDATED) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/IngestionPipelineRepository.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/IngestionPipelineRepository.java index 0f4c9bbbe17d..cd52c9bd05ed 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/IngestionPipelineRepository.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/IngestionPipelineRepository.java @@ -156,6 +156,7 @@ public void setPipelineServiceClient(PipelineServiceClient client) { private ChangeEvent getChangeEvent( EntityInterface updated, ChangeDescription change, String entityType, Double prevVersion) { return new ChangeEvent() + .withId(UUID.randomUUID()) .withEntity(updated) .withChangeDescription(change) .withEventType(EventType.ENTITY_UPDATED) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/KpiRepository.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/KpiRepository.java index 2a49225d5911..9ee9073ff87b 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/KpiRepository.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/KpiRepository.java @@ -6,6 +6,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import javax.ws.rs.core.Response; import javax.ws.rs.core.UriInfo; import org.jdbi.v3.sqlobject.transaction.Transaction; @@ -185,6 +186,7 @@ public ResultList getKpiResults( private ChangeEvent getChangeEvent( EntityInterface updated, ChangeDescription change, String entityType, Double prevVersion) { return new ChangeEvent() + .withId(UUID.randomUUID()) .withEntity(updated) .withChangeDescription(change) .withEventType(EventType.ENTITY_UPDATED) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/QueryRepository.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/QueryRepository.java index fb9e10af19cb..1d7aad0e8eb0 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/QueryRepository.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/QueryRepository.java @@ -229,6 +229,7 @@ private ChangeEvent getQueryChangeEvent( new ChangeDescription().withPreviousVersion(updatedQuery.getVersion()); change.getFieldsUpdated().add(fieldChange); return new ChangeEvent() + .withId(UUID.randomUUID()) .withEntity(updatedQuery) .withChangeDescription(change) .withEventType(EventType.ENTITY_UPDATED) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/TestCaseRepository.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/TestCaseRepository.java index fd80d95126da..adab81ee5e3b 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/TestCaseRepository.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/TestCaseRepository.java @@ -446,6 +446,7 @@ private ChangeEvent getChangeEvent( String entityType, Double prevVersion) { return new ChangeEvent() + .withId(UUID.randomUUID()) .withEntity(updated) .withChangeDescription(change) .withEventType(EventType.ENTITY_UPDATED) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/UsageRepository.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/UsageRepository.java index c24f6a968714..d91566ce7210 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/UsageRepository.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/UsageRepository.java @@ -211,6 +211,7 @@ private void insertToUsageRepository( private ChangeEvent getChangeEvent( EntityInterface updated, ChangeDescription change, String entityType, Double prevVersion) { return new ChangeEvent() + .withId(UUID.randomUUID()) .withEntity(updated) .withChangeDescription(change) .withEventType(EventType.ENTITY_UPDATED) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java b/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java index ad1a8646fe48..3a47bb48de26 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java @@ -269,7 +269,8 @@ public EventSubscription getEventsSubscriptionByName( public Response createEventSubscription( @Context UriInfo uriInfo, @Context SecurityContext securityContext, - @Valid CreateEventSubscription request) { + @Valid CreateEventSubscription request) + throws SchedulerException { EventSubscription eventSub = getEventSubscription(request, securityContext.getUserPrincipal().getName()); // Only one Creation is allowed @@ -556,7 +557,9 @@ public EventSubscription getEventSubscription(CreateEventSubscription create, St .withFilteringRules(create.getFilteringRules()) .withSubscriptionType(create.getSubscriptionType()) .withSubscriptionConfig(create.getSubscriptionConfig()) - .withProvider(create.getProvider()); + .withProvider(create.getProvider()) + .withRetries(create.getRetries()) + .withPollInterval(create.getPollInterval()); } public static List getDescriptors() throws IOException { diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java index ec105214ae09..c3093e6d7fa2 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/util/SubscriptionUtil.java @@ -229,7 +229,6 @@ public static void postWebhookMessage( attemptTime, response.getStatus(), response.getStatusInfo().getReasonPhrase()); } else if (response.getStatus() >= 400 && response.getStatus() < 600) { // 4xx, 5xx response retry delivering events after timeout - publisher.setNextBackOff(); publisher.setAwaitingRetry( attemptTime, response.getStatus(), response.getStatusInfo().getReasonPhrase()); } else if (response.getStatus() == 200) { diff --git a/openmetadata-service/src/test/java/org/openmetadata/service/monitoring/CloudWatchEventMonitorTest.java b/openmetadata-service/src/test/java/org/openmetadata/service/monitoring/CloudWatchEventMonitorTest.java index b2d3ccda8d11..53af4f9c5a04 100644 --- a/openmetadata-service/src/test/java/org/openmetadata/service/monitoring/CloudWatchEventMonitorTest.java +++ b/openmetadata-service/src/test/java/org/openmetadata/service/monitoring/CloudWatchEventMonitorTest.java @@ -63,6 +63,7 @@ public class CloudWatchEventMonitorTest { private ChangeEvent buildChangeEvent(EventType eventType) { return new ChangeEvent() + .withId(UUID.randomUUID()) .withEntityType(Entity.INGESTION_PIPELINE) .withEventType(eventType) .withEntityFullyQualifiedName(FQN) diff --git a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java index ca8798613fd5..199469766637 100644 --- a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java +++ b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java @@ -139,14 +139,14 @@ void post_alertActionWithEnabledStateChange(TestInfo test) throws IOException { @Test void put_updateEndpointURL(TestInfo test) throws IOException { String webhookName = getEntityName(test); - LOG.info("creating webhook in disabled state"); String uri = "http://invalidUnknowHost"; Webhook genericWebhook = getWebhook(uri); CreateEventSubscription genericWebhookActionRequest = createRequest(webhookName) .withEnabled(true) .withSubscriptionType(CreateEventSubscription.SubscriptionType.GENERIC_WEBHOOK) - .withSubscriptionConfig(genericWebhook); + .withSubscriptionConfig(genericWebhook) + .withRetries(0); EventSubscription alert = createAndCheckEntity(genericWebhookActionRequest, ADMIN_AUTH_HEADERS); // Wait for webhook to be marked as failed @@ -447,8 +447,8 @@ public void waitAndCheckForEvents( } Awaitility.await() - .pollInterval(Duration.ofMillis(100L)) - .atMost(Duration.ofMillis(iteration * 100L)) + .pollInterval(Duration.ofMillis(10000L)) + .atMost(Duration.ofMillis(iteration * 1000L)) .untilTrue(receivedAllEvents(expected, callbackEvents)); if (expected.size() > callbackEvents.size()) { // Failed to receive all the events expected.forEach( @@ -557,6 +557,8 @@ public CreateEventSubscription createRequest(String name) { .withSubscriptionConfig(getWebhook(uri)) .withEnabled(true) .withBatchSize(10) + .withRetries(0) + .withPollInterval(0) .withAlertType(CreateEventSubscription.AlertType.CHANGE_EVENT); } diff --git a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/WebhookCallbackResource.java b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/WebhookCallbackResource.java index 017bab3673de..4e3d11339006 100644 --- a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/WebhookCallbackResource.java +++ b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/WebhookCallbackResource.java @@ -29,7 +29,6 @@ import org.openmetadata.common.utils.CommonUtil; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.schema.type.EventType; -import org.openmetadata.service.resources.events.EventResource.EventList; import org.openmetadata.service.util.JsonUtils; import org.openmetadata.service.util.RestUtil; @@ -55,11 +54,11 @@ public Response receiveEventCount( @Parameter(description = "Name of the Webhook callback", schema = @Schema(type = "string")) @PathParam("name") String name, - EventList events) { - String payload = JsonUtils.pojoToJson(events); + ChangeEvent event) { + String payload = JsonUtils.pojoToJson(event); String computedSignature = "sha256=" + CommonUtil.calculateHMAC("webhookTest", payload); assertEquals(computedSignature, signature); - addEventDetails(name, events); + addEventDetails(name, event); return Response.ok().build(); } @@ -67,8 +66,8 @@ public Response receiveEventCount( @POST @Path("/simulate/slowServer") public Response receiveEventWithDelay( - @Context UriInfo uriInfo, @Context SecurityContext securityContext, EventList events) { - addEventDetails("simulate-slowServer", events); + @Context UriInfo uriInfo, @Context SecurityContext securityContext, ChangeEvent event) { + addEventDetails("simulate-slowServer", event); return Response.ok().build(); } @@ -76,8 +75,8 @@ public Response receiveEventWithDelay( @POST @Path("/simulate/timeout") public Response receiveEventWithTimeout( - @Context UriInfo uriInfo, @Context SecurityContext securityContext, EventList events) { - addEventDetails("simulate-timeout", events); + @Context UriInfo uriInfo, @Context SecurityContext securityContext, ChangeEvent event) { + addEventDetails("simulate-timeout", event); Awaitility.await().pollDelay(Duration.ofSeconds(100L)).untilTrue(new AtomicBoolean(true)); return Response.ok().build(); } @@ -86,8 +85,8 @@ public Response receiveEventWithTimeout( @POST @Path("/simulate/300") public Response receiveEvent300( - @Context UriInfo uriInfo, @Context SecurityContext securityContext, EventList events) { - addEventDetails("simulate-300", events); + @Context UriInfo uriInfo, @Context SecurityContext securityContext, ChangeEvent event) { + addEventDetails("simulate-300", event); return Response.status(Response.Status.MOVED_PERMANENTLY).build(); } @@ -95,8 +94,8 @@ public Response receiveEvent300( @POST @Path("/simulate/400") public Response receiveEvent400( - @Context UriInfo uriInfo, @Context SecurityContext securityContext, EventList events) { - addEventDetails("simulate-400", events); + @Context UriInfo uriInfo, @Context SecurityContext securityContext, ChangeEvent event) { + addEventDetails("simulate-400", event); return Response.status(Response.Status.BAD_REQUEST).build(); } @@ -104,8 +103,8 @@ public Response receiveEvent400( @POST @Path("/simulate/500") public Response receiveEvent500( - @Context UriInfo uriInfo, @Context SecurityContext securityContext, EventList events) { - addEventDetails("simulate-500", events); + @Context UriInfo uriInfo, @Context SecurityContext securityContext, ChangeEvent event) { + addEventDetails("simulate-500", event); return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build(); } @@ -124,14 +123,15 @@ public Response receiveEntityEvents( @Parameter(description = "Type of entity", schema = @Schema(type = "string")) @PathParam("entityType") String entityType, - EventList events) { + ChangeEvent event) { String key = eventType + ":" + entityType; List list = entityCallbackMap.get(key); if (list == null) { - list = new ArrayList<>(events.getData()); + list = new ArrayList<>(); + list.add(event); entityCallbackMap.put(key, list); } else { - list.addAll(events.getData()); + list.add(event); } LOG.debug( "callback /{}/{} received event. Current count {}", eventType, entityType, list.size()); @@ -146,15 +146,15 @@ public EventDetails getEventDetails(String endpoint) { return eventMap.get(endpoint); } - private void addEventDetails(String endpoint, EventList events) { + private void addEventDetails(String endpoint, ChangeEvent event) { EventDetails details = eventMap.get(endpoint); // Default endpoint if (details == null) { details = new EventDetails(); - details.setFirstEventTime(events.getData().get(0).getTimestamp()); + details.setFirstEventTime(event.getTimestamp()); eventMap.put(endpoint, details); } - details.getEvents().addAll(events.getData()); - details.setLatestEventTime(events.getData().get(events.getData().size() - 1).getTimestamp()); + details.getEvents().add(event); + details.setLatestEventTime(event.getTimestamp()); LOG.info("Event received {}, total count {}", endpoint, details.getEvents().size()); } diff --git a/openmetadata-spec/src/main/resources/json/schema/events/api/createEventSubscription.json b/openmetadata-spec/src/main/resources/json/schema/events/api/createEventSubscription.json index 079ed19c960e..df3ff13705a3 100644 --- a/openmetadata-spec/src/main/resources/json/schema/events/api/createEventSubscription.json +++ b/openmetadata-spec/src/main/resources/json/schema/events/api/createEventSubscription.json @@ -63,6 +63,16 @@ }, "provider" : { "$ref": "../../type/basic.json#/definitions/providerType" + }, + "retries": { + "description": "Number of times to retry callback on failure. (Default 3).", + "type": "integer", + "default": 3 + }, + "pollInterval": { + "description": "Poll Interval in seconds.", + "type": "integer", + "default": 10 } }, "required": ["name", "alertType", "subscriptionType"], diff --git a/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json b/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json index 5f73146b1e93..99c155a1631f 100644 --- a/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json +++ b/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json @@ -248,6 +248,11 @@ "description": "Number of times to retry callback on failure. (Default 3).", "type": "integer", "default": 3 + }, + "pollInterval": { + "description": "Poll Interval in seconds.", + "type": "integer", + "default": 10 } }, "required": ["id", "name", "alertType", "subscriptionType"], diff --git a/openmetadata-spec/src/main/resources/json/schema/type/changeEvent.json b/openmetadata-spec/src/main/resources/json/schema/type/changeEvent.json index 64840bab977e..95a656b68c04 100644 --- a/openmetadata-spec/src/main/resources/json/schema/type/changeEvent.json +++ b/openmetadata-spec/src/main/resources/json/schema/type/changeEvent.json @@ -205,6 +205,10 @@ } }, "properties": { + "id": { + "description": "Unique identifier for the event.", + "$ref": "basic.json#/definitions/uuid" + }, "eventType": { "$ref": "#/definitions/eventType" }, @@ -244,6 +248,6 @@ "description": "For `eventType` `entityCreated`, this field captures JSON coded string of the entity using the schema corresponding to `entityType`." } }, - "required": ["eventType", "entityType", "entityId", "timestamp"], + "required": ["id", "eventType", "entityType", "entityId", "timestamp"], "additionalProperties": false } From a1af7151eb0e4c34ac013fe8741c2220a1118551 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Wed, 3 Jan 2024 16:54:10 +0530 Subject: [PATCH 12/28] Fix Failing Tests --- .../changeEvent/AbstractEventConsumer.java | 30 ++++++--- .../scheduled/EventSubscriptionScheduler.java | 15 +++-- .../EventSubscriptionResource.java | 27 ++++++++ .../events/EventSubscriptionResourceTest.java | 64 ++++++++++++++----- 4 files changed, 105 insertions(+), 31 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index 37aec3b49f54..625b5a24e534 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -49,8 +49,8 @@ public abstract class AbstractEventConsumer implements Consumer, Job { public static final String ALERT_OFFSET_KEY = "alertOffsetKey"; public static final String ALERT_INFO_KEY = "alertInfoKey"; - private static final String OFFSET_EXTENSION = "eventSubscription.Offset"; - private static final String METRICS_EXTENSION = "eventSubscription.metrics"; + public static final String OFFSET_EXTENSION = "eventSubscription.Offset"; + public static final String METRICS_EXTENSION = "eventSubscription.metrics"; public static final String FAILED_EVENT_EXTENSION = "eventSubscription.failedEvent"; private int offset = -1; private AlertMetrics alertMetrics; @@ -65,7 +65,7 @@ private void init(JobExecutionContext context) { (EventSubscription) context.getJobDetail().getJobDataMap().get(ALERT_INFO_KEY); this.jobDetail = context.getJobDetail(); this.eventSubscription = sub; - this.offset = loadInitialOffset(); + this.offset = loadInitialOffset(context); this.alertMetrics = loadInitialMetrics(); this.doInit(context); } @@ -142,7 +142,7 @@ public void handleFailedEvent(EventPublisherException ex) { } } - private int loadInitialOffset() { + private int loadInitialOffset(JobExecutionContext context) { EventSubscriptionOffset jobStoredOffset = (EventSubscriptionOffset) jobDetail.getJobDataMap().get(ALERT_OFFSET_KEY); // If the Job Data Map has the latest offset, use it @@ -162,6 +162,14 @@ private int loadInitialOffset() { eventSubscriptionOffset = Entity.getCollectionDAO().changeEventDAO().listCount(); } // Update the Job Data Map with the latest offset + context + .getJobDetail() + .getJobDataMap() + .put( + ALERT_OFFSET_KEY, + new EventSubscriptionOffset() + .withOffset(eventSubscriptionOffset) + .withTimestamp(System.currentTimeMillis())); return eventSubscriptionOffset; } } @@ -307,13 +315,15 @@ public void execute(JobExecutionContext jobExecutionContext) throws JobExecution batch.addAll(failedChangeEvents); } - // Publish Events - alertMetrics.withTotalEvents(alertMetrics.getTotalEvents() + batch.size()); - publishEvents(batch); + if (!batch.isEmpty()) { + // Publish Events + alertMetrics.withTotalEvents(alertMetrics.getTotalEvents() + batch.size()); + publishEvents(batch); - // Commit the Offset - offset += batchSize; - commit(jobExecutionContext); + // Commit the Offset + offset += batchSize; + commit(jobExecutionContext); + } } public EventSubscription getEventSubscription() { diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java index 9403f3e33736..4da855493367 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java @@ -15,6 +15,7 @@ import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.ACTIVITY_FEED; import static org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer.ALERT_INFO_KEY; +import static org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer.ALERT_OFFSET_KEY; import java.util.Objects; import java.util.UUID; @@ -24,11 +25,11 @@ import org.openmetadata.schema.EntityInterface; import org.openmetadata.schema.api.events.CreateEventSubscription; import org.openmetadata.schema.entity.events.EventSubscription; +import org.openmetadata.schema.entity.events.EventSubscriptionOffset; import org.openmetadata.schema.entity.events.SubscriptionStatus; import org.openmetadata.service.Entity; import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; import org.openmetadata.service.events.subscription.AlertUtil; -import org.openmetadata.service.exception.UnhandledServerException; import org.openmetadata.service.jdbi3.EntityRepository; import org.quartz.JobBuilder; import org.quartz.JobDataMap; @@ -188,13 +189,17 @@ public EventSubscription getEventSubscriptionFromScheduledJob(UUID id) { return null; } - public EventSubscription putInJobDataMap(UUID subscriptionID, String key, Object obj) { + public boolean checkIfPublisherPublishedAllEvents(UUID subscriptionID) { + int countOfEvents = Entity.getCollectionDAO().changeEventDAO().listCount(); try { JobDetail jobDetail = alertsScheduler.getJobDetail(new JobKey(subscriptionID.toString(), ALERT_JOB_GROUP)); if (jobDetail != null) { - jobDetail.getJobDataMap().put(key, obj); - return ((EventSubscription) jobDetail.getJobDataMap().get(ALERT_INFO_KEY)); + EventSubscriptionOffset offset = + ((EventSubscriptionOffset) jobDetail.getJobDataMap().get(ALERT_OFFSET_KEY)); + if (offset != null) { + return offset.getOffset() == countOfEvents; + } } } catch (Exception ex) { LOG.error( @@ -202,7 +207,7 @@ public EventSubscription putInJobDataMap(UUID subscriptionID, String key, Object subscriptionID.toString(), ex); } - throw new UnhandledServerException("Cannot find Job Data Map for give Subscription"); + return false; } public static void shutDown() throws SchedulerException { diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java b/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java index 3a47bb48de26..03021e4530e1 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java @@ -362,6 +362,33 @@ public EntityHistory listEventSubscriptionVersions( return super.listVersionsInternal(securityContext, id); } + @GET + @Path("/{id}/processedEvents") + @Operation( + operationId = "checkIfThePublisherProcessedALlEvents", + summary = "Check If the Publisher Processed All Events", + description = + "Return a boolean 'true' or 'false' to indicate if the publisher processed all events", + responses = { + @ApiResponse( + responseCode = "200", + description = "List of Event Subscription versions", + content = + @Content( + mediaType = "application/json", + schema = @Schema(implementation = EntityHistory.class))) + }) + public Response checkIfThePublisherProcessedALlEvents( + @Context UriInfo uriInfo, + @Context SecurityContext securityContext, + @Parameter(description = "Id of the Event Subscription", schema = @Schema(type = "UUID")) + @PathParam("id") + UUID id) { + return Response.ok() + .entity(EventSubscriptionScheduler.getInstance().checkIfPublisherPublishedAllEvents(id)) + .build(); + } + @GET @Path("/{id}/versions/{version}") @Operation( diff --git a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java index 199469766637..9c18f63bd103 100644 --- a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java +++ b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java @@ -186,6 +186,21 @@ void put_updateEndpointURL(TestInfo test) throws IOException { deleteEntity(alert.getId(), ADMIN_AUTH_HEADERS); } + private void waitForAllEventToComplete(UUID alertId) throws HttpResponseException { + boolean result; + do { + WebTarget target = + getResource(String.format("%s/%s/processedEvents", collectionName, alertId.toString())); + result = TestUtils.getWithResponse(target, Boolean.class, ADMIN_AUTH_HEADERS, 200); + LOG.info("waitForAllEventToComplete alertId: {} , result: {}", alertId, result); + try { + Thread.sleep(3000L); + } catch (InterruptedException e) { + LOG.error("waitForAllEventToComplete InterruptedException: {}", e.getMessage()); + } + } while (!result); + } + @Test void put_updateAlertUpdateFields(TestInfo test) throws IOException { // @@ -332,6 +347,14 @@ void testDifferentTypesOfAlerts() throws IOException { CreateEventSubscription w6ActionRequest = createRequest(alertName).withSubscriptionConfig(w6); EventSubscription w6Alert = createAndCheckEntity(w6ActionRequest, ADMIN_AUTH_HEADERS); + // Wait for events to complete + waitForAllEventToComplete(w1Alert.getId()); + waitForAllEventToComplete(w2Alert.getId()); + waitForAllEventToComplete(w3Alert.getId()); + waitForAllEventToComplete(w4Alert.getId()); + waitForAllEventToComplete(w5Alert.getId()); + waitForAllEventToComplete(w6Alert.getId()); + // Now check state of webhooks created WebhookCallbackResource.EventDetails details = waitForFirstEvent("simulate-slowServer", 25); ConcurrentLinkedQueue callbackEvents = details.getEvents(); @@ -359,6 +382,7 @@ void testDifferentTypesOfAlerts() throws IOException { private AtomicBoolean testExpectedStatus(UUID id, SubscriptionStatus.Status expectedStatus) throws HttpResponseException { + waitForAllEventToComplete(id); SubscriptionStatus status = getStatus(id, Response.Status.OK.getStatusCode()); LOG.info("webhook status {}", status.getStatus()); return new AtomicBoolean(status.getStatus() == expectedStatus); @@ -382,6 +406,8 @@ public void startWebhookSubscription() throws IOException { */ public void validateWebhookEvents() throws HttpResponseException { // Check the healthy callback server received all the change events + EventSubscription healthySub = getEntityByName("healthy", null, "", ADMIN_AUTH_HEADERS); + waitForAllEventToComplete(healthySub.getId()); WebhookCallbackResource.EventDetails details = webhookCallbackResource.getEventDetails("healthy"); assertNotNull(details); @@ -397,6 +423,14 @@ public void validateWebhookEvents() throws HttpResponseException { public void validateWebhookEntityEvents(String entity) throws HttpResponseException { // Check the healthy callback server received all the change events // For the entity all the webhooks registered for created events have the right number of events + EventSubscription createdSub = + getEntityByName(EventType.ENTITY_CREATED + "_" + entity, null, "", ADMIN_AUTH_HEADERS); + EventSubscription updatedSub = + getEntityByName(EventType.ENTITY_UPDATED + "_" + entity, null, "", ADMIN_AUTH_HEADERS); + + waitForAllEventToComplete(createdSub.getId()); + waitForAllEventToComplete(updatedSub.getId()); + List callbackEvents = webhookCallbackResource.getEntityCallbackEvents(EventType.ENTITY_CREATED, entity); assertTrue(callbackEvents.size() > 0); @@ -430,22 +464,6 @@ public void waitAndCheckForEvents( ADMIN_AUTH_HEADERS) .getData(); - // Comparison if all callBack Event are there in expected - for (ChangeEvent changeEvent : callbackEvents) { - boolean found = false; - for (ChangeEvent expectedChangeEvent : expected) { - if (changeEvent.getEventType().equals(expectedChangeEvent.getEventType()) - && changeEvent.getEntityId().equals(expectedChangeEvent.getEntityId())) { - found = true; - break; - } - } - if (!found) { - LOG.error( - "[ChangeEventError] Change Events Missing from Expected: {}", changeEvent.toString()); - } - } - Awaitility.await() .pollInterval(Duration.ofMillis(10000L)) .atMost(Duration.ofMillis(iteration * 1000L)) @@ -499,6 +517,20 @@ private SubscriptionStatus getStatus(UUID alertId, int statusCode) throws HttpRe private static AtomicBoolean receivedAllEvents( List expected, Collection callbackEvents) { + for (ChangeEvent expectedChangeEvent : expected) { + boolean found = false; + for (ChangeEvent changeEvent : callbackEvents) { + if (changeEvent.getId().equals(expectedChangeEvent.getId())) { + found = true; + break; + } + } + if (!found) { + LOG.error( + "[ChangeEventError] Change Events Missing from Callback: {}", + expectedChangeEvent.toString()); + } + } LOG.info("expected size {} callback events size {}", expected.size(), callbackEvents.size()); return new AtomicBoolean(expected.size() <= callbackEvents.size()); } From 7cc2b8684bb3c41ff945706f389e28a3d5ebdf2c Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Wed, 3 Jan 2024 16:58:31 +0530 Subject: [PATCH 13/28] Fix Failing Tests --- .../resources/events/EventSubscriptionResourceTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java index 9c18f63bd103..9e9ec8f29505 100644 --- a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java +++ b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java @@ -192,7 +192,7 @@ private void waitForAllEventToComplete(UUID alertId) throws HttpResponseExceptio WebTarget target = getResource(String.format("%s/%s/processedEvents", collectionName, alertId.toString())); result = TestUtils.getWithResponse(target, Boolean.class, ADMIN_AUTH_HEADERS, 200); - LOG.info("waitForAllEventToComplete alertId: {} , result: {}", alertId, result); + LOG.debug("waitForAllEventToComplete alertId: {} , result: {}", alertId, result); try { Thread.sleep(3000L); } catch (InterruptedException e) { @@ -369,7 +369,6 @@ void testDifferentTypesOfAlerts() throws IOException { assertAlertStatus(w3Alert.getId(), FAILED, 301, "Moved Permanently"); assertAlertStatus(w4Alert.getId(), AWAITING_RETRY, 400, "Bad Request"); assertAlertStatus(w5Alert.getId(), AWAITING_RETRY, 500, "Internal Server Error"); - assertAlertStatus(w6Alert.getId(), FAILED, 400, "UnknownHostException"); // Delete all webhooks deleteEntity(w1Alert.getId(), ADMIN_AUTH_HEADERS); From 22b8973dcec7819cc63ff55011f665feba0d9c28 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Wed, 3 Jan 2024 18:48:40 +0530 Subject: [PATCH 14/28] Postgres Fix --- .../java/org/openmetadata/service/jdbi3/CollectionDAO.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java index dcb84819f4c2..738f809108fa 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java @@ -1564,7 +1564,7 @@ default boolean supportsSoftDelete() { @ConnectionAwareSqlUpdate( value = "INSERT INTO change_event_consumers(id, extension, jsonSchema, json) " - + "VALUES (:id, :extension, :jsonSchema, (:json :: jsonb)) " + + "VALUES (:id, :extension, :jsonSchema, (:json :: jsonb)) ON CONFLICT (id, extension) " + "DO UPDATE SET json = EXCLUDED.json, jsonSchema = EXCLUDED.jsonSchema", connectionType = POSTGRES) void upsertSubscriberExtension( @@ -1582,7 +1582,7 @@ void upsertSubscriberExtension( @ConnectionAwareSqlUpdate( value = "INSERT INTO consumers_dlq(id, extension, json) " - + "VALUES (:id, :extension, (:json :: jsonb)) " + + "VALUES (:id, :extension, (:json :: jsonb)) ON CONFLICT (id, extension) " + "DO UPDATE SET json = EXCLUDED.json", connectionType = POSTGRES) void upsertFailedEvent( From 18e7c59a3adfa62169a8513fb010923c499f08fe Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Wed, 3 Jan 2024 23:45:15 +0530 Subject: [PATCH 15/28] Fix Schema Change for Change Event Table --- .../sql/migrations/native/1.3.0/mysql/schemaChanges.sql | 6 ++---- .../sql/migrations/native/1.3.0/postgres/schemaChanges.sql | 6 ++---- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql index fdae5ded840d..8056b989592d 100644 --- a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql +++ b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql @@ -133,9 +133,6 @@ UPDATE dbservice_entity de SET de.json = JSON_REMOVE(de.json, '$.connection.config.useUnityCatalog') WHERE de.serviceType IN ('Databricks','UnityCatalog'); --- Update Change Event Table -ALTER TABLE change_event ADD COLUMN offset INT AUTO_INCREMENT PRIMARY KEY; - -- Add new table for event subscription extensions CREATE TABLE IF NOT EXISTS change_event_consumers ( id VARCHAR(36) NOT NULL, @@ -148,7 +145,8 @@ CREATE TABLE IF NOT EXISTS change_event_consumers ( DELETE FROM event_subscription_entity ese where name = 'DataInsightReport'; -ALTER TABLE change_event_consumers ADD COLUMN offset INT AUTO_INCREMENT; +-- Update Change Event Table +ALTER TABLE change_event ADD COLUMN offset INT AUTO_INCREMENT; CREATE TABLE IF NOT EXISTS consumers_dlq ( id VARCHAR(36) NOT NULL, diff --git a/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql b/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql index 2b12bca100f3..8ea491ef8995 100644 --- a/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql +++ b/bootstrap/sql/migrations/native/1.3.0/postgres/schemaChanges.sql @@ -145,9 +145,6 @@ UPDATE dbservice_entity de SET json = json #- '{connection,config,useUnityCatalog}' WHERE de.serviceType IN ('Databricks','UnityCatalog'); --- Update Change Event Table -ALTER TABLE change_event ADD COLUMN offset SERIAL PRIMARY KEY; - -- Add new table for event subscription extensions CREATE TABLE IF NOT EXISTS change_event_consumers ( id VARCHAR(36) NOT NULL, @@ -160,7 +157,8 @@ CREATE TABLE IF NOT EXISTS change_event_consumers ( DELETE FROM event_subscription_entity ese where name = 'DataInsightReport'; -ALTER TABLE change_event_consumers ADD COLUMN offset SERIAL; +-- Update Change Event Table +ALTER TABLE change_event ADD COLUMN "offset" SERIAL PRIMARY KEY; CREATE TABLE IF NOT EXISTS consumers_dlq ( id VARCHAR(36) NOT NULL, From d2a4186f4d8402a30b64eb229118dfde37a3931e Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Thu, 4 Jan 2024 00:42:23 +0530 Subject: [PATCH 16/28] Failing test --- .../changeEvent/AbstractEventConsumer.java | 26 ++--------- .../scheduled/EventSubscriptionScheduler.java | 1 + .../events/subscription/AlertUtil.java | 20 ++++++++ .../events/EventSubscriptionResourceTest.java | 46 ++++++++++++------- 4 files changed, 56 insertions(+), 37 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index 625b5a24e534..af93010de287 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -17,6 +17,7 @@ import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.AWAITING_RETRY; import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.FAILED; import static org.openmetadata.service.events.subscription.AlertUtil.getFilteredEvent; +import static org.openmetadata.service.events.subscription.AlertUtil.getInitialAlertOffsetFromDb; import com.fasterxml.jackson.core.type.TypeReference; import java.util.ArrayList; @@ -149,28 +150,11 @@ private int loadInitialOffset(JobExecutionContext context) { if (jobStoredOffset != null) { return jobStoredOffset.getOffset(); } else { - int eventSubscriptionOffset; - String json = - Entity.getCollectionDAO() - .eventSubscriptionDAO() - .getSubscriberExtension(eventSubscription.getId().toString(), OFFSET_EXTENSION); - if (json != null) { - EventSubscriptionOffset offsetFromDb = - JsonUtils.readValue(json, EventSubscriptionOffset.class); - eventSubscriptionOffset = offsetFromDb.getOffset(); - } else { - eventSubscriptionOffset = Entity.getCollectionDAO().changeEventDAO().listCount(); - } + EventSubscriptionOffset eventSubscriptionOffset = + getInitialAlertOffsetFromDb(eventSubscription.getId()); // Update the Job Data Map with the latest offset - context - .getJobDetail() - .getJobDataMap() - .put( - ALERT_OFFSET_KEY, - new EventSubscriptionOffset() - .withOffset(eventSubscriptionOffset) - .withTimestamp(System.currentTimeMillis())); - return eventSubscriptionOffset; + context.getJobDetail().getJobDataMap().put(ALERT_OFFSET_KEY, eventSubscriptionOffset); + return eventSubscriptionOffset.getOffset(); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java index 4da855493367..0ce89572f1e8 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java @@ -110,6 +110,7 @@ private JobDetail jobBuilder( AbstractEventConsumer consumer, EventSubscription eventSubscription, String jobIdentity) { JobDataMap dataMap = new JobDataMap(); dataMap.put(ALERT_INFO_KEY, eventSubscription); + dataMap.put(ALERT_OFFSET_KEY, getEventSubscriptionFromScheduledJob(eventSubscription.getId())); JobBuilder jobBuilder = JobBuilder.newJob(consumer.getClass()) .withIdentity(jobIdentity, ALERT_JOB_GROUP) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java index d9d25f9dc84c..5b736e5065d0 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java @@ -16,6 +16,7 @@ import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.ACTIVITY_FEED; import static org.openmetadata.service.Entity.TEAM; import static org.openmetadata.service.Entity.USER; +import static org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer.OFFSET_EXTENSION; import static org.openmetadata.service.security.policyevaluator.CompiledRule.parseExpression; import java.util.ArrayList; @@ -24,12 +25,14 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import java.util.stream.Stream; import javax.ws.rs.BadRequestException; import lombok.extern.slf4j.Slf4j; import org.openmetadata.schema.api.events.CreateEventSubscription; import org.openmetadata.schema.entity.events.EventFilterRule; import org.openmetadata.schema.entity.events.EventSubscription; +import org.openmetadata.schema.entity.events.EventSubscriptionOffset; import org.openmetadata.schema.entity.events.FilteringRules; import org.openmetadata.schema.entity.events.SubscriptionStatus; import org.openmetadata.schema.entity.services.ingestionPipelines.PipelineStatusType; @@ -49,6 +52,7 @@ import org.openmetadata.service.exception.CatalogExceptionMessage; import org.openmetadata.service.resources.CollectionRegistry; import org.openmetadata.service.search.models.IndexMapping; +import org.openmetadata.service.util.JsonUtils; import org.quartz.JobDataMap; import org.springframework.expression.Expression; @@ -260,4 +264,20 @@ public static T getValueFromQtzJobMap(JobDataMap dataMap, String key, Class< } return tClass.cast(value); } + + public static EventSubscriptionOffset getInitialAlertOffsetFromDb(UUID eventSubscriptionId) { + int eventSubscriptionOffset; + String json = + Entity.getCollectionDAO() + .eventSubscriptionDAO() + .getSubscriberExtension(eventSubscriptionId.toString(), OFFSET_EXTENSION); + if (json != null) { + EventSubscriptionOffset offsetFromDb = + JsonUtils.readValue(json, EventSubscriptionOffset.class); + eventSubscriptionOffset = offsetFromDb.getOffset(); + } else { + eventSubscriptionOffset = Entity.getCollectionDAO().changeEventDAO().listCount(); + } + return new EventSubscriptionOffset().withOffset(eventSubscriptionOffset); + } } diff --git a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java index 9e9ec8f29505..0b0513d667a0 100644 --- a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java +++ b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java @@ -101,7 +101,7 @@ void post_alertActionWithEnabledStateChange(TestInfo test) throws IOException { assertEquals(SubscriptionStatus.Status.ACTIVE, status2.getStatus()); // Ensure the call back notification has started - details = waitForFirstEvent(webhookName, 25); + details = waitForFirstEvent(alert.getId(), webhookName, 25); assertEquals(1, details.getEvents().size()); SubscriptionStatus successDetails = getStatus(alert.getId(), Response.Status.OK.getStatusCode()); @@ -176,6 +176,7 @@ void put_updateEndpointURL(TestInfo test) throws IOException { change); // Wait for webhook to be marked as failed + waitForAllEventToComplete(alert.getId()); Awaitility.await() .pollInterval(Duration.ofMillis(100L)) .atMost(Duration.ofMillis(100 * 100L)) @@ -192,7 +193,7 @@ private void waitForAllEventToComplete(UUID alertId) throws HttpResponseExceptio WebTarget target = getResource(String.format("%s/%s/processedEvents", collectionName, alertId.toString())); result = TestUtils.getWithResponse(target, Boolean.class, ADMIN_AUTH_HEADERS, 200); - LOG.debug("waitForAllEventToComplete alertId: {} , result: {}", alertId, result); + LOG.info("waitForAllEventToComplete alertId: {} , result: {}", alertId, result); try { Thread.sleep(3000L); } catch (InterruptedException e) { @@ -347,22 +348,22 @@ void testDifferentTypesOfAlerts() throws IOException { CreateEventSubscription w6ActionRequest = createRequest(alertName).withSubscriptionConfig(w6); EventSubscription w6Alert = createAndCheckEntity(w6ActionRequest, ADMIN_AUTH_HEADERS); - // Wait for events to complete - waitForAllEventToComplete(w1Alert.getId()); - waitForAllEventToComplete(w2Alert.getId()); - waitForAllEventToComplete(w3Alert.getId()); - waitForAllEventToComplete(w4Alert.getId()); - waitForAllEventToComplete(w5Alert.getId()); - waitForAllEventToComplete(w6Alert.getId()); - // Now check state of webhooks created - WebhookCallbackResource.EventDetails details = waitForFirstEvent("simulate-slowServer", 25); + WebhookCallbackResource.EventDetails details = + waitForFirstEvent(w1Alert.getId(), "simulate-slowServer", 25); ConcurrentLinkedQueue callbackEvents = details.getEvents(); assertNotNull(callbackEvents); assertNotNull(callbackEvents.peek()); waitAndCheckForEvents( - "*", "*", "*", "*", callbackEvents.peek().getTimestamp(), callbackEvents, 30); + w1Alert.getId(), + "*", + "*", + "*", + "*", + callbackEvents.peek().getTimestamp(), + callbackEvents, + 30); // Check all webhook status assertAlertStatusSuccessWithId(w1Alert.getId()); @@ -414,7 +415,14 @@ public void validateWebhookEvents() throws HttpResponseException { assertNotNull(callbackEvents); assertNotNull(callbackEvents.peek()); waitAndCheckForEvents( - "*", "*", "*", "*", callbackEvents.peek().getTimestamp(), callbackEvents, 40); + healthySub.getId(), + "*", + "*", + "*", + "*", + callbackEvents.peek().getTimestamp(), + callbackEvents, + 40); assertAlertStatusSuccessWithName("healthy"); } @@ -434,17 +442,20 @@ public void validateWebhookEntityEvents(String entity) throws HttpResponseExcept webhookCallbackResource.getEntityCallbackEvents(EventType.ENTITY_CREATED, entity); assertTrue(callbackEvents.size() > 0); long timestamp = callbackEvents.get(0).getTimestamp(); - waitAndCheckForEvents(entity, null, null, null, timestamp, callbackEvents, 50); + waitAndCheckForEvents( + createdSub.getId(), entity, null, null, null, timestamp, callbackEvents, 50); // For the entity all the webhooks registered for updated events have the right number of events callbackEvents = webhookCallbackResource.getEntityCallbackEvents(EventType.ENTITY_UPDATED, entity); // Use previous date if no update events timestamp = callbackEvents.size() > 0 ? callbackEvents.get(0).getTimestamp() : timestamp; - waitAndCheckForEvents(null, entity, null, null, timestamp, callbackEvents, 50); + waitAndCheckForEvents( + updatedSub.getId(), null, entity, null, null, timestamp, callbackEvents, 50); } public void waitAndCheckForEvents( + UUID alertId, String entityCreated, String entityUpdated, String entityRestored, @@ -453,6 +464,7 @@ public void waitAndCheckForEvents( Collection callbackEvents, int iteration) throws HttpResponseException { + waitForAllEventToComplete(alertId); List expected = getChangeEvents( entityCreated, @@ -629,7 +641,9 @@ public Webhook getWebhook(String uri) { return new Webhook().withEndpoint(URI.create(uri)).withSecretKey("webhookTest"); } - public WebhookCallbackResource.EventDetails waitForFirstEvent(String endpoint, int iteration) { + public WebhookCallbackResource.EventDetails waitForFirstEvent( + UUID alertId, String endpoint, int iteration) throws HttpResponseException { + waitForAllEventToComplete(alertId); Awaitility.await() .pollInterval(Duration.ofMillis(100L)) .atMost(Duration.ofMillis(iteration * 100L)) From 5f3f15fd7f388483e5bc19ecf3c6879b7ef2e15b Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Thu, 4 Jan 2024 01:10:42 +0530 Subject: [PATCH 17/28] Fix Error for non ChangeDescriptions --- .../formatter/decorators/EmailMessageDecorator.java | 12 +++++------- .../formatter/decorators/GChatMessageDecorator.java | 12 +++++------- .../decorators/MSTeamsMessageDecorator.java | 12 +++++------- .../formatter/decorators/SlackMessageDecorator.java | 12 +++++------- 4 files changed, 20 insertions(+), 28 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/EmailMessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/EmailMessageDecorator.java index d49db0b4307d..95543510cc57 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/EmailMessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/EmailMessageDecorator.java @@ -14,17 +14,16 @@ package org.openmetadata.service.formatter.decorators; import static org.openmetadata.service.events.subscription.AlertsRuleEvaluator.getEntity; -import static org.openmetadata.service.formatter.util.FormatterUtil.getFormattedMessages; import static org.openmetadata.service.util.EmailUtil.getSmtpSettings; import java.util.ArrayList; import java.util.List; -import java.util.Map; import org.openmetadata.schema.EntityInterface; +import org.openmetadata.schema.entity.feed.Thread; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.service.Entity; import org.openmetadata.service.apps.bundles.changeEvent.email.EmailMessage; -import org.openmetadata.service.resources.feeds.MessageParser; +import org.openmetadata.service.util.FeedUtils; public class EmailMessageDecorator implements MessageDecorator { @Override @@ -77,11 +76,10 @@ public EmailMessage buildMessage(ChangeEvent event) { emailMessage.setEntityUrl(this.buildEntityUrl(event.getEntityType(), entityInterface)); } } - Map messages = - getFormattedMessages(this, event.getChangeDescription(), entityInterface); + List thread = FeedUtils.getThreads(event, "admin"); List changeMessage = new ArrayList<>(); - for (Map.Entry entry : messages.entrySet()) { - changeMessage.add(entry.getValue()); + for (Thread entry : thread) { + changeMessage.add(entry.getMessage()); } emailMessage.setChangeMessage(changeMessage); return emailMessage; diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java index 75fb6047e44d..8e2d3c8626a8 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/GChatMessageDecorator.java @@ -14,16 +14,15 @@ package org.openmetadata.service.formatter.decorators; import static org.openmetadata.service.events.subscription.AlertsRuleEvaluator.getEntity; -import static org.openmetadata.service.formatter.util.FormatterUtil.getFormattedMessages; import static org.openmetadata.service.util.EmailUtil.getSmtpSettings; import java.util.ArrayList; import java.util.List; -import java.util.Map; import org.openmetadata.schema.EntityInterface; +import org.openmetadata.schema.entity.feed.Thread; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.service.apps.bundles.changeEvent.gchat.GChatMessage; -import org.openmetadata.service.resources.feeds.MessageParser; +import org.openmetadata.service.util.FeedUtils; public class GChatMessageDecorator implements MessageDecorator { @@ -93,12 +92,11 @@ public GChatMessage buildMessage(ChangeEvent event) { cardHeader.setTitle(cardHeaderText); card.setHeader(cardHeader); } - Map messages = - getFormattedMessages(this, event.getChangeDescription(), entityInterface); + List thread = FeedUtils.getThreads(event, "admin"); List widgets = new ArrayList<>(); - for (Map.Entry entry : messages.entrySet()) { + for (Thread entry : thread) { GChatMessage.Widget widget = new GChatMessage.Widget(); - widget.setTextParagraph(new GChatMessage.TextParagraph(entry.getValue())); + widget.setTextParagraph(new GChatMessage.TextParagraph(entry.getMessage())); widgets.add(widget); } section.setWidgets(widgets); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java index 4e8c3d292f4f..58adbf1926bb 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/MSTeamsMessageDecorator.java @@ -14,16 +14,15 @@ package org.openmetadata.service.formatter.decorators; import static org.openmetadata.service.events.subscription.AlertsRuleEvaluator.getEntity; -import static org.openmetadata.service.formatter.util.FormatterUtil.getFormattedMessages; import static org.openmetadata.service.util.EmailUtil.getSmtpSettings; import java.util.ArrayList; import java.util.List; -import java.util.Map; import org.openmetadata.schema.EntityInterface; +import org.openmetadata.schema.entity.feed.Thread; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.service.apps.bundles.changeEvent.msteams.TeamsMessage; -import org.openmetadata.service.resources.feeds.MessageParser; +import org.openmetadata.service.util.FeedUtils; public class MSTeamsMessageDecorator implements MessageDecorator { @@ -78,13 +77,12 @@ public TeamsMessage buildMessage(ChangeEvent event) { this.buildEntityUrl(event.getEntityType(), entityInterface)); teamsSections.setActivityTitle(headerText); } - Map messages = - getFormattedMessages(this, event.getChangeDescription(), entityInterface); + List thread = FeedUtils.getThreads(event, "admin"); List attachmentList = new ArrayList<>(); - for (Map.Entry entry : messages.entrySet()) { + for (Thread entry : thread) { TeamsMessage.Section section = new TeamsMessage.Section(); section.setActivityTitle(teamsSections.getActivityTitle()); - section.setActivityText(entry.getValue()); + section.setActivityText(entry.getMessage()); attachmentList.add(section); } teamsMessage.setSections(attachmentList); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java index 381b43105c3b..17deffc010d7 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/formatter/decorators/SlackMessageDecorator.java @@ -14,19 +14,18 @@ package org.openmetadata.service.formatter.decorators; import static org.openmetadata.service.events.subscription.AlertsRuleEvaluator.getEntity; -import static org.openmetadata.service.formatter.util.FormatterUtil.getFormattedMessages; import static org.openmetadata.service.util.EmailUtil.getSmtpSettings; import java.util.ArrayList; import java.util.List; -import java.util.Map; import org.openmetadata.schema.EntityInterface; +import org.openmetadata.schema.entity.feed.Thread; import org.openmetadata.schema.tests.TestCase; import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.service.Entity; import org.openmetadata.service.apps.bundles.changeEvent.slack.SlackAttachment; import org.openmetadata.service.apps.bundles.changeEvent.slack.SlackMessage; -import org.openmetadata.service.resources.feeds.MessageParser; +import org.openmetadata.service.util.FeedUtils; public class SlackMessageDecorator implements MessageDecorator { @@ -96,15 +95,14 @@ public SlackMessage buildMessage(ChangeEvent event) { } slackMessage.setText(headerText); } - Map messages = - getFormattedMessages(this, event.getChangeDescription(), entityInterface); + List thread = FeedUtils.getThreads(event, "admin"); List attachmentList = new ArrayList<>(); - for (Map.Entry entry : messages.entrySet()) { + for (Thread entry : thread) { SlackAttachment attachment = new SlackAttachment(); List mark = new ArrayList<>(); mark.add("text"); attachment.setMarkdownIn(mark); - attachment.setText(entry.getValue()); + attachment.setText(entry.getMessage()); attachmentList.add(attachment); } slackMessage.setAttachments(attachmentList.toArray(new SlackAttachment[0])); From d3543c99bfd3fddc895c7efcbdf88345a5096c3a Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Sat, 6 Jan 2024 15:49:56 +0530 Subject: [PATCH 18/28] Adds Observability On Top of Event Subscription --- .../subscription/AbstractAlertPublisher.java | 102 ------ .../events/subscription/AlertUtil.java | 163 ++++++--- .../subscription/AlertsRuleEvaluator.java | 60 ++++ .../EventsSubscriptionRegistry.java | 30 +- .../subscription/SubscriptionPublisher.java | 143 -------- .../EventSubscriptionResource.java | 79 +++- .../EntityObservabilityFilterDescriptor.json | 340 ++++++++++++++++++ .../events/api/createEventSubscription.json | 3 + .../json/schema/events/entitySpelFilters.json | 23 -- .../json/schema/events/eventFilterRule.json | 19 + .../json/schema/events/eventSubscription.json | 65 ++++ .../events/filterResourceDescriptor.json | 29 ++ 12 files changed, 744 insertions(+), 312 deletions(-) delete mode 100644 openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AbstractAlertPublisher.java delete mode 100644 openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/SubscriptionPublisher.java create mode 100644 openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json delete mode 100644 openmetadata-spec/src/main/resources/json/schema/events/entitySpelFilters.json create mode 100644 openmetadata-spec/src/main/resources/json/schema/events/filterResourceDescriptor.json diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AbstractAlertPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AbstractAlertPublisher.java deleted file mode 100644 index 22b55895f01a..000000000000 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AbstractAlertPublisher.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Copyright 2021 Collate - * Licensed 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.openmetadata.service.events.subscription; - -import java.util.ArrayList; -import java.util.List; -import lombok.extern.slf4j.Slf4j; -import org.openmetadata.schema.entity.events.EventSubscription; -import org.openmetadata.schema.type.ChangeEvent; -import org.openmetadata.service.events.EventPubSub; -import org.openmetadata.service.events.EventPublisher; -import org.openmetadata.service.events.errors.RetriableException; -import org.openmetadata.service.resources.events.EventResource.EventList; - -@Slf4j -public abstract class AbstractAlertPublisher implements EventPublisher { - // Backoff timeout in seconds. Delivering events is retried 5 times. - protected static final int BACKOFF_NORMAL = 0; - protected static final int BACKOFF_3_SECONDS = 3 * 1000; - protected static final int BACKOFF_30_SECONDS = 30 * 1000; - protected static final int BACKOFF_5_MINUTES = 5 * 60 * 1000; - protected static final int BACKOFF_1_HOUR = 60 * 60 * 1000; - protected static final int BACKOFF_24_HOUR = 24 * 60 * 60 * 1000; - protected int currentBackoffTime = BACKOFF_NORMAL; - protected final List batch = new ArrayList<>(); - - protected final EventSubscription eventSubscription; - private final int batchSize; - - protected AbstractAlertPublisher(EventSubscription eventSub) { - this.eventSubscription = eventSub; - this.batchSize = eventSub.getBatchSize(); - } - - @Override - public void onEvent( - EventPubSub.ChangeEventHolder changeEventHolder, long sequence, boolean endOfBatch) - throws Exception { - // Ignore events that don't match the webhook event filters - ChangeEvent changeEvent = changeEventHolder.getEvent(); - - // Evaluate Alert Trigger Config - if (!AlertUtil.shouldTriggerAlert( - changeEvent.getEntityType(), eventSubscription.getFilteringRules())) { - return; - } - - // Evaluate ChangeEvent Alert Filtering - if (eventSubscription.getFilteringRules() != null - && !AlertUtil.evaluateAlertConditions( - changeEvent, eventSubscription.getFilteringRules().getRules())) { - return; - } - - // Batch until either the batch has ended or batch size has reached the max size - batch.add(changeEventHolder.getEvent()); - if (!endOfBatch && batch.size() < batchSize) { - return; - } - - EventList list = new EventList(batch, null, null, batch.size()); - try { - publish(list); - batch.clear(); - } catch (RetriableException ex) { - setNextBackOff(); - LOG.error( - "Failed to publish event in batch {} due to {}, will try again in {} ms", - list, - ex, - currentBackoffTime); - Thread.sleep(currentBackoffTime); - } catch (Exception e) { - LOG.error("[AbstractAlertPublisher] error {}", e.getMessage(), e); - } - } - - public void setNextBackOff() { - if (currentBackoffTime == BACKOFF_NORMAL) { - currentBackoffTime = BACKOFF_3_SECONDS; - } else if (currentBackoffTime == BACKOFF_3_SECONDS) { - currentBackoffTime = BACKOFF_30_SECONDS; - } else if (currentBackoffTime == BACKOFF_30_SECONDS) { - currentBackoffTime = BACKOFF_5_MINUTES; - } else if (currentBackoffTime == BACKOFF_5_MINUTES) { - currentBackoffTime = BACKOFF_1_HOUR; - } else if (currentBackoffTime == BACKOFF_1_HOUR) { - currentBackoffTime = BACKOFF_24_HOUR; - } - } -} diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java index 5b736e5065d0..d33d2ed390eb 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java @@ -13,6 +13,8 @@ package org.openmetadata.service.events.subscription; +import static org.openmetadata.common.utils.CommonUtil.listOrEmpty; +import static org.openmetadata.common.utils.CommonUtil.nullOrEmpty; import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.ACTIVITY_FEED; import static org.openmetadata.service.Entity.TEAM; import static org.openmetadata.service.Entity.USER; @@ -26,14 +28,19 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; import java.util.stream.Stream; import javax.ws.rs.BadRequestException; import lombok.extern.slf4j.Slf4j; +import org.openmetadata.common.utils.CommonUtil; import org.openmetadata.schema.api.events.CreateEventSubscription; +import org.openmetadata.schema.api.events.Observability; +import org.openmetadata.schema.entity.events.Argument; import org.openmetadata.schema.entity.events.EventFilterRule; import org.openmetadata.schema.entity.events.EventSubscription; import org.openmetadata.schema.entity.events.EventSubscriptionOffset; import org.openmetadata.schema.entity.events.FilteringRules; +import org.openmetadata.schema.entity.events.ObservabilityFilters; import org.openmetadata.schema.entity.events.SubscriptionStatus; import org.openmetadata.schema.entity.services.ingestionPipelines.PipelineStatusType; import org.openmetadata.schema.tests.type.TestCaseStatus; @@ -53,7 +60,6 @@ import org.openmetadata.service.resources.CollectionRegistry; import org.openmetadata.service.search.models.IndexMapping; import org.openmetadata.service.util.JsonUtils; -import org.quartz.JobDataMap; import org.springframework.expression.Expression; @Slf4j @@ -62,29 +68,16 @@ private AlertUtil() {} public static AbstractEventConsumer getNotificationsPublisher(EventSubscription subscription) { validateSubscriptionConfig(subscription); - AbstractEventConsumer publisher; - switch (subscription.getSubscriptionType()) { - case SLACK_WEBHOOK: - publisher = new SlackEventPublisher(); - break; - case MS_TEAMS_WEBHOOK: - publisher = new MSTeamsPublisher(); - break; - case G_CHAT_WEBHOOK: - publisher = new GChatPublisher(); - break; - case GENERIC_WEBHOOK: - publisher = new GenericPublisher(); - break; - case EMAIL: - publisher = new EmailPublisher(); - break; - case ACTIVITY_FEED: - throw new IllegalArgumentException("Cannot create Activity Feed as Publisher."); - default: - throw new IllegalArgumentException("Invalid Alert Action Specified."); - } - return publisher; + return switch (subscription.getSubscriptionType()) { + case SLACK_WEBHOOK -> new SlackEventPublisher(); + case MS_TEAMS_WEBHOOK -> new MSTeamsPublisher(); + case G_CHAT_WEBHOOK -> new GChatPublisher(); + case GENERIC_WEBHOOK -> new GenericPublisher(); + case EMAIL -> new EmailPublisher(); + case ACTIVITY_FEED -> throw new IllegalArgumentException( + "Cannot create Activity Feed as Publisher."); + default -> throw new IllegalArgumentException("Invalid Alert Action Specified."); + }; } public static void validateSubscriptionConfig(EventSubscription eventSubscription) { @@ -202,12 +195,11 @@ public static String buildCompleteCondition(List alertFilterRul public static boolean shouldTriggerAlert(String entityType, FilteringRules config) { // OpenMetadataWide Setting apply to all ChangeEvents - if (config == null) { - return true; - } - if (config.getResources().size() == 1 && config.getResources().get(0).equals("all")) { + if (config == null + || (config.getResources().size() == 1 && config.getResources().get(0).equals("all"))) { return true; } + return config.getResources().contains(entityType); // Use Trigger Specific Settings } @@ -244,10 +236,15 @@ public static List getFilteredEvent( boolean triggerChangeEvent = AlertUtil.shouldTriggerAlert(event.getEntityType(), filteringRules); - // Evaluate ChangeEvent Alert Filtering - if (filteringRules != null - && !AlertUtil.evaluateAlertConditions(event, filteringRules.getRules())) { - triggerChangeEvent = false; + if (filteringRules != null) { + // Evaluate Rules + triggerChangeEvent = AlertUtil.evaluateAlertConditions(event, filteringRules.getRules()); + + if (triggerChangeEvent) { + // Evaluate Actions + triggerChangeEvent = + AlertUtil.evaluateAlertConditions(event, filteringRules.getActions()); + } } if (triggerChangeEvent) { @@ -257,14 +254,6 @@ public static List getFilteredEvent( return filteredEvents; } - public static T getValueFromQtzJobMap(JobDataMap dataMap, String key, Class tClass) { - Object value = dataMap.get(key); - if (value == null) { - return null; - } - return tClass.cast(value); - } - public static EventSubscriptionOffset getInitialAlertOffsetFromDb(UUID eventSubscriptionId) { int eventSubscriptionOffset; String json = @@ -280,4 +269,98 @@ public static EventSubscriptionOffset getInitialAlertOffsetFromDb(UUID eventSubs } return new EventSubscriptionOffset().withOffset(eventSubscriptionOffset); } + + public static FilteringRules validateAndBuildFilteringConditions( + CreateEventSubscription createEventSubscription) { + // Resource Validation + List finalRules = new ArrayList<>(); + List actions = new ArrayList<>(); + List resource = createEventSubscription.getFilteringRules().getResources(); + if (resource.size() > 1) { + throw new BadRequestException( + "Only one resource can be specified for Observability filtering"); + } + + // Build a Map of Entity Filter Name + Map supportedFilters = + EventsSubscriptionRegistry.getObservabilityDescriptor(resource.get(0)) + .getSupportedFilters() + .stream() + .collect( + Collectors.toMap(EventFilterRule::getName, eventFilterRule -> eventFilterRule)); + // Build a Map of Actions + Map supportedActions = + EventsSubscriptionRegistry.getObservabilityDescriptor(resource.get(0)) + .getSupportedActions() + .stream() + .collect( + Collectors.toMap(EventFilterRule::getName, eventFilterRule -> eventFilterRule)); + + // Input validation + if (createEventSubscription.getObservability() != null) { + Observability obscFilter = createEventSubscription.getObservability(); + listOrEmpty(obscFilter.getFilters()) + .forEach( + filter -> + finalRules.add( + getFilterRule( + supportedFilters, filter.getName(), buildInputArgumentsMap(filter)))); + listOrEmpty(obscFilter.getActions()) + .forEach( + action -> + actions.add( + getFilterRule( + supportedActions, action.getName(), buildInputArgumentsMap(action)))); + } + return new FilteringRules() + .withResources(createEventSubscription.getFilteringRules().getResources()) + .withRules(finalRules) + .withActions(actions); + } + + private static Map> buildInputArgumentsMap(ObservabilityFilters filter) { + return filter.getArguments().stream() + .collect(Collectors.toMap(Argument::getName, Argument::getInput)); + } + + private static EventFilterRule getFilterRule( + Map supportedFilters, + String name, + Map> inputArgMap) { + if (!supportedFilters.containsKey(name)) { + throw new BadRequestException("Give Resource doesn't support the filter " + name); + } + EventFilterRule rule = supportedFilters.get(name); + if (rule.getInputType().equals(EventFilterRule.InputType.NONE)) { + return rule; + } else { + String formulatedCondition = rule.getCondition(); + for (String argName : rule.getArguments()) { + List inputList = inputArgMap.get(argName); + if (nullOrEmpty(inputList)) { + throw new BadRequestException("Input for argument " + argName + " is missing"); + } + + formulatedCondition = + formulatedCondition.replace( + String.format("${%s}", argName), convertInputListToString(inputList)); + } + return rule.withCondition(formulatedCondition); + } + } + + private static String convertInputListToString(List valueList) { + if (CommonUtil.nullOrEmpty(valueList)) { + return ""; + } + + StringBuilder result = new StringBuilder(); + result.append("'").append(valueList.get(0)).append("'"); + + for (int i = 1; i < valueList.size(); i++) { + result.append(",'").append(valueList.get(i)).append("'"); + } + + return result.toString(); + } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java index 90ce8846dc61..1ab02b341340 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java @@ -196,6 +196,42 @@ public boolean matchTestResult(String... testResults) { return false; } + @Function( + name = "getTestCaseStatusIfInTestSuite", + input = "List of comma separated Test Suite", + description = + "Returns true if the change event entity being accessed has following entityId from the List.", + examples = {"matchTestResult('Success', 'Failed', 'Aborted')"}, + paramInputType = READ_FROM_PARAM_CONTEXT) + public boolean getTestCaseStatusIfInTestSuite(String... testResults) { + if (changeEvent == null || changeEvent.getChangeDescription() == null) { + return false; + } + if (!changeEvent.getEntityType().equals(TEST_CASE)) { + // in case the entity is not test case return since the filter doesn't apply + return true; + } + + // we need to handle both fields updated and fields added + List fieldChanges = changeEvent.getChangeDescription().getFieldsUpdated(); + if (!changeEvent.getChangeDescription().getFieldsAdded().isEmpty()) { + fieldChanges.addAll(changeEvent.getChangeDescription().getFieldsAdded()); + } + + for (FieldChange fieldChange : fieldChanges) { + if (fieldChange.getName().equals("testCaseResult") && fieldChange.getNewValue() != null) { + TestCaseResult testCaseResult = (TestCaseResult) fieldChange.getNewValue(); + TestCaseStatus status = testCaseResult.getTestCaseStatus(); + for (String givenStatus : testResults) { + if (givenStatus.equals(status.value())) { + return true; + } + } + } + } + return false; + } + @Function( name = "matchUpdatedBy", input = "List of comma separated user names that updated the entity", @@ -265,6 +301,30 @@ public boolean matchAnyFieldChange(String... fieldChangeUpdate) { return false; } + @Function( + name = "matchAnyDomain", + input = "List of comma separated Domains", + description = "Returns true if the change event entity belongs to a domain from the list", + examples = {"matchAnyDomain('domain1', 'domain2')"}, + paramInputType = NOT_REQUIRED) + public boolean matchAnyDomain(String... fieldChangeUpdate) { + if (changeEvent == null || changeEvent.getChangeDescription() == null) { + return false; + } + EntityInterface entity = getEntity(changeEvent); + EntityInterface entityWithDomainData = + Entity.getEntity( + changeEvent.getEntityType(), entity.getId(), "domain", Include.NON_DELETED); + if (entityWithDomainData.getDomain() != null) { + for (String name : fieldChangeUpdate) { + if (entityWithDomainData.getDomain().getFullyQualifiedName().equals(name)) { + return true; + } + } + } + return false; + } + public static EntityInterface getEntity(ChangeEvent event) { Class entityClass = Entity.getEntityClassFromType(event.getEntityType()); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/EventsSubscriptionRegistry.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/EventsSubscriptionRegistry.java index 9b3dfb897c9b..337e955905ef 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/EventsSubscriptionRegistry.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/EventsSubscriptionRegistry.java @@ -4,6 +4,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import org.openmetadata.schema.type.FilterResourceDescriptor; import org.openmetadata.schema.type.SubscriptionResourceDescriptor; import org.openmetadata.service.exception.CatalogExceptionMessage; @@ -11,18 +12,32 @@ public class EventsSubscriptionRegistry { private static final List SUB_RESOURCE_DESCRIPTORS = new ArrayList<>(); + private static final List ENTITY_OBSERVABILITY_DESCRIPTORS = + new ArrayList<>(); + private EventsSubscriptionRegistry() {} - public static void initialize(List resourceDescriptors) { + public static void initialize( + List resourceDescriptors, + List filterResourceDescriptors) { SUB_RESOURCE_DESCRIPTORS.clear(); SUB_RESOURCE_DESCRIPTORS.addAll(resourceDescriptors); SUB_RESOURCE_DESCRIPTORS.sort(Comparator.comparing(SubscriptionResourceDescriptor::getName)); + + // Observability descriptors + ENTITY_OBSERVABILITY_DESCRIPTORS.clear(); + ENTITY_OBSERVABILITY_DESCRIPTORS.addAll(filterResourceDescriptors); + ENTITY_OBSERVABILITY_DESCRIPTORS.sort(Comparator.comparing(FilterResourceDescriptor::getName)); } public static List listResourceDescriptors() { return Collections.unmodifiableList(SUB_RESOURCE_DESCRIPTORS); } + public static List listObservabilityDescriptors() { + return Collections.unmodifiableList(ENTITY_OBSERVABILITY_DESCRIPTORS); + } + public static SubscriptionResourceDescriptor getResourceDescriptor(String resourceType) { SubscriptionResourceDescriptor rd = SUB_RESOURCE_DESCRIPTORS.stream() @@ -35,4 +50,17 @@ public static SubscriptionResourceDescriptor getResourceDescriptor(String resour } return rd; } + + public static FilterResourceDescriptor getObservabilityDescriptor(String resourceType) { + FilterResourceDescriptor rd = + ENTITY_OBSERVABILITY_DESCRIPTORS.stream() + .filter(r -> r.getName().equalsIgnoreCase(resourceType)) + .findAny() + .orElse(null); + if (rd == null) { + throw new IllegalArgumentException( + CatalogExceptionMessage.resourceTypeNotFound(resourceType)); + } + return rd; + } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/SubscriptionPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/SubscriptionPublisher.java deleted file mode 100644 index ebd3933f5792..000000000000 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/SubscriptionPublisher.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * Copyright 2021 Collate - * Licensed 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.openmetadata.service.events.subscription; - -import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.ACTIVE; -import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.AWAITING_RETRY; -import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.FAILED; - -import com.lmax.disruptor.BatchEventProcessor; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import lombok.Getter; -import lombok.SneakyThrows; -import lombok.extern.slf4j.Slf4j; -import org.openmetadata.schema.entity.events.EventSubscription; -import org.openmetadata.schema.entity.events.SubscriptionStatus; -import org.openmetadata.service.events.EventPubSub; -import org.openmetadata.service.events.errors.EventPublisherException; -import org.openmetadata.service.resources.events.EventResource; - -/** - * SubscriptionPublisher publishes events to the alert endpoint using POST http requests/ Email. There is one instance - * of SubscriptionPublisher per alert subscription. Each SubscriptionPublisher is an EventHandler that runs in a - * separate thread and receives events from LMAX Disruptor {@link EventPubSub} through {@link BatchEventProcessor}. - * - *

The failures during callback to Alert are handled in this class as follows: - * - *

    - *
  • Alerts with unresolvable URLs are marked as "failed" and no further attempt is made to deliver the events - *
  • Alerts callbacks that return 3xx are marked as "failed" and no further attempt is made to deliver the events - *
  • Alerts callbacks that return 4xx, 5xx, or timeout are marked as "awaitingRetry" and 5 retry attempts are made - * to deliver the events with the following backoff - 3 seconds, 30 seconds, 5 minutes, 1 hours, and 24 hour. When - * all the 5 delivery attempts fail, the alerts state is marked as "retryLimitReached" and no further attempt is - * made to deliver the events. - *
- */ -@Slf4j -public class SubscriptionPublisher extends AbstractAlertPublisher { - private final CountDownLatch shutdownLatch = new CountDownLatch(1); - @Getter private BatchEventProcessor processor; - - public SubscriptionPublisher(EventSubscription eventSub) { - super(eventSub); - } - - @SneakyThrows - @Override - public void onStart() { - setSuccessStatus(System.currentTimeMillis()); - onStartDelegate(); - LOG.info("Alert-lifecycle-onStart {}", eventSubscription.getName()); - } - - @Override - public void onShutdown() { - currentBackoffTime = BACKOFF_NORMAL; - shutdownLatch.countDown(); - onShutdownDelegate(); - LOG.info("Alert-lifecycle-onShutdown {}", eventSubscription.getName()); - } - - public synchronized EventSubscription getEventSubscription() { - return eventSubscription; - } - - public synchronized void setErrorStatus(Long attemptTime, Integer statusCode, String reason) { - setStatus(FAILED, attemptTime, statusCode, reason, null); - } - - public synchronized void setAwaitingRetry(Long attemptTime, int statusCode, String reason) { - setStatus(AWAITING_RETRY, attemptTime, statusCode, reason, attemptTime + currentBackoffTime); - } - - public synchronized void setSuccessStatus(Long updateTime) { - SubscriptionStatus subStatus = - AlertUtil.buildSubscriptionStatus( - ACTIVE, updateTime, null, null, null, updateTime, updateTime); - eventSubscription.setStatusDetails(subStatus); - } - - protected synchronized void setStatus( - SubscriptionStatus.Status status, - Long attemptTime, - Integer statusCode, - String reason, - Long timestamp) { - SubscriptionStatus subStatus = - AlertUtil.buildSubscriptionStatus( - status, null, attemptTime, statusCode, reason, timestamp, attemptTime); - eventSubscription.setStatusDetails(subStatus); - } - - public void awaitShutdown() throws InterruptedException { - LOG.info("Awaiting shutdown alertActionPublisher-lifecycle {}", eventSubscription.getName()); - shutdownLatch.await(5, TimeUnit.SECONDS); - } - - public void setProcessor(BatchEventProcessor processor) { - this.processor = processor; - } - - protected void sendAlert(EventResource.EventList list) { - /* This method needs to be over-ridden by specific Publisher for sending Alert */ - } - - protected void onStartDelegate() { - /* Called on start of Publisher */ - } - - protected void onShutdownDelegate() { - /* Called on shutdown of Publisher */ - } - - public int getCurrentBackOff() { - return currentBackoffTime; - } - - @Override - public void publish(EventResource.EventList list) throws EventPublisherException { - // Publish to the given Alert Actions - try { - LOG.info( - "Sending Alert {}:{}:{}", - eventSubscription.getName(), - eventSubscription.getStatusDetails().getStatus(), - batch.size()); - sendAlert(list); - } catch (Exception ex) { - LOG.warn("Invalid Exception in Alert {}", eventSubscription.getName()); - } - } -} diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java b/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java index 03021e4530e1..09a1c0696184 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/resources/events/subscription/EventSubscriptionResource.java @@ -15,6 +15,7 @@ import static org.openmetadata.common.utils.CommonUtil.listOrEmpty; import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.ACTIVITY_FEED; +import static org.openmetadata.service.events.subscription.AlertUtil.validateAndBuildFilteringConditions; import io.swagger.v3.oas.annotations.ExternalDocumentation; import io.swagger.v3.oas.annotations.Operation; @@ -56,6 +57,7 @@ import org.openmetadata.schema.entity.events.EventSubscription; import org.openmetadata.schema.entity.events.SubscriptionStatus; import org.openmetadata.schema.type.EntityHistory; +import org.openmetadata.schema.type.FilterResourceDescriptor; import org.openmetadata.schema.type.Function; import org.openmetadata.schema.type.MetadataOperation; import org.openmetadata.schema.type.SubscriptionResourceDescriptor; @@ -113,7 +115,8 @@ public void initialize(OpenMetadataApplicationConfig config) { try { repository.initSeedDataFromResources(); EventsSubscriptionRegistry.initialize( - listOrEmpty(EventSubscriptionResource.getDescriptors())); + listOrEmpty(EventSubscriptionResource.getDescriptors()), + listOrEmpty(EventSubscriptionResource.getFilterDescriptors())); initializeEventSubscriptions(); } catch (Exception ex) { // Starting application should not fail @@ -279,6 +282,34 @@ public Response createEventSubscription( return response; } + @POST + @Path("/observability") + @Operation( + operationId = "createObservabilitySubscription", + summary = "Create a new Observability Subscription", + description = "Create a new Observability Subscription", + responses = { + @ApiResponse( + responseCode = "200", + description = "Observability Subscription Created", + content = + @Content( + mediaType = "application/json", + schema = @Schema(implementation = CreateEventSubscription.class))), + @ApiResponse(responseCode = "400", description = "Bad request") + }) + public Response createObservabilitySubscription( + @Context UriInfo uriInfo, + @Context SecurityContext securityContext, + @Valid CreateEventSubscription request) + throws SchedulerException { + EventSubscription eventSub = + getObservabilitySubscription(request, securityContext.getUserPrincipal().getName()); + Response response = create(uriInfo, securityContext, eventSub); + EventSubscriptionScheduler.getInstance().addSubscriptionPublisher(eventSub); + return response; + } + @PUT @Operation( operationId = "createOrUpdateEventSubscription", @@ -553,6 +584,19 @@ public ResultList listEventSubResources( return new ResultList<>(EventsSubscriptionRegistry.listResourceDescriptors()); } + @GET + @Path("/observability/resources") + @Operation( + operationId = "listDataObservabilityResources", + summary = "Get list of Data Observability Resources used in filtering Event Subscription", + description = + "Get list of EventSubscription functions used in filtering conditions in Event Subscription") + public ResultList listObservabilityResources( + @Context UriInfo uriInfo, @Context SecurityContext securityContext) { + authorizer.authorizeAdmin(securityContext); + return new ResultList<>(EventsSubscriptionRegistry.listObservabilityDescriptors()); + } + @GET @Path("/validation/condition/{expression}") @Operation( @@ -589,9 +633,38 @@ public EventSubscription getEventSubscription(CreateEventSubscription create, St .withPollInterval(create.getPollInterval()); } + public EventSubscription getObservabilitySubscription( + CreateEventSubscription create, String user) { + return repository + .copy(new EventSubscription(), create, user) + .withAlertType(create.getAlertType()) + .withTrigger(create.getTrigger()) + .withEnabled(create.getEnabled()) + .withBatchSize(create.getBatchSize()) + .withTimeout(create.getTimeout()) + .withFilteringRules(validateAndBuildFilteringConditions(create)) + .withSubscriptionType(create.getSubscriptionType()) + .withSubscriptionConfig(create.getSubscriptionConfig()) + .withProvider(create.getProvider()) + .withRetries(create.getRetries()) + .withPollInterval(create.getPollInterval()) + .withObservability(create.getObservability()); + } + public static List getDescriptors() throws IOException { + return getDescriptorsFromFile( + "EventSubResourceDescriptor.json", SubscriptionResourceDescriptor.class); + } + + public static List getFilterDescriptors() throws IOException { + return getDescriptorsFromFile( + "EntityObservabilityFilterDescriptor.json", FilterResourceDescriptor.class); + } + + public static List getDescriptorsFromFile(String fileName, Class classType) + throws IOException { List jsonDataFiles = - EntityUtil.getJsonDataResources(".*json/data/EventSubResourceDescriptor.json$"); + EntityUtil.getJsonDataResources(String.format(".*json/data/%s$", fileName)); if (jsonDataFiles.size() != 1) { LOG.warn("Invalid number of jsonDataFiles {}. Only one expected.", jsonDataFiles.size()); return Collections.emptyList(); @@ -601,7 +674,7 @@ public static List getDescriptors() throws IOExc String json = CommonUtil.getResourceAsStream( EventSubscriptionResource.class.getClassLoader(), jsonDataFile); - return JsonUtils.readObjects(json, SubscriptionResourceDescriptor.class); + return JsonUtils.readObjects(json, classType); } catch (Exception e) { LOG.warn( "Failed to initialize the events subscription resource descriptors from file {}", diff --git a/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json b/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json new file mode 100644 index 000000000000..6b9db2e0ce4e --- /dev/null +++ b/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json @@ -0,0 +1,340 @@ +[ + { + "name" : "table", + "supportedFilters" : [ + { + "name": "filterByFqn", + "fullyQualifiedName": "eventSubscription.filterByFqn", + "displayName": "Filter By Table Name", + "description" : "Filter A Change Event By Table Name", + "effect" : "include", + "condition": "matchAnyEntityFqn(${fqnList})", + "arguments": [ + "fqnList" + ], + "inputType": "runtime" + }, + { + "name": "filterByDomain", + "fullyQualifiedName": "eventSubscription.filterByDomain", + "displayName": "Filter By Domain", + "description" : "Filter An Change Event By Domain a Entity Belongs To", + "effect" : "include", + "condition": "matchAnyDomain(${domainList})", + "arguments": [ + "domainList" + ], + "inputType": "runtime" + } + ], + "supportedActions" : [ + { + "name": "GetSchemaChanges", + "fullyQualifiedName": "eventSubscription.filterBySchemaChange", + "displayName": "Get Schema Changes", + "description" : "Get Updates for Schema Changes", + "effect" : "include", + "condition": "matchAnyFieldChange('columns', 'description')", + "inputType": "none" + }, + { + "name": "GetTableMetricsUpdates", + "fullyQualifiedName": "eventSubscription.GetTableMetricsUpdates", + "displayName": "Get Table Metrics Updates", + "description" : "Get Updates About Table Metrics", + "effect" : "include", + "condition": "matchAnyFieldChange('customMetrics', 'profile')", + "inputType": "none" + } + ] + }, + { + "name" : "topic", + "supportedFilters" : [ + { + "name": "filterByFqn", + "fullyQualifiedName": "eventSubscription.filterByFqn", + "displayName": "Filter By Table Name", + "description" : "Filter A Change Event By Topic Name", + "effect" : "include", + "condition": "matchAnyEntityFqn(${fqnList})", + "arguments": [ + "fqnList" + ], + "inputType": "runtime" + }, + { + "name": "filterByDomain", + "fullyQualifiedName": "eventSubscription.filterByDomain", + "displayName": "Filter By Domain", + "description" : "Filter An Change Event By Domain a Entity Belongs To", + "effect" : "include", + "condition": "matchAnyDomain(${domainList})", + "arguments": [ + "domainList" + ], + "inputType": "runtime" + } + ], + "supportedActions" : [ + { + "name": "GetSchemaChanges", + "fullyQualifiedName": "eventSubscription.filterBySchemaChange", + "displayName": "Get Schema Changes", + "description" : "Get Updates for Schema Changes", + "effect" : "include", + "condition": "matchAnyFieldChange(${fieldList})", + "arguments": [ + "fieldList" + ], + "inputType": "none" + } + ] + }, + { + "name" : "container", + "supportedFilters" : [ + { + "name": "filterByFqn", + "fullyQualifiedName": "eventSubscription.filterByFqn", + "displayName": "Filter By Table Name", + "description" : "Filter A Change Event By Container Name", + "effect" : "include", + "condition": "matchAnyEntityFqn(${fqnList})", + "arguments": [ + "fqnList" + ], + "inputType": "runtime" + }, + { + "name": "filterByDomain", + "fullyQualifiedName": "eventSubscription.filterByDomain", + "displayName": "Filter By Domain", + "description" : "Filter An Change Event By Domain a Entity Belongs To", + "effect" : "include", + "condition": "matchAnyDomain(${domainList})", + "arguments": [ + "domainList" + ], + "inputType": "runtime" + } + ], + "supportedActions" : [ + { + "name": "GetSchemaChanges", + "fullyQualifiedName": "eventSubscription.filterBySchemaChange", + "displayName": "Get Schema Changes", + "description" : "Get Updates for Schema Changes", + "effect" : "include", + "condition": "matchAnyFieldChange(${fieldList})", + "arguments": [ + "fieldList" + ], + "inputType": "none" + } + ] + }, + { + "name" : "pipeline", + "supportedFilters" : [ + { + "name": "filterByFqn", + "fullyQualifiedName": "eventSubscription.filterByFqn", + "displayName": "Filter By Table Name", + "description" : "Filter A Change Event By Pipeline Name", + "effect" : "include", + "condition": "matchAnyEntityFqn(${fqnList})", + "arguments": [ + "fqnList" + ], + "inputType": "runtime" + }, + { + "name": "filterByDomain", + "fullyQualifiedName": "eventSubscription.filterByDomain", + "displayName": "Filter By Domain", + "description" : "Filter An Change Event By Domain a Entity Belongs To", + "effect" : "include", + "condition": "matchAnyDomain(${domainList})", + "arguments": [ + "domainList" + ], + "inputType": "runtime" + } + ], + "supportedActions" : [ + { + "name": "GetSchemaChanges", + "fullyQualifiedName": "eventSubscription.filterBySchemaChange", + "displayName": "Get Schema Changes", + "description" : "Get Updates for Schema Changes", + "effect" : "include", + "condition": "matchAnyFieldChange(${fieldList})", + "arguments": [ + "fieldList" + ], + "inputType": "none" + }, + { + "name": "GetPipelineStatusUpdates", + "fullyQualifiedName": "eventSubscription.GetPipelineStatusUpdates", + "displayName": "Get Pipeline Status Updates", + "description" : "Get Updates for Pipeline Status Changes", + "effect" : "include", + "condition": "matchIngestionPipelineState(${pipelineStateList})", + "arguments": [ + "pipelineStateList" + ], + "inputType": "none" + } + ] + }, + { + "name" : "testCase", + "supportedFilters" : [ + { + "name": "filterByFqn", + "fullyQualifiedName": "eventSubscription.filterByFqn", + "displayName": "Filter By Test Case Name", + "description" : "Filter A Change Event By Test Case Name", + "effect" : "include", + "condition": "matchAnyEntityFqn(${fqnList})", + "arguments": [ + "fqnList" + ], + "inputType": "runtime" + }, + { + "name": "filterByDomain", + "fullyQualifiedName": "eventSubscription.filterByDomain", + "displayName": "Filter By Domain", + "description" : "Filter An Change Event By Domain a Entity Belongs To", + "effect" : "include", + "condition": "matchAnyDomain(${domainList})", + "arguments": [ + "domainList" + ], + "inputType": "runtime" + }, + { + "name": "filterByTableNameTestCaseBelongsTo", + "fullyQualifiedName": "eventSubscription.filterByTableNameTestCaseBelongsTo", + "displayName": "Filter By Table Name", + "description" : "Filter A Change Event By Pipeline Name", + "effect" : "include", + "condition": "matchAnyFieldChange(${fieldList})", + "arguments": [ + "fieldList" + ], + "inputType": "runtime" + } + ], + "supportedActions" : [ + { + "name": "GetSchemaChanges", + "fullyQualifiedName": "eventSubscription.filterBySchemaChange", + "displayName": "Get Schema Changes", + "description" : "Get Updates for Schema Changes", + "effect" : "include", + "condition": "matchAnyFieldChange(${fieldList})", + "arguments": [ + "fieldList" + ], + "inputType": "none" + }, + { + "name": "GetTestCaseStatusUpdates", + "fullyQualifiedName": "eventSubscription.GetTestCaseStatusUpdates", + "displayName": "Get Test Case Status Updates", + "description" : "Get Status Updates Test Cases", + "effect" : "include", + "condition": "matchTestResult(${testResultList})", + "arguments": [ + "testResultList" + ], + "inputType": "runtime" + } + ] + }, + { + "name" : "testSuite", + "supportedFilters" : [ + { + "name": "filterByFqn", + "fullyQualifiedName": "eventSubscription.filterByFqn", + "displayName": "Filter By Test Suite Name", + "description" : "Filter By Test Suite Name", + "effect" : "include", + "condition": "matchAnyEntityFqn(${fqnList})", + "arguments": [ + "fqnList" + ], + "inputType": "runtime" + }, + { + "name": "filterByTableNameTestCaseBelongsTo", + "fullyQualifiedName": "eventSubscription.filterByTableNameTestCaseBelongsTo", + "displayName": "Filter By Table Name For A Test Suite", + "description" : "Filter A Change Event By Pipeline Name", + "effect" : "include", + "condition": "matchByTableNameTestCaseBelongsTo(${tableNameList})", + "arguments": [ + "tableNameList" + ], + "inputType": "runtime" + }, + { + "name": "filterByDomain", + "fullyQualifiedName": "eventSubscription.filterByDomain", + "displayName": "Filter By Domain", + "description" : "Filter An Change Event By Domain a Entity Belongs To", + "effect" : "include", + "condition": "matchAnyDomain(${domainList})", + "arguments": [ + "domainList" + ], + "inputType": "runtime" + } + ], + "supportedActions" : [ + { + "name": "GetSchemaChanges", + "fullyQualifiedName": "eventSubscription.filterBySchemaChange", + "displayName": "Get Schema Changes", + "description" : "Get Updates for Schema Changes", + "effect" : "include", + "condition": "matchAnyFieldChange(${fieldList})", + "arguments": [ + "fieldList" + ], + "inputType": "none" + }, + { + "name": "GetTestCaseStatusUpdatesUnderSuite", + "fullyQualifiedName": "eventSubscription.GetTestCaseStatusUpdatesUnderSuite", + "displayName": "Get Test Case Status Updates belonging to a Test Suite", + "description" : "Get Status Updates Test Cases belonging to a Test Suite", + "effect" : "include", + "condition": "getTestCaseStatusIfInTestSuite(${testSuiteList}, ${testStatusList})", + "arguments": [ + "testSuiteList", + "testStatusList" + ], + "inputType": "runtime" + }, + { + "name": "GetTestCaseAdditionOrDeletionToSuite", + "fullyQualifiedName": "eventSubscription.GetTestCaseAdditionOrDeletionToSuite", + "displayName": "Get Test Case Addition Or Deletion To Suite Updates", + "description" : "Get Status Updates when a Test Case is added or deleted to a Test Suite", + "effect" : "include", + "condition": "getTestCaseAdditionOrDeletionToSuite(${testSuiteList}, ${opList})", + "arguments": [ + "testSuiteList", + "opList" + ], + "inputType": "runtime" + } + ] + } +] \ No newline at end of file diff --git a/openmetadata-spec/src/main/resources/json/schema/events/api/createEventSubscription.json b/openmetadata-spec/src/main/resources/json/schema/events/api/createEventSubscription.json index df3ff13705a3..a461fbd21304 100644 --- a/openmetadata-spec/src/main/resources/json/schema/events/api/createEventSubscription.json +++ b/openmetadata-spec/src/main/resources/json/schema/events/api/createEventSubscription.json @@ -73,6 +73,9 @@ "description": "Poll Interval in seconds.", "type": "integer", "default": 10 + }, + "observability": { + "$ref": "../eventSubscription.json#/definitions/observability" } }, "required": ["name", "alertType", "subscriptionType"], diff --git a/openmetadata-spec/src/main/resources/json/schema/events/entitySpelFilters.json b/openmetadata-spec/src/main/resources/json/schema/events/entitySpelFilters.json deleted file mode 100644 index 262d1b4ed943..000000000000 --- a/openmetadata-spec/src/main/resources/json/schema/events/entitySpelFilters.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "$id": "https://open-metadata.org/schema/alerts/entitySpelFilters.json", - "$schema": "http://json-schema.org/draft-07/schema#", - "title": "EntitySpelFilters", - "description": "Entity Spel Filters.", - "type": "object", - "javaType": "org.openmetadata.schema.entity.events.EntitySpelFilters", - "properties": { - "entityType" : { - "description": "EntityType", - "type": "string" - }, - "supportedFunctions" : { - "description": "List of Supported functions", - "type": "array", - "items": { - "type": "string" - } - } - }, - "required": ["entityType", "supportedFunctions"], - "additionalProperties": false -} diff --git a/openmetadata-spec/src/main/resources/json/schema/events/eventFilterRule.json b/openmetadata-spec/src/main/resources/json/schema/events/eventFilterRule.json index 2c06ad8a90e4..2137f9380cd9 100644 --- a/openmetadata-spec/src/main/resources/json/schema/events/eventFilterRule.json +++ b/openmetadata-spec/src/main/resources/json/schema/events/eventFilterRule.json @@ -10,6 +10,10 @@ "description": "Name of this Event Filter.", "type": "string" }, + "displayName": { + "description": "Display Name of the Filter.", + "type": "string" + }, "fullyQualifiedName": { "description": "FullyQualifiedName in the form `eventSubscription.eventFilterRuleName`.", "$ref": "../type/basic.json#/definitions/fullyQualifiedEntityName" @@ -28,6 +32,21 @@ "condition" : { "description": "Expression in SpEL used for matching of a `Rule` based on entity, resource, and environmental attributes.", "$ref" : "../type/basic.json#/definitions/expression" + }, + "arguments": { + "description": "Arguments to the Condition.", + "type": "array", + "items": { + "type": "string" + } + }, + "inputType": { + "type": "string", + "enum": [ + "static", + "runtime", + "none" + ] } }, "required": ["effect", "condition"], diff --git a/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json b/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json index 99c155a1631f..91514e1cd425 100644 --- a/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json +++ b/openmetadata-spec/src/main/resources/json/schema/events/eventSubscription.json @@ -7,6 +7,61 @@ "javaType": "org.openmetadata.schema.entity.events.EventSubscription", "javaInterfaces": ["org.openmetadata.schema.EntityInterface"], "definitions": { + "observabilityFilters": { + "description": "Observability Filters for Event Subscription.", + "type": "object", + "javaType": "org.openmetadata.schema.entity.events.ObservabilityFilters", + "properties": { + "name": { + "description": "Name of the filter", + "type": "string" + }, + "arguments": { + "description": "Arguments List", + "type": "array", + "items": { + "description": "Argument for the filter.", + "type": "object", + "properties": { + "name": { + "description": "Name of the Argument", + "type": "string" + }, + "input": { + "description": "Value of the Argument", + "type": "array", + "items": { + "type": "string" + } + } + }, + "additionalProperties": false + } + } + }, + "additionalProperties": false + }, + "observability": { + "description": "Observability of the event subscription.", + "type": "object", + "properties": { + "filters": { + "description": "List of filters for the event subscription.", + "type": "array", + "items": { + "$ref": "#/definitions/observabilityFilters" + } + }, + "actions": { + "description": "List of filters for the event subscription.", + "type": "array", + "items": { + "$ref": "#/definitions/observabilityFilters" + } + } + }, + "additionalProperties": false + }, "triggerType": { "description": "Trigger Configuration for Alerts.", "type": "string", @@ -126,6 +181,13 @@ "items": { "$ref": "./eventFilterRule.json" } + }, + "actions": { + "description": "A set of filter rules associated with the Alert.", + "type": "array", + "items": { + "$ref": "./eventFilterRule.json" + } } }, "required": ["resources"], @@ -253,6 +315,9 @@ "description": "Poll Interval in seconds.", "type": "integer", "default": 10 + }, + "observability": { + "$ref": "#/definitions/observability" } }, "required": ["id", "name", "alertType", "subscriptionType"], diff --git a/openmetadata-spec/src/main/resources/json/schema/events/filterResourceDescriptor.json b/openmetadata-spec/src/main/resources/json/schema/events/filterResourceDescriptor.json new file mode 100644 index 000000000000..0440c5cc0ec5 --- /dev/null +++ b/openmetadata-spec/src/main/resources/json/schema/events/filterResourceDescriptor.json @@ -0,0 +1,29 @@ +{ + "$id": "https://open-metadata.org/schema/events/filterResourceDescriptor.json", + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "FilterResourceDescriptor", + "description": "Filter descriptor", + "type": "object", + "javaType": "org.openmetadata.schema.type.FilterResourceDescriptor", + "properties": { + "name": { + "description": "Name of the resource. For entity related resources, resource name is same as the entity name. Some resources such as lineage are not entities but are resources.", + "type": "string" + }, + "supportedFilters": { + "description": "List of operations supported filters by the resource.", + "type": "array", + "items": { + "$ref": "./eventFilterRule.json" + } + }, + "supportedActions": { + "description": "List of actions supported filters by the resource.", + "type": "array", + "items": { + "$ref": "./eventFilterRule.json" + } + } + }, + "additionalProperties": false +} From 2b74ab8c0cc3aae35b92c1fdd12a564e20c9637d Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Sat, 6 Jan 2024 16:22:56 +0530 Subject: [PATCH 19/28] Make List Separate --- .../events/subscription/AlertUtil.java | 3 ++- .../subscription/AlertsRuleEvaluator.java | 25 ++++++++++--------- 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java index d33d2ed390eb..9c7f303c380c 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java @@ -343,7 +343,8 @@ private static EventFilterRule getFilterRule( formulatedCondition = formulatedCondition.replace( - String.format("${%s}", argName), convertInputListToString(inputList)); + String.format("${%s}", argName), + String.format("{%s}", convertInputListToString(inputList))); } return rule.withCondition(formulatedCondition); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java index 1ab02b341340..5dbe4d8fe932 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java @@ -46,7 +46,7 @@ public AlertsRuleEvaluator(ChangeEvent event) { "Returns true if the change event entity being accessed has source as mentioned in condition", examples = {"matchAnySource('bot', 'user')"}, paramInputType = READ_FROM_PARAM_CONTEXT) - public boolean matchAnySource(String... originEntities) { + public boolean matchAnySource(List originEntities) { if (changeEvent == null || changeEvent.getEntityType() == null) { return false; } @@ -66,7 +66,7 @@ public boolean matchAnySource(String... originEntities) { "Returns true if the change event entity being accessed has following owners from the List.", examples = {"matchAnyOwnerName('Owner1', 'Owner2')"}, paramInputType = SPECIFIC_INDEX_ELASTIC_SEARCH) - public boolean matchAnyOwnerName(String... ownerNameList) { + public boolean matchAnyOwnerName(List ownerNameList) { if (changeEvent == null || changeEvent.getEntity() == null) { return false; } @@ -99,7 +99,7 @@ public boolean matchAnyOwnerName(String... ownerNameList) { "Returns true if the change event entity being accessed has following entityName from the List.", examples = {"matchAnyEntityFqn('Name1', 'Name')"}, paramInputType = ALL_INDEX_ELASTIC_SEARCH) - public boolean matchAnyEntityFqn(String... entityNames) { + public boolean matchAnyEntityFqn(List entityNames) { if (changeEvent == null || changeEvent.getEntity() == null) { return false; } @@ -125,7 +125,7 @@ public boolean matchAnyEntityFqn(String... entityNames) { "Returns true if the change event entity being accessed has following entityId from the List.", examples = {"matchAnyEntityId('uuid1', 'uuid2')"}, paramInputType = ALL_INDEX_ELASTIC_SEARCH) - public boolean matchAnyEntityId(String... entityIds) { + public boolean matchAnyEntityId(List entityIds) { if (changeEvent == null || changeEvent.getEntity() == null) { return false; } @@ -147,7 +147,7 @@ public boolean matchAnyEntityId(String... entityIds) { "matchAnyEventType('entityCreated', 'entityUpdated', 'entityDeleted', 'entitySoftDeleted')" }, paramInputType = READ_FROM_PARAM_CONTEXT) - public boolean matchAnyEventType(String... eventTypesList) { + public boolean matchAnyEventType(List eventTypesList) { if (changeEvent == null || changeEvent.getEventType() == null) { return false; } @@ -167,7 +167,7 @@ public boolean matchAnyEventType(String... eventTypesList) { "Returns true if the change event entity being accessed has following entityId from the List.", examples = {"matchTestResult('Success', 'Failed', 'Aborted')"}, paramInputType = READ_FROM_PARAM_CONTEXT) - public boolean matchTestResult(String... testResults) { + public boolean matchTestResult(List testResults) { if (changeEvent == null || changeEvent.getChangeDescription() == null) { return false; } @@ -201,9 +201,10 @@ public boolean matchTestResult(String... testResults) { input = "List of comma separated Test Suite", description = "Returns true if the change event entity being accessed has following entityId from the List.", - examples = {"matchTestResult('Success', 'Failed', 'Aborted')"}, + examples = {"getTestCaseStatusIfInTestSuite('Success', 'Failed', 'Aborted')"}, paramInputType = READ_FROM_PARAM_CONTEXT) - public boolean getTestCaseStatusIfInTestSuite(String... testResults) { + public boolean getTestCaseStatusIfInTestSuite( + List testSuiteList, List testResults) { if (changeEvent == null || changeEvent.getChangeDescription() == null) { return false; } @@ -238,7 +239,7 @@ public boolean getTestCaseStatusIfInTestSuite(String... testResults) { description = "Returns true if the change event entity is updated by the mentioned users", examples = {"matchUpdatedBy('user1', 'user2')"}, paramInputType = READ_FROM_PARAM_CONTEXT) - public boolean matchUpdatedBy(String... updatedByUserList) { + public boolean matchUpdatedBy(List updatedByUserList) { if (changeEvent == null || changeEvent.getUserName() == null) { return false; } @@ -260,7 +261,7 @@ public boolean matchUpdatedBy(String... updatedByUserList) { "matchIngestionPipelineState('queued', 'success', 'failed', 'running', 'partialSuccess')" }, paramInputType = READ_FROM_PARAM_CONTEXT) - public boolean matchIngestionPipelineState(String... pipelineState) { + public boolean matchIngestionPipelineState(List pipelineState) { if (changeEvent == null || changeEvent.getChangeDescription() == null) { return false; } @@ -288,7 +289,7 @@ public boolean matchIngestionPipelineState(String... pipelineState) { description = "Returns true if the change event entity is updated by the mentioned users", examples = {"matchAnyFieldChange('fieldName1', 'fieldName')"}, paramInputType = NOT_REQUIRED) - public boolean matchAnyFieldChange(String... fieldChangeUpdate) { + public boolean matchAnyFieldChange(List fieldChangeUpdate) { if (changeEvent == null || changeEvent.getChangeDescription() == null) { return false; } @@ -307,7 +308,7 @@ public boolean matchAnyFieldChange(String... fieldChangeUpdate) { description = "Returns true if the change event entity belongs to a domain from the list", examples = {"matchAnyDomain('domain1', 'domain2')"}, paramInputType = NOT_REQUIRED) - public boolean matchAnyDomain(String... fieldChangeUpdate) { + public boolean matchAnyDomain(List fieldChangeUpdate) { if (changeEvent == null || changeEvent.getChangeDescription() == null) { return false; } From 950df98e09a0564ad89ccbf9473b26353edb04da Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Sun, 7 Jan 2024 14:03:33 +0530 Subject: [PATCH 20/28] Add more filters for Test Suite --- .../subscription/AlertsRuleEvaluator.java | 80 ++++++++++++------- .../openmetadata/service/util/JsonUtils.java | 8 ++ .../EntityObservabilityFilterDescriptor.json | 8 +- 3 files changed, 65 insertions(+), 31 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java index 5dbe4d8fe932..08b2994d2eda 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java @@ -1,5 +1,6 @@ package org.openmetadata.service.events.subscription; +import static org.openmetadata.common.utils.CommonUtil.listOrEmpty; import static org.openmetadata.schema.type.Function.ParameterType.ALL_INDEX_ELASTIC_SEARCH; import static org.openmetadata.schema.type.Function.ParameterType.NOT_REQUIRED; import static org.openmetadata.schema.type.Function.ParameterType.READ_FROM_PARAM_CONTEXT; @@ -44,7 +45,7 @@ public AlertsRuleEvaluator(ChangeEvent event) { input = "List of comma separated source", description = "Returns true if the change event entity being accessed has source as mentioned in condition", - examples = {"matchAnySource('bot', 'user')"}, + examples = {"matchAnySource({'bot', 'user'})"}, paramInputType = READ_FROM_PARAM_CONTEXT) public boolean matchAnySource(List originEntities) { if (changeEvent == null || changeEvent.getEntityType() == null) { @@ -64,7 +65,7 @@ public boolean matchAnySource(List originEntities) { input = "List of comma separated ownerName", description = "Returns true if the change event entity being accessed has following owners from the List.", - examples = {"matchAnyOwnerName('Owner1', 'Owner2')"}, + examples = {"matchAnyOwnerName({'Owner1', 'Owner2'})"}, paramInputType = SPECIFIC_INDEX_ELASTIC_SEARCH) public boolean matchAnyOwnerName(List ownerNameList) { if (changeEvent == null || changeEvent.getEntity() == null) { @@ -97,7 +98,7 @@ public boolean matchAnyOwnerName(List ownerNameList) { input = "List of comma separated entityName", description = "Returns true if the change event entity being accessed has following entityName from the List.", - examples = {"matchAnyEntityFqn('Name1', 'Name')"}, + examples = {"matchAnyEntityFqn({'FQN1', 'FQN2'})"}, paramInputType = ALL_INDEX_ELASTIC_SEARCH) public boolean matchAnyEntityFqn(List entityNames) { if (changeEvent == null || changeEvent.getEntity() == null) { @@ -123,7 +124,7 @@ public boolean matchAnyEntityFqn(List entityNames) { input = "List of comma separated entity Ids", description = "Returns true if the change event entity being accessed has following entityId from the List.", - examples = {"matchAnyEntityId('uuid1', 'uuid2')"}, + examples = {"matchAnyEntityId({'uuid1', 'uuid2'})"}, paramInputType = ALL_INDEX_ELASTIC_SEARCH) public boolean matchAnyEntityId(List entityIds) { if (changeEvent == null || changeEvent.getEntity() == null) { @@ -165,7 +166,7 @@ public boolean matchAnyEventType(List eventTypesList) { input = "List of comma separated eventTypes", description = "Returns true if the change event entity being accessed has following entityId from the List.", - examples = {"matchTestResult('Success', 'Failed', 'Aborted')"}, + examples = {"matchTestResult({'Success', 'Failed', 'Aborted'})"}, paramInputType = READ_FROM_PARAM_CONTEXT) public boolean matchTestResult(List testResults) { if (changeEvent == null || changeEvent.getChangeDescription() == null) { @@ -184,7 +185,8 @@ public boolean matchTestResult(List testResults) { for (FieldChange fieldChange : fieldChanges) { if (fieldChange.getName().equals("testCaseResult") && fieldChange.getNewValue() != null) { - TestCaseResult testCaseResult = (TestCaseResult) fieldChange.getNewValue(); + TestCaseResult testCaseResult = + JsonUtils.readOrConvertValue(fieldChange.getNewValue(), TestCaseResult.class); TestCaseStatus status = testCaseResult.getTestCaseStatus(); for (String givenStatus : testResults) { if (givenStatus.equals(status.value())) { @@ -196,12 +198,39 @@ public boolean matchTestResult(List testResults) { return false; } + @Function( + name = "filterByTableNameTestCaseBelongsTo", + input = "List of comma separated Test Suite", + description = + "Returns true if the change event entity being accessed has following entityId from the List.", + examples = {"filterByTableNameTestCaseBelongsTo({'tableName1', 'tableName2'})"}, + paramInputType = READ_FROM_PARAM_CONTEXT) + public boolean filterByTableNameTestCaseBelongsTo(List tableNameList) { + if (changeEvent == null) { + return false; + } + if (!changeEvent.getEntityType().equals(TEST_CASE)) { + // in case the entity is not test case return since the filter doesn't apply + return true; + } + + EntityInterface entity = getEntity(changeEvent); + for (String name : tableNameList) { + if (entity.getFullyQualifiedName().contains(name)) { + return true; + } + } + return false; + } + @Function( name = "getTestCaseStatusIfInTestSuite", input = "List of comma separated Test Suite", description = "Returns true if the change event entity being accessed has following entityId from the List.", - examples = {"getTestCaseStatusIfInTestSuite('Success', 'Failed', 'Aborted')"}, + examples = { + "getTestCaseStatusIfInTestSuite({'testSuite1','testSuite2'}, {'Success', 'Failed', 'Aborted'})" + }, paramInputType = READ_FROM_PARAM_CONTEXT) public boolean getTestCaseStatusIfInTestSuite( List testSuiteList, List testResults) { @@ -214,21 +243,18 @@ public boolean getTestCaseStatusIfInTestSuite( } // we need to handle both fields updated and fields added - List fieldChanges = changeEvent.getChangeDescription().getFieldsUpdated(); - if (!changeEvent.getChangeDescription().getFieldsAdded().isEmpty()) { - fieldChanges.addAll(changeEvent.getChangeDescription().getFieldsAdded()); - } - - for (FieldChange fieldChange : fieldChanges) { - if (fieldChange.getName().equals("testCaseResult") && fieldChange.getNewValue() != null) { - TestCaseResult testCaseResult = (TestCaseResult) fieldChange.getNewValue(); - TestCaseStatus status = testCaseResult.getTestCaseStatus(); - for (String givenStatus : testResults) { - if (givenStatus.equals(status.value())) { - return true; - } - } - } + EntityInterface entity = getEntity(changeEvent); + TestCase entityWithTestSuite = + Entity.getEntity( + changeEvent.getEntityType(), entity.getId(), "testSuites", Include.NON_DELETED); + boolean testSuiteFiltering = + listOrEmpty(entityWithTestSuite.getTestSuites()).stream() + .anyMatch( + testSuite -> + testSuiteList.stream() + .anyMatch(name -> testSuite.getFullyQualifiedName().equals(name))); + if (testSuiteFiltering) { + return matchTestResult(testResults); } return false; } @@ -237,7 +263,7 @@ public boolean getTestCaseStatusIfInTestSuite( name = "matchUpdatedBy", input = "List of comma separated user names that updated the entity", description = "Returns true if the change event entity is updated by the mentioned users", - examples = {"matchUpdatedBy('user1', 'user2')"}, + examples = {"matchUpdatedBy({'user1', 'user2'})"}, paramInputType = READ_FROM_PARAM_CONTEXT) public boolean matchUpdatedBy(List updatedByUserList) { if (changeEvent == null || changeEvent.getUserName() == null) { @@ -258,7 +284,7 @@ public boolean matchUpdatedBy(List updatedByUserList) { description = "Returns true if the change event entity being accessed has following entityId from the List.", examples = { - "matchIngestionPipelineState('queued', 'success', 'failed', 'running', 'partialSuccess')" + "matchIngestionPipelineState({'queued', 'success', 'failed', 'running', 'partialSuccess'})" }, paramInputType = READ_FROM_PARAM_CONTEXT) public boolean matchIngestionPipelineState(List pipelineState) { @@ -287,7 +313,7 @@ public boolean matchIngestionPipelineState(List pipelineState) { name = "matchAnyFieldChange", input = "List of comma separated fields change", description = "Returns true if the change event entity is updated by the mentioned users", - examples = {"matchAnyFieldChange('fieldName1', 'fieldName')"}, + examples = {"matchAnyFieldChange({'fieldName1', 'fieldName'})"}, paramInputType = NOT_REQUIRED) public boolean matchAnyFieldChange(List fieldChangeUpdate) { if (changeEvent == null || changeEvent.getChangeDescription() == null) { @@ -306,10 +332,10 @@ public boolean matchAnyFieldChange(List fieldChangeUpdate) { name = "matchAnyDomain", input = "List of comma separated Domains", description = "Returns true if the change event entity belongs to a domain from the list", - examples = {"matchAnyDomain('domain1', 'domain2')"}, + examples = {"matchAnyDomain({'domain1', 'domain2'})"}, paramInputType = NOT_REQUIRED) public boolean matchAnyDomain(List fieldChangeUpdate) { - if (changeEvent == null || changeEvent.getChangeDescription() == null) { + if (changeEvent == null) { return false; } EntityInterface entity = getEntity(changeEvent); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/util/JsonUtils.java b/openmetadata-service/src/main/java/org/openmetadata/service/util/JsonUtils.java index 75ce138d250e..4ed576e417b3 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/util/JsonUtils.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/util/JsonUtils.java @@ -118,6 +118,14 @@ public static Map getMap(Object o) { return map; } + public static T readOrConvertValue(Object obj, Class clz) { + if (obj instanceof String) { + return (T) readValue((String) obj, clz); + } else { + return (T) convertValue(obj, clz); + } + } + public static T readValue(String json, String clazzName) { try { return (T) readValue(json, Class.forName(clazzName)); diff --git a/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json b/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json index 6b9db2e0ce4e..49dd20a61e83 100644 --- a/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json +++ b/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json @@ -34,7 +34,7 @@ "displayName": "Get Schema Changes", "description" : "Get Updates for Schema Changes", "effect" : "include", - "condition": "matchAnyFieldChange('columns', 'description')", + "condition": "matchAnyFieldChange({'columns', 'description'})", "inputType": "none" }, { @@ -43,7 +43,7 @@ "displayName": "Get Table Metrics Updates", "description" : "Get Updates About Table Metrics", "effect" : "include", - "condition": "matchAnyFieldChange('customMetrics', 'profile')", + "condition": "matchAnyFieldChange({'customMetrics', 'profile'})", "inputType": "none" } ] @@ -222,9 +222,9 @@ "displayName": "Filter By Table Name", "description" : "Filter A Change Event By Pipeline Name", "effect" : "include", - "condition": "matchAnyFieldChange(${fieldList})", + "condition": "filterByTableNameTestCaseBelongsTo(${tableNameList})", "arguments": [ - "fieldList" + "tableNameList" ], "inputType": "runtime" } From f9f09bfe85dd43fdb23656e9a012af8b1bccc6cd Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Sun, 7 Jan 2024 14:16:45 +0530 Subject: [PATCH 21/28] Populate Default Schema Updates to Look for --- .../EntityObservabilityFilterDescriptor.json | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json b/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json index 49dd20a61e83..6f606028c5ca 100644 --- a/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json +++ b/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json @@ -29,12 +29,12 @@ ], "supportedActions" : [ { - "name": "GetSchemaChanges", - "fullyQualifiedName": "eventSubscription.filterBySchemaChange", + "name": "GetTableSchemaChanges", + "fullyQualifiedName": "eventSubscription.GetTableSchemaChanges", "displayName": "Get Schema Changes", "description" : "Get Updates for Schema Changes", "effect" : "include", - "condition": "matchAnyFieldChange({'columns', 'description'})", + "condition": "matchAnyFieldChange({'columns'})", "inputType": "none" }, { @@ -78,12 +78,12 @@ ], "supportedActions" : [ { - "name": "GetSchemaChanges", - "fullyQualifiedName": "eventSubscription.filterBySchemaChange", + "name": "GetTopicSchemaChanges", + "fullyQualifiedName": "eventSubscription.GetTopicSchemaChanges", "displayName": "Get Schema Changes", "description" : "Get Updates for Schema Changes", "effect" : "include", - "condition": "matchAnyFieldChange(${fieldList})", + "condition": "matchAnyFieldChange({'messageSchema'})", "arguments": [ "fieldList" ], @@ -121,12 +121,12 @@ ], "supportedActions" : [ { - "name": "GetSchemaChanges", - "fullyQualifiedName": "eventSubscription.filterBySchemaChange", + "name": "GetContainerSchemaChanges", + "fullyQualifiedName": "eventSubscription.GetContainerSchemaChanges", "displayName": "Get Schema Changes", "description" : "Get Updates for Schema Changes", "effect" : "include", - "condition": "matchAnyFieldChange(${fieldList})", + "condition": "matchAnyFieldChange({'parent','children'})", "arguments": [ "fieldList" ], @@ -164,12 +164,12 @@ ], "supportedActions" : [ { - "name": "GetSchemaChanges", + "name": "GetPipelineSchemaChanges", "fullyQualifiedName": "eventSubscription.filterBySchemaChange", "displayName": "Get Schema Changes", "description" : "Get Updates for Schema Changes", "effect" : "include", - "condition": "matchAnyFieldChange(${fieldList})", + "condition": "matchAnyFieldChange({'sourceUrl','pipelineLocation',''})", "arguments": [ "fieldList" ], @@ -231,12 +231,12 @@ ], "supportedActions" : [ { - "name": "GetSchemaChanges", + "name": "GetTestCaseSchemaChanges", "fullyQualifiedName": "eventSubscription.filterBySchemaChange", "displayName": "Get Schema Changes", "description" : "Get Updates for Schema Changes", "effect" : "include", - "condition": "matchAnyFieldChange(${fieldList})", + "condition": "matchAnyFieldChange({'testDefinition','parameterValues'})", "arguments": [ "fieldList" ], @@ -298,12 +298,12 @@ ], "supportedActions" : [ { - "name": "GetSchemaChanges", + "name": "GetTestSuiteSchemaChanges", "fullyQualifiedName": "eventSubscription.filterBySchemaChange", "displayName": "Get Schema Changes", "description" : "Get Updates for Schema Changes", "effect" : "include", - "condition": "matchAnyFieldChange(${fieldList})", + "condition": "matchAnyFieldChange({'connection','pipelines'})", "arguments": [ "fieldList" ], From def4bd53945eda0c9bd4395d061851503470e293 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Sun, 7 Jan 2024 14:18:22 +0530 Subject: [PATCH 22/28] typo --- .../json/data/EntityObservabilityFilterDescriptor.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json b/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json index 6f606028c5ca..db844147c9a1 100644 --- a/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json +++ b/openmetadata-service/src/main/resources/json/data/EntityObservabilityFilterDescriptor.json @@ -169,7 +169,7 @@ "displayName": "Get Schema Changes", "description" : "Get Updates for Schema Changes", "effect" : "include", - "condition": "matchAnyFieldChange({'sourceUrl','pipelineLocation',''})", + "condition": "matchAnyFieldChange({'sourceUrl','pipelineLocation'})", "arguments": [ "fieldList" ], From 1250c87766b91d295cef01e021451ac8297b1bd4 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Sun, 7 Jan 2024 21:27:30 +0530 Subject: [PATCH 23/28] Add matchFieldChange and other in paramAdditionalContext --- .../service/events/subscription/AlertUtil.java | 15 +++++++++++++++ .../events/subscription/AlertsRuleEvaluator.java | 6 +++--- .../events/subscriptionResourceDescriptor.json | 14 +++++++++++++- .../main/resources/json/schema/type/function.json | 9 ++------- 4 files changed, 33 insertions(+), 11 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java index 9c7f303c380c..9ec8417c40a8 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java @@ -16,6 +16,7 @@ import static org.openmetadata.common.utils.CommonUtil.listOrEmpty; import static org.openmetadata.common.utils.CommonUtil.nullOrEmpty; import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.ACTIVITY_FEED; +import static org.openmetadata.service.Entity.DOMAIN; import static org.openmetadata.service.Entity.TEAM; import static org.openmetadata.service.Entity.USER; import static org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer.OFFSET_EXTENSION; @@ -33,6 +34,7 @@ import javax.ws.rs.BadRequestException; import lombok.extern.slf4j.Slf4j; import org.openmetadata.common.utils.CommonUtil; +import org.openmetadata.schema.EntityInterface; import org.openmetadata.schema.api.events.CreateEventSubscription; import org.openmetadata.schema.api.events.Observability; import org.openmetadata.schema.entity.events.Argument; @@ -140,6 +142,19 @@ public static Map getAlertFilterFunctions() { func.setParamAdditionalContext( paramAdditionalContext.withData(new HashSet<>(testResultStatus))); } + case matchAnyFieldChange -> { + Map> entityToFieldMap = + EntityInterface.ENTITY_TYPE_TO_CLASS_MAP.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> + Entity.getEntityFields( + EntityInterface.ENTITY_TYPE_TO_CLASS_MAP.get(entry.getKey())))); + func.setParamAdditionalContext(paramAdditionalContext.withData(entityToFieldMap)); + } + case matchAnyDomain -> func.setParamAdditionalContext( + paramAdditionalContext.withData(getEntitiesIndex(List.of(DOMAIN)))); default -> LOG.error("Invalid Function name : {}", type); } alertFunctions.put(func.getName(), func); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java index 08b2994d2eda..964bec950ab9 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertsRuleEvaluator.java @@ -2,8 +2,8 @@ import static org.openmetadata.common.utils.CommonUtil.listOrEmpty; import static org.openmetadata.schema.type.Function.ParameterType.ALL_INDEX_ELASTIC_SEARCH; -import static org.openmetadata.schema.type.Function.ParameterType.NOT_REQUIRED; import static org.openmetadata.schema.type.Function.ParameterType.READ_FROM_PARAM_CONTEXT; +import static org.openmetadata.schema.type.Function.ParameterType.READ_FROM_PARAM_CONTEXT_PER_ENTITY; import static org.openmetadata.schema.type.Function.ParameterType.SPECIFIC_INDEX_ELASTIC_SEARCH; import static org.openmetadata.service.Entity.INGESTION_PIPELINE; import static org.openmetadata.service.Entity.TEAM; @@ -314,7 +314,7 @@ public boolean matchIngestionPipelineState(List pipelineState) { input = "List of comma separated fields change", description = "Returns true if the change event entity is updated by the mentioned users", examples = {"matchAnyFieldChange({'fieldName1', 'fieldName'})"}, - paramInputType = NOT_REQUIRED) + paramInputType = READ_FROM_PARAM_CONTEXT_PER_ENTITY) public boolean matchAnyFieldChange(List fieldChangeUpdate) { if (changeEvent == null || changeEvent.getChangeDescription() == null) { return false; @@ -333,7 +333,7 @@ public boolean matchAnyFieldChange(List fieldChangeUpdate) { input = "List of comma separated Domains", description = "Returns true if the change event entity belongs to a domain from the list", examples = {"matchAnyDomain({'domain1', 'domain2'})"}, - paramInputType = NOT_REQUIRED) + paramInputType = SPECIFIC_INDEX_ELASTIC_SEARCH) public boolean matchAnyDomain(List fieldChangeUpdate) { if (changeEvent == null) { return false; diff --git a/openmetadata-spec/src/main/resources/json/schema/events/subscriptionResourceDescriptor.json b/openmetadata-spec/src/main/resources/json/schema/events/subscriptionResourceDescriptor.json index 1275c900cd95..9bf8bf8469a8 100644 --- a/openmetadata-spec/src/main/resources/json/schema/events/subscriptionResourceDescriptor.json +++ b/openmetadata-spec/src/main/resources/json/schema/events/subscriptionResourceDescriptor.json @@ -17,9 +17,12 @@ "matchAnyEntityId", "matchAnyEventType", "matchTestResult", + "filterByTableNameTestCaseBelongsTo", + "getTestCaseStatusIfInTestSuite", "matchUpdatedBy", "matchIngestionPipelineState", - "matchAnyFieldChange" + "matchAnyFieldChange", + "matchAnyDomain" ], "javaEnums": [ { @@ -40,6 +43,12 @@ { "name": "matchTestResult" }, + { + "name": "filterByTableNameTestCaseBelongsTo" + }, + { + "name": "getTestCaseStatusIfInTestSuite" + }, { "name": "matchUpdatedBy" }, @@ -48,6 +57,9 @@ }, { "name": "matchAnyFieldChange" + }, + { + "name": "matchAnyDomain" } ] } diff --git a/openmetadata-spec/src/main/resources/json/schema/type/function.json b/openmetadata-spec/src/main/resources/json/schema/type/function.json index e14ecc5689d4..6a8466196683 100644 --- a/openmetadata-spec/src/main/resources/json/schema/type/function.json +++ b/openmetadata-spec/src/main/resources/json/schema/type/function.json @@ -8,19 +8,14 @@ "definitions": { "parameterType": { "type": "string", - "enum": ["NotRequired", "AllIndexElasticSearch", "SpecificIndexElasticSearch", "ReadFromParamContext"] + "enum": ["NotRequired", "AllIndexElasticSearch", "SpecificIndexElasticSearch", "ReadFromParamContext", "ReadFromParamContextPerEntity"] }, "paramAdditionalContext": { "description": "Additional Context", "type": "object", "properties": { "data": { - "description": "List of Entities", - "type": "array", - "items": { - "type": "string" - }, - "uniqueItems": true + "description": "List of Entities" } }, "additionalProperties": false From 54f9ede98658fc98cdacd7baba0cc461355a03cb Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Mon, 8 Jan 2024 14:22:44 +0530 Subject: [PATCH 24/28] Review Comments --- .../changeEvent/AbstractEventConsumer.java | 27 +++++++------------ .../apps/bundles/changeEvent/Alert.java | 20 ++++++++++++++ .../scheduled/EventSubscriptionScheduler.java | 2 +- .../events/subscription/AlertUtil.java | 6 ++--- .../service/jdbi3/CollectionDAO.java | 2 +- .../events/eventSubscriptionOffset.json | 3 ++- 6 files changed, 37 insertions(+), 23 deletions(-) create mode 100644 openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Alert.java diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index af93010de287..3c256f6234f6 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -17,7 +17,7 @@ import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.AWAITING_RETRY; import static org.openmetadata.schema.entity.events.SubscriptionStatus.Status.FAILED; import static org.openmetadata.service.events.subscription.AlertUtil.getFilteredEvent; -import static org.openmetadata.service.events.subscription.AlertUtil.getInitialAlertOffsetFromDb; +import static org.openmetadata.service.events.subscription.AlertUtil.getStartingOffset; import com.fasterxml.jackson.core.type.TypeReference; import java.util.ArrayList; @@ -47,13 +47,14 @@ @Slf4j @DisallowConcurrentExecution @PersistJobDataAfterExecution -public abstract class AbstractEventConsumer implements Consumer, Job { +public abstract class AbstractEventConsumer + implements Alert, Consumer, Job { public static final String ALERT_OFFSET_KEY = "alertOffsetKey"; public static final String ALERT_INFO_KEY = "alertInfoKey"; public static final String OFFSET_EXTENSION = "eventSubscription.Offset"; public static final String METRICS_EXTENSION = "eventSubscription.metrics"; public static final String FAILED_EVENT_EXTENSION = "eventSubscription.failedEvent"; - private int offset = -1; + private long offset = -1; private AlertMetrics alertMetrics; @Getter @Setter private JobDetail jobDetail; @@ -71,11 +72,8 @@ private void init(JobExecutionContext context) { this.doInit(context); } - protected abstract void doInit(JobExecutionContext context); - - protected void sendAlert(ChangeEvent event) throws EventPublisherException { - /* This method needs to be over-ridden by specific Publisher for sending Alert */ - + protected void doInit(JobExecutionContext context) { + // To be implemented by the Subclass if needed } @Override @@ -143,7 +141,7 @@ public void handleFailedEvent(EventPublisherException ex) { } } - private int loadInitialOffset(JobExecutionContext context) { + private long loadInitialOffset(JobExecutionContext context) { EventSubscriptionOffset jobStoredOffset = (EventSubscriptionOffset) jobDetail.getJobDataMap().get(ALERT_OFFSET_KEY); // If the Job Data Map has the latest offset, use it @@ -151,7 +149,7 @@ private int loadInitialOffset(JobExecutionContext context) { return jobStoredOffset.getOffset(); } else { EventSubscriptionOffset eventSubscriptionOffset = - getInitialAlertOffsetFromDb(eventSubscription.getId()); + getStartingOffset(eventSubscription.getId()); // Update the Job Data Map with the latest offset context.getJobDetail().getJobDataMap().put(ALERT_OFFSET_KEY, eventSubscriptionOffset); return eventSubscriptionOffset.getOffset(); @@ -200,10 +198,9 @@ public void publishEvents(List events) { @Override public void commit(JobExecutionContext jobExecutionContext) { long currentTime = System.currentTimeMillis(); + // Upsert Offset EventSubscriptionOffset eventSubscriptionOffset = new EventSubscriptionOffset().withOffset(offset).withTimestamp(currentTime); - - // Upsert Offset to Database Entity.getCollectionDAO() .eventSubscriptionDAO() .upsertSubscriberExtension( @@ -211,14 +208,12 @@ public void commit(JobExecutionContext jobExecutionContext) { OFFSET_EXTENSION, "eventSubscriptionOffset", JsonUtils.pojoToJson(eventSubscriptionOffset)); - - // Update the Job Data Map with the latest offset jobExecutionContext .getJobDetail() .getJobDataMap() .put(ALERT_OFFSET_KEY, eventSubscriptionOffset); - // Upsert Metrics to Database + // Upsert Metrics AlertMetrics metrics = new AlertMetrics() .withTotalEvents(alertMetrics.getTotalEvents()) @@ -232,8 +227,6 @@ public void commit(JobExecutionContext jobExecutionContext) { METRICS_EXTENSION, "alertMetrics", JsonUtils.pojoToJson(metrics)); - - // Update the Job Data Map with latest Metrics jobExecutionContext.getJobDetail().getJobDataMap().put(METRICS_EXTENSION, alertMetrics); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Alert.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Alert.java new file mode 100644 index 000000000000..36ff9b01bdd1 --- /dev/null +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/Alert.java @@ -0,0 +1,20 @@ +/* + * Copyright 2021 Collate + * Licensed 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.openmetadata.service.apps.bundles.changeEvent; + +import org.openmetadata.service.events.errors.EventPublisherException; + +public interface Alert { + void sendAlert(T event) throws EventPublisherException; +} diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java index 0ce89572f1e8..5fdc59f5c620 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java @@ -191,7 +191,7 @@ public EventSubscription getEventSubscriptionFromScheduledJob(UUID id) { } public boolean checkIfPublisherPublishedAllEvents(UUID subscriptionID) { - int countOfEvents = Entity.getCollectionDAO().changeEventDAO().listCount(); + long countOfEvents = Entity.getCollectionDAO().changeEventDAO().getLatestOffset(); try { JobDetail jobDetail = alertsScheduler.getJobDetail(new JobKey(subscriptionID.toString(), ALERT_JOB_GROUP)); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java index 9ec8417c40a8..33aa561d15ba 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/subscription/AlertUtil.java @@ -269,8 +269,8 @@ public static List getFilteredEvent( return filteredEvents; } - public static EventSubscriptionOffset getInitialAlertOffsetFromDb(UUID eventSubscriptionId) { - int eventSubscriptionOffset; + public static EventSubscriptionOffset getStartingOffset(UUID eventSubscriptionId) { + long eventSubscriptionOffset; String json = Entity.getCollectionDAO() .eventSubscriptionDAO() @@ -280,7 +280,7 @@ public static EventSubscriptionOffset getInitialAlertOffsetFromDb(UUID eventSubs JsonUtils.readValue(json, EventSubscriptionOffset.class); eventSubscriptionOffset = offsetFromDb.getOffset(); } else { - eventSubscriptionOffset = Entity.getCollectionDAO().changeEventDAO().listCount(); + eventSubscriptionOffset = Entity.getCollectionDAO().changeEventDAO().getLatestOffset(); } return new EventSubscriptionOffset().withOffset(eventSubscriptionOffset); } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java index 28f54360e38c..00b6ffd2ad88 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/jdbi3/CollectionDAO.java @@ -3197,7 +3197,7 @@ List listWithoutEntityFilter( List list(@Bind("limit") long limit, @Bind("offset") long offset); @SqlQuery("SELECT count(*) FROM change_event") - int listCount(); + long getLatestOffset(); } interface TypeEntityDAO extends EntityDAO { diff --git a/openmetadata-spec/src/main/resources/json/schema/events/eventSubscriptionOffset.json b/openmetadata-spec/src/main/resources/json/schema/events/eventSubscriptionOffset.json index 464cab327345..cc0c61dfa5ed 100644 --- a/openmetadata-spec/src/main/resources/json/schema/events/eventSubscriptionOffset.json +++ b/openmetadata-spec/src/main/resources/json/schema/events/eventSubscriptionOffset.json @@ -8,7 +8,8 @@ "properties": { "offset": { "description": "Name of this Event Filter.", - "type": "integer" + "type": "integer", + "existingJavaType": "Long" }, "timestamp": { "description": "Update time of the job status.", From abe9b08a1f78f47f86c61ac025a17b3fc1c5e0a3 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Mon, 8 Jan 2024 14:41:30 +0530 Subject: [PATCH 25/28] Spotless fix --- .../service/apps/bundles/changeEvent/AbstractEventConsumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java index 3c256f6234f6..a777bef57ba0 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/AbstractEventConsumer.java @@ -149,7 +149,7 @@ private long loadInitialOffset(JobExecutionContext context) { return jobStoredOffset.getOffset(); } else { EventSubscriptionOffset eventSubscriptionOffset = - getStartingOffset(eventSubscription.getId()); + getStartingOffset(eventSubscription.getId()); // Update the Job Data Map with the latest offset context.getJobDetail().getJobDataMap().put(ALERT_OFFSET_KEY, eventSubscriptionOffset); return eventSubscriptionOffset.getOffset(); From 7c56334dfb1b9c7f8cb7519e25d14541e4eefd60 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Mon, 8 Jan 2024 19:59:23 +0530 Subject: [PATCH 26/28] Merge issue --- bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql index 732b6cddd9b8..53e4c470560b 100644 --- a/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql +++ b/bootstrap/sql/migrations/native/1.3.0/mysql/schemaChanges.sql @@ -133,7 +133,6 @@ UPDATE dbservice_entity de SET de.json = JSON_REMOVE(de.json, '$.connection.config.useUnityCatalog') WHERE de.serviceType IN ('Databricks','UnityCatalog'); -======= -- Add Incident ID for test case results ALTER TABLE data_quality_data_time_series ADD COLUMN incidentId varchar(36); ALTER TABLE data_quality_data_time_series ADD INDEX data_quality_data_time_series_incidentId(incidentId); From c6fab7f9db3ee5b72b38ca72ed583b79f8c1f48f Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Mon, 8 Jan 2024 20:07:07 +0530 Subject: [PATCH 27/28] Update Filter as per new Syntax --- .../resources/events/EventSubscriptionResourceTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java index 0b0513d667a0..9f891fea9b2b 100644 --- a/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java +++ b/openmetadata-service/src/test/java/org/openmetadata/service/resources/events/EventSubscriptionResourceTest.java @@ -223,7 +223,7 @@ void put_updateAlertUpdateFields(TestInfo test) throws IOException { List.of( new EventFilterRule() .withName("EventTypeCreated") - .withCondition("matchAnyEventType('entityCreated')") + .withCondition("matchAnyEventType({'entityCreated'})") .withEffect(INCLUDE))); FilteringRules rule2 = @@ -234,7 +234,7 @@ void put_updateAlertUpdateFields(TestInfo test) throws IOException { new EventFilterRule() .withName("EventTypeCreated") .withCondition( - "matchAnyEventType('entityCreated', 'entityUpdated', 'entityDeleted')") + "matchAnyEventType({'entityCreated', 'entityUpdated', 'entityDeleted'})") .withEffect(INCLUDE))); FilteringRules rule3 = @@ -244,7 +244,7 @@ void put_updateAlertUpdateFields(TestInfo test) throws IOException { List.of( new EventFilterRule() .withName("EventTypeCreated") - .withCondition("matchAnyEventType('entityUpdated', 'entityDeleted')") + .withCondition("matchAnyEventType({'entityUpdated', 'entityDeleted'})") .withEffect(INCLUDE))); FilteringRules rule4 = @@ -254,7 +254,7 @@ void put_updateAlertUpdateFields(TestInfo test) throws IOException { List.of( new EventFilterRule() .withName("EventTypeCreated") - .withCondition("matchAnyEventType('entityUpdated')") + .withCondition("matchAnyEventType({'entityUpdated'})") .withEffect(INCLUDE))); // Set Filter Rules From 879777014151632a298c2abf0863c566f177fc99 Mon Sep 17 00:00:00 2001 From: mohitdeuex Date: Mon, 8 Jan 2024 23:04:31 +0530 Subject: [PATCH 28/28] Fix Failing Test --- .../changeEvent/generic/GenericPublisher.java | 19 ++++++++++++++----- .../scheduled/EventSubscriptionScheduler.java | 3 ++- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java index 973d270f66ca..ac66f4ee4573 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/changeEvent/generic/GenericPublisher.java @@ -28,6 +28,8 @@ import org.openmetadata.schema.type.ChangeEvent; import org.openmetadata.schema.type.Webhook; import org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer; +import org.openmetadata.service.events.errors.EventPublisherException; +import org.openmetadata.service.exception.CatalogExceptionMessage; import org.openmetadata.service.security.SecurityUtil; import org.openmetadata.service.util.JsonUtils; import org.openmetadata.service.util.RestUtil; @@ -53,7 +55,7 @@ protected void doInit(JobExecutionContext context) { } @Override - public void sendAlert(ChangeEvent event) { + public void sendAlert(ChangeEvent event) throws EventPublisherException { long attemptTime = System.currentTimeMillis(); try { // Post Message to default @@ -76,14 +78,21 @@ public void sendAlert(ChangeEvent event) { } } catch (Exception ex) { Throwable cause = ex.getCause(); + String message = ""; if (cause != null && cause.getClass() == UnknownHostException.class) { - LOG.warn( - "Invalid webhook {} endpoint {}", eventSubscription.getName(), webhook.getEndpoint()); + message = + String.format( + "Unknown Host Exception for EventSubscription : %s , WebhookEndpoint : %s", + eventSubscription.getName(), webhook.getEndpoint()); + LOG.warn(message); setErrorStatus(attemptTime, 400, "UnknownHostException"); - } else { - LOG.debug("Exception occurred while publishing webhook", ex); + message = + CatalogExceptionMessage.eventPublisherFailedToPublish( + GENERIC_WEBHOOK, event, ex.getMessage()); + LOG.error(message); } + throw new EventPublisherException(message, event); } } diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java index 5fdc59f5c620..05bcb8407f0b 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/events/scheduled/EventSubscriptionScheduler.java @@ -16,6 +16,7 @@ import static org.openmetadata.schema.api.events.CreateEventSubscription.SubscriptionType.ACTIVITY_FEED; import static org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer.ALERT_INFO_KEY; import static org.openmetadata.service.apps.bundles.changeEvent.AbstractEventConsumer.ALERT_OFFSET_KEY; +import static org.openmetadata.service.events.subscription.AlertUtil.getStartingOffset; import java.util.Objects; import java.util.UUID; @@ -110,7 +111,7 @@ private JobDetail jobBuilder( AbstractEventConsumer consumer, EventSubscription eventSubscription, String jobIdentity) { JobDataMap dataMap = new JobDataMap(); dataMap.put(ALERT_INFO_KEY, eventSubscription); - dataMap.put(ALERT_OFFSET_KEY, getEventSubscriptionFromScheduledJob(eventSubscription.getId())); + dataMap.put(ALERT_OFFSET_KEY, getStartingOffset(eventSubscription.getId())); JobBuilder jobBuilder = JobBuilder.newJob(consumer.getClass()) .withIdentity(jobIdentity, ALERT_JOB_GROUP)