From 644cf7e6ba2bec7188817482c3f2526fa9f0af82 Mon Sep 17 00:00:00 2001 From: Radhika Kundam Date: Thu, 11 Jun 2026 11:27:35 -0700 Subject: [PATCH 1/3] ATLAS-5320: Distributed Notification Processing --- .../apache/atlas/repository/Constants.java | 6 +- distro/src/conf/atlas-logback.xml | 18 + .../org/apache/atlas/AtlasConfiguration.java | 6 +- .../atlas/model/metrics/AtlasMetrics.java | 18 +- .../apache/atlas/kafka/KafkaNotification.java | 6 +- .../notification/AbstractNotification.java | 20 +- .../AtlasNotificationMessageDeserializer.java | 37 +- .../notification/NotificationInterface.java | 5 + .../graph/GraphBackedSearchIndexer.java | 89 + .../store/graph/AtlasTypeDefGraphStore.java | 21 + .../store/graph/v1/SoftDeleteHandlerV1.java | 9 + .../store/graph/v2/AtlasGraphUtilsV2.java | 86 +- .../store/graph/v2/EntityGraphMapper.java | 29 + .../graph/v2/UniqAttrBasedEntityResolver.java | 26 +- .../apache/atlas/util/AtlasMetricsUtil.java | 278 ++- .../java/org/apache/atlas/RequestContext.java | 18 + .../NotificationHookConsumer.java | 53 +- .../notification/QualifiedNameRouter.java | 103 + .../notification/SerialEntityProcessor.java | 193 +- .../NotificationPreProcessor.java | 1814 +++++++++++++++++ .../NotificationHookConsumerKafkaTest.java | 8 +- .../NotificationHookConsumerTest.java | 52 +- .../NotificationHookParallelConsumerTest.java | 833 ++++++++ 23 files changed, 3603 insertions(+), 125 deletions(-) create mode 100644 webapp/src/main/java/org/apache/atlas/notification/QualifiedNameRouter.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/preprocessor/NotificationPreProcessor.java create mode 100644 webapp/src/test/java/org/apache/atlas/notification/NotificationHookParallelConsumerTest.java diff --git a/common/src/main/java/org/apache/atlas/repository/Constants.java b/common/src/main/java/org/apache/atlas/repository/Constants.java index 81fd2eb1a4a..0c718c901c9 100644 --- a/common/src/main/java/org/apache/atlas/repository/Constants.java +++ b/common/src/main/java/org/apache/atlas/repository/Constants.java @@ -135,8 +135,10 @@ public final class Constants { public static final String PROVENANCE_TYPE_KEY = encodePropertyKey(INTERNAL_PROPERTY_KEY_PREFIX + "provenanceType"); public static final String TIMESTAMP_PROPERTY_KEY = encodePropertyKey(INTERNAL_PROPERTY_KEY_PREFIX + "timestamp"); public static final String ENTITY_DELETED_TIMESTAMP_PROPERTY_KEY = encodePropertyKey(INTERNAL_PROPERTY_KEY_PREFIX + "entityDeletedTimestamp"); - public static final String MODIFICATION_TIMESTAMP_PROPERTY_KEY = encodePropertyKey(INTERNAL_PROPERTY_KEY_PREFIX + "modificationTimestamp"); - public static final String IS_INCOMPLETE_PROPERTY_KEY = encodePropertyKey(INTERNAL_PROPERTY_KEY_PREFIX + "isIncomplete"); + public static final String MODIFICATION_TIMESTAMP_PROPERTY_KEY = encodePropertyKey(INTERNAL_PROPERTY_KEY_PREFIX + "modificationTimestamp"); + public static final String ENTITY_DELETE_EVENT_TIME_PROPERTY_KEY = encodePropertyKey(INTERNAL_PROPERTY_KEY_PREFIX + "entityDeleteEventTime"); + public static final String ENTITY_CREATE_EVENT_TIME_PROPERTY_KEY = encodePropertyKey(INTERNAL_PROPERTY_KEY_PREFIX + "entityCreateEventTime"); + public static final String IS_INCOMPLETE_PROPERTY_KEY = encodePropertyKey(INTERNAL_PROPERTY_KEY_PREFIX + "isIncomplete"); /** * search backing index name. */ diff --git a/distro/src/conf/atlas-logback.xml b/distro/src/conf/atlas-logback.xml index 2e305611167..e6f54b14158 100755 --- a/distro/src/conf/atlas-logback.xml +++ b/distro/src/conf/atlas-logback.xml @@ -106,6 +106,19 @@ + + ${atlas.log.dir}/notification_processor.log + true + + %date [%thread] %level{5} [%file:%line] %msg%n + + + ${atlas.log.dir}/notification_processor-%d.log + 30 + false + + + + + + + diff --git a/intg/src/main/java/org/apache/atlas/AtlasConfiguration.java b/intg/src/main/java/org/apache/atlas/AtlasConfiguration.java index e30b80995b9..6ce7d8eb6df 100644 --- a/intg/src/main/java/org/apache/atlas/AtlasConfiguration.java +++ b/intg/src/main/java/org/apache/atlas/AtlasConfiguration.java @@ -92,8 +92,12 @@ public enum AtlasConfiguration { TASKS_USE_ENABLED("atlas.tasks.enabled", true), SESSION_TIMEOUT_SECS("atlas.session.timeout.secs", -1), UPDATE_COMPOSITE_INDEX_STATUS("atlas.update.composite.index.status", true), - NOTIFICATION_CONCURRENT_PROCESSING("atlas.notifications.concurrent", false), METRICS_TIME_TO_LIVE_HOURS("atlas.metrics.ttl.hours", 336), // 14 days default + NOTIFICATION_CONCURRENT_PROCESSING("atlas.notifications.concurrent", false), + ATLAS_PARALLEL_PROCESSING_ENABLED("atlas.notification.parallel.processing.enabled", false), + ATLAS_METADATA_TOPIC_PREFIX("atlas.notification.metadata.topic.prefix", "ATLAS_METADATA_"), + ATLAS_LINEAGE_TOPIC_PREFIX("atlas.notification.lineage.topic.prefix", "ATLAS_LINEAGE_"), + ATLAS_PARALLEL_PROCESSING_INPUT_TOPICS("atlas.notification.parallel.processing.input.topics", "ATLAS_HOOK"), SOLR_INDEX_TX_LOG_TTL_CONF("write.ahead.log.ttl.in.hours", 240), //10 days default ATLAS_AUDIT_AGING_ENABLED("atlas.audit.aging.enabled", false), diff --git a/intg/src/main/java/org/apache/atlas/model/metrics/AtlasMetrics.java b/intg/src/main/java/org/apache/atlas/model/metrics/AtlasMetrics.java index f50876e4823..5142042abea 100644 --- a/intg/src/main/java/org/apache/atlas/model/metrics/AtlasMetrics.java +++ b/intg/src/main/java/org/apache/atlas/model/metrics/AtlasMetrics.java @@ -38,9 +38,10 @@ public class AtlasMetrics implements Serializable { private static final long serialVersionUID = 1L; - public static final String PREFIX_CONNECTION_STATUS = "ConnectionStatus:"; - public static final String PREFIX_NOTIFICATION = "Notification:"; - public static final String PREFIX_SERVER = "Server:"; + public static final String PREFIX_CONNECTION_STATUS = "ConnectionStatus:"; + public static final String PREFIX_NOTIFICATION = "Notification:"; + public static final String PREFIX_SERVER = "Server:"; + public static final String PREFIX_NOTIFICATION_PROCESSOR = "NotificationProcessor:"; public static final String STAT_NOTIFY_COUNT_CURR_DAY = PREFIX_NOTIFICATION + "currentDay"; public static final String STAT_NOTIFY_AVG_TIME_CURR_DAY = PREFIX_NOTIFICATION + "currentDayAvgTime"; @@ -82,6 +83,17 @@ public class AtlasMetrics implements Serializable { public static final String STAT_SERVER_STATUS_INDEX_STORE = PREFIX_SERVER + "statusIndexStore"; public static final String STAT_SERVER_UP_TIME = PREFIX_SERVER + "upTime"; + public static final String STAT_TOTAL_MESSAGES_CONSUMED_TOTAL = PREFIX_NOTIFICATION_PROCESSOR + "messagesConsumedTotal"; + public static final String STAT_TOTAL_MESSAGES_PROCESSED_TOTAL = PREFIX_NOTIFICATION_PROCESSOR + "messagesProcessedTotal"; + public static final String STAT_TOTAL_MESSAGES_FAILED_TOTAL = PREFIX_NOTIFICATION_PROCESSOR + "messagesFailedTotal"; + public static final String STAT_AVG_PROCESSING_TIME_TOTAL = PREFIX_NOTIFICATION_PROCESSOR + "averageTimeTotal"; + public static final String STAT_GLOBAL_ENTITY_TYPE_COUNTS_TOTAL = PREFIX_NOTIFICATION_PROCESSOR + "entityTypeCounts"; + public static final String STAT_GLOBAL_FAILED_ENTITY_TYPE_COUNTS_TOTAL = PREFIX_NOTIFICATION_PROCESSOR + "failedEntityTypeCounts"; + public static final String STAT_MESSAGES_PRODUCED_BY_TOPIC = PREFIX_NOTIFICATION_PROCESSOR + "messagesPublishedByTopic"; + public static final String STAT_MESSAGES_FAILED_BY_TOPIC = PREFIX_NOTIFICATION_PROCESSOR + "messagesFailedByTopic"; + public static final String STAT_INPUT_TOPIC_STATS = PREFIX_NOTIFICATION_PROCESSOR + "inputTopicStats"; + public static final String STAT_OUTPUT_TOPIC_STATS = PREFIX_NOTIFICATION_PROCESSOR + "outputTopicStats"; + private Map> data; public AtlasMetrics() { diff --git a/notification/src/main/java/org/apache/atlas/kafka/KafkaNotification.java b/notification/src/main/java/org/apache/atlas/kafka/KafkaNotification.java index a72645f8c5f..3c5ded14535 100644 --- a/notification/src/main/java/org/apache/atlas/kafka/KafkaNotification.java +++ b/notification/src/main/java/org/apache/atlas/kafka/KafkaNotification.java @@ -77,8 +77,9 @@ public class KafkaNotification extends AbstractNotification implements Service { protected static final String CONSUMER_GROUP_ID_PROPERTY = "group.id"; - private static final String[] ATLAS_HOOK_CONSUMER_TOPICS = AtlasConfiguration.NOTIFICATION_HOOK_CONSUMER_TOPIC_NAMES.getStringArray(ATLAS_HOOK_TOPIC); - private static final String[] ATLAS_ENTITIES_CONSUMER_TOPICS = AtlasConfiguration.NOTIFICATION_ENTITIES_CONSUMER_TOPIC_NAMES.getStringArray(ATLAS_ENTITIES_TOPIC); + private static final String[] ATLAS_HOOK_CONSUMER_TOPICS = AtlasConfiguration.NOTIFICATION_HOOK_CONSUMER_TOPIC_NAMES.getStringArray(ATLAS_HOOK_TOPIC); + private static final String[] ATLAS_ENTITIES_CONSUMER_TOPICS = AtlasConfiguration.NOTIFICATION_ENTITIES_CONSUMER_TOPIC_NAMES.getStringArray(ATLAS_ENTITIES_TOPIC); + private static final String[] ATLAS_PARALLEL_PROCESSING_INPUT_TOPICS = AtlasConfiguration.ATLAS_PARALLEL_PROCESSING_INPUT_TOPICS.getStringArray(ATLAS_HOOK_TOPIC); private static final String DEFAULT_CONSUMER_CLOSED_ERROR_MESSAGE = "This consumer has already been closed."; private static final Map PRODUCER_TOPIC_MAP = new HashMap<>(); @@ -543,5 +544,6 @@ public String getMessage() { CONSUMER_TOPICS_MAP.put(NotificationType.HOOK, trimAndPurge(ATLAS_HOOK_CONSUMER_TOPICS)); CONSUMER_TOPICS_MAP.put(NotificationType.HOOK_UNSORTED, trimAndPurge(ATLAS_HOOK_UNSORTED_CONSUMER_TOPICS)); CONSUMER_TOPICS_MAP.put(NotificationType.ENTITIES, trimAndPurge(ATLAS_ENTITIES_CONSUMER_TOPICS)); + CONSUMER_TOPICS_MAP.put(NotificationType.HOOK_PREPROCESS, trimAndPurge(ATLAS_PARALLEL_PROCESSING_INPUT_TOPICS)); } } diff --git a/notification/src/main/java/org/apache/atlas/notification/AbstractNotification.java b/notification/src/main/java/org/apache/atlas/notification/AbstractNotification.java index 8e9eca9d6e6..e263a1daead 100644 --- a/notification/src/main/java/org/apache/atlas/notification/AbstractNotification.java +++ b/notification/src/main/java/org/apache/atlas/notification/AbstractNotification.java @@ -92,8 +92,11 @@ public static String getMessageJson(Object message) { * @param source * @return the message as a JSON string */ - public static void createNotificationMessages(Object message, List msgJsonList, MessageSource source) { + public static void createNotificationMessages(Object message, List msgJsonList, MessageSource source, long msgCreationTime) { AtlasNotificationMessage notificationMsg = new AtlasNotificationMessage<>(CURRENT_MESSAGE_VERSION, message, getHostAddress(), getCurrentUser(), false, source); + if (msgCreationTime > 0) { + notificationMsg.setMsgCreationTime(msgCreationTime); + } String msgJson = AtlasType.toV1Json(notificationMsg); boolean msgLengthExceedsLimit = (msgJson.length() * MAX_BYTES_PER_CHAR) > MESSAGE_MAX_LENGTH_BYTES; @@ -162,6 +165,10 @@ public static void createNotificationMessages(Object message, List msgJs } } + public static void createNotificationMessages(Object message, List msgJsonList, MessageSource source) { + createNotificationMessages(message, msgJsonList, source, 0); + } + @Override public void init(String source, Object failedMessagesLogger) { } @@ -245,6 +252,17 @@ public void send(String topic, List messages, MessageSource source) throw sendInternal(topic, strMessages); } + @Override + public void send(String topic, List messages, MessageSource source, long msgCreationTime) throws NotificationException { + List strMessages = new ArrayList<>(messages.size()); + + for (T message : messages) { + createNotificationMessages(message, strMessages, source, msgCreationTime); + } + + sendInternal(topic, strMessages); + } + private static String getNextMessageId() { String nextMsgIdPrefix = msgIdPrefix; int nextMsgIdSuffix = msgIdSuffix.getAndIncrement(); diff --git a/notification/src/main/java/org/apache/atlas/notification/AtlasNotificationMessageDeserializer.java b/notification/src/main/java/org/apache/atlas/notification/AtlasNotificationMessageDeserializer.java index 6a53ffe70f9..15e6ef71fb5 100644 --- a/notification/src/main/java/org/apache/atlas/notification/AtlasNotificationMessageDeserializer.java +++ b/notification/src/main/java/org/apache/atlas/notification/AtlasNotificationMessageDeserializer.java @@ -60,9 +60,13 @@ public abstract class AtlasNotificationMessageDeserializer implements Message private final AtomicLong messageCountTotal = new AtomicLong(0); private final AtomicLong messageCountSinceLastInterval = new AtomicLong(0); private long splitMessagesLastPurgeTime = System.currentTimeMillis(); - private long msgCreated; - private boolean spooled; - private String source; + + // Thread-local storage for message metadata to support concurrent deserialization + // These values are set during deserialize() and read by getMsgCreated(), getSpooled(), etc. + private final ThreadLocal msgCreated = ThreadLocal.withInitial(() -> 0L); + private final ThreadLocal spooled = ThreadLocal.withInitial(() -> Boolean.FALSE); + private final ThreadLocal source = ThreadLocal.withInitial(() -> null); + private final ThreadLocal sourceVersion = ThreadLocal.withInitial(() -> null); // ----- Constructors ---------------------------------------------------- /** @@ -127,15 +131,19 @@ public TypeReference> getNotificationMessageType() { // ----- MessageDeserializer --------------------------------------------- public long getMsgCreated() { - return this.msgCreated; + return this.msgCreated.get(); } public boolean getSpooled() { - return this.spooled; + return this.spooled.get(); } public String getSource() { - return this.source; + return this.source.get(); + } + + public String getSourceVersion() { + return this.sourceVersion.get(); } @Override @@ -144,19 +152,20 @@ public T deserialize(String messageJson) { messageCountTotal.incrementAndGet(); messageCountSinceLastInterval.incrementAndGet(); - - this.msgCreated = 0; - this.spooled = false; - this.source = null; + this.msgCreated.set(0L); + this.spooled.set(false); + this.source.set(null); + this.sourceVersion.set(null); AtlasNotificationBaseMessage msg = AtlasType.fromV1Json(messageJson, AtlasNotificationMessage.class); if (msg == null || msg.getVersion() == null) { // older style messages not wrapped with AtlasNotificationMessage ret = AtlasType.fromV1Json(messageJson, messageType); - } else { - this.msgCreated = ((AtlasNotificationMessage) msg).getMsgCreationTime(); - this.spooled = ((AtlasNotificationMessage) msg).getSpooled(); - this.source = msg.getSource() != null ? msg.getSource().getSource() : null; + } else { + this.msgCreated.set(((AtlasNotificationMessage) msg).getMsgCreationTime()); + this.spooled.set(((AtlasNotificationMessage) msg).getSpooled()); + this.source.set(msg.getSource() != null ? msg.getSource().getSource() : null); + this.sourceVersion.set(msg.getSource() != null ? msg.getSource().getVersion() : null); String msgJson = messageJson; diff --git a/notification/src/main/java/org/apache/atlas/notification/NotificationInterface.java b/notification/src/main/java/org/apache/atlas/notification/NotificationInterface.java index bcb58f2839b..286c8abfa59 100644 --- a/notification/src/main/java/org/apache/atlas/notification/NotificationInterface.java +++ b/notification/src/main/java/org/apache/atlas/notification/NotificationInterface.java @@ -110,6 +110,8 @@ public interface NotificationInterface { */ default void send(String topic, List messages, MessageSource source) throws NotificationException {} + default void send(String topic, List messages, MessageSource source, long msgCreationTime) throws NotificationException {} + /** * Associates the specified topic with the given notification type. * @@ -156,6 +158,9 @@ enum NotificationType { // Notifications to entity change consumers. ENTITIES(new EntityMessageDeserializer()), + // Notification to pre-process hook messages before sending to parallel topics + HOOK_PREPROCESS(new HookMessageDeserializer()), + // Notifications from Atlas async importer ASYNC_IMPORT(new HookMessageDeserializer()); diff --git a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java index 72fcf363b47..a62d6fc822c 100755 --- a/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java +++ b/repository/src/main/java/org/apache/atlas/repository/graph/GraphBackedSearchIndexer.java @@ -76,6 +76,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import static com.google.common.base.Preconditions.checkNotNull; @@ -101,7 +102,9 @@ import static org.apache.atlas.repository.Constants.CREATED_BY_KEY; import static org.apache.atlas.repository.Constants.CUSTOM_ATTRIBUTES_PROPERTY_KEY; import static org.apache.atlas.repository.Constants.EDGE_INDEX; +import static org.apache.atlas.repository.Constants.ENTITY_CREATE_EVENT_TIME_PROPERTY_KEY; import static org.apache.atlas.repository.Constants.ENTITY_DELETED_TIMESTAMP_PROPERTY_KEY; +import static org.apache.atlas.repository.Constants.ENTITY_DELETE_EVENT_TIME_PROPERTY_KEY; import static org.apache.atlas.repository.Constants.ENTITY_TEXT_PROPERTY_KEY; import static org.apache.atlas.repository.Constants.ENTITY_TYPE_PROPERTY_KEY; import static org.apache.atlas.repository.Constants.FULLTEXT_INDEX; @@ -620,6 +623,13 @@ private void initialize(AtlasGraph graph) throws RepositoryException, IndexExcep createPropertyKey(management, RELATIONSHIPTYPE_LABEL_KEY, String.class, SINGLE); createPropertyKey(management, RELATIONSHIPTYPE_TAG_PROPAGATION_KEY, String.class, SINGLE); + // Parallel processing: entity timestamp tracking for temporal entity resolution + createCommonVertexIndex(management, ENTITY_CREATE_EVENT_TIME_PROPERTY_KEY, UniqueKind.NONE, Long.class, SINGLE, true, false); + createCommonVertexIndex(management, ENTITY_DELETE_EVENT_TIME_PROPERTY_KEY, UniqueKind.NONE, Long.class, SINGLE, true, false); + + // Composite indexes for optimizing parallel processing queries + createParallelProcessingCompositeIndexes(management); + management.setIsSuccess(true); LOG.info("Index creation for global keys complete."); @@ -1058,6 +1068,85 @@ private void createVertexCompositeIndexWithSystemProperty(AtlasGraphManagement m } } + /** + * Composite index creation for parallel processing. + */ + private void createParallelProcessingCompositeIndexes(AtlasGraphManagement management) { + LOG.info("Creating enhanced composite indexes for parallel processing"); + + try { + // Get required property keys + AtlasPropertyKey entityTypeKey = management.getPropertyKey(ENTITY_TYPE_PROPERTY_KEY); + AtlasPropertyKey stateKey = management.getPropertyKey(STATE_PROPERTY_KEY); + AtlasPropertyKey createTimeKey = management.getPropertyKey(ENTITY_CREATE_EVENT_TIME_PROPERTY_KEY); + AtlasPropertyKey deleteTimeKey = management.getPropertyKey(ENTITY_DELETE_EVENT_TIME_PROPERTY_KEY); + + if (entityTypeKey == null || stateKey == null || createTimeKey == null || deleteTimeKey == null) { + LOG.warn("Failed to create parallel processing indexes: missing property keys. entityTypeKey={}, stateKey={}, createTimeKey={}, deleteTimeKey={}", + entityTypeKey, stateKey, createTimeKey, deleteTimeKey); + return; + } + + // Create enhanced per-entity-type indexes (qualifiedName always present) + int enhancedIndexesCreated = createEnhancedPerTypeIndexes(management, entityTypeKey, stateKey, createTimeKey, deleteTimeKey); + + LOG.info("Enhanced index creation complete: {} per-type indexes created", enhancedIndexesCreated); + } catch (Exception e) { + LOG.error("Error creating enhanced parallel processing indexes", e); + } + } + + /** + * Creates enhanced per-entity-type indexes. + */ + private int createEnhancedPerTypeIndexes(AtlasGraphManagement management, AtlasPropertyKey entityTypeKey, AtlasPropertyKey stateKey, AtlasPropertyKey createTimeKey, AtlasPropertyKey deleteTimeKey) { + int createdCount = 0; + Collection allEntityTypes = typeRegistry.getAllEntityDefNames(); + + for (String entityTypeName : allEntityTypes) { + try { + AtlasEntityType entityType = typeRegistry.getEntityTypeByName(entityTypeName); + + if (entityType != null) { + Map uniqueAttributes = entityType.getUniqAttributes(); + + // All entity types have qualifiedName, but verify for safety + if (uniqueAttributes != null && uniqueAttributes.containsKey("qualifiedName")) { + AtlasAttribute qualifiedNameAttr = uniqueAttributes.get("qualifiedName"); + String qualifiedNameProp = qualifiedNameAttr.getVertexPropertyName(); + AtlasPropertyKey qNameKey = management.getPropertyKey(qualifiedNameProp); + + if (qNameKey != null) { + String sanitizedTypeName = entityTypeName.replaceAll("[^a-zA-Z0-9]", ""); + String indexName = String.format("parallel_processing_%s_index", sanitizedTypeName); + AtlasGraphIndex existingIndex = management.getGraphIndex(indexName); + + if (existingIndex == null) { + // Create optimal 5-key composite index + List indexKeys = Arrays.asList( + entityTypeKey, // Filter 1: entityType (exact) + stateKey, // Filter 2: state (exact) + qNameKey, // Filter 3: qualifiedName (exact, highly selective) + createTimeKey, // Filter 4: createTime (range) + deleteTimeKey); // Filter 5: deleteTime (range) + + management.createVertexCompositeIndex(indexName, false, indexKeys); + LOG.info("Created enhanced temporal index '{}' for entity type '{}'", indexName, entityTypeName); + createdCount++; + } + } + } else { + LOG.warn("Entity type '{}' does not have qualifiedName unique attribute - this should not happen!", entityTypeName); + } + } + } catch (Exception e) { + LOG.warn("Error creating enhanced index for entity type '{}': {}", entityTypeName, e.getMessage()); + } + } + + return createdCount; + } + private boolean isIndexApplicable(Class propertyClass, AtlasCardinality cardinality) { return !(INDEX_EXCLUSION_CLASSES.contains(propertyClass) || cardinality.isMany()); } diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStore.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStore.java index 06a5bcee989..0913466c909 100644 --- a/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStore.java +++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/AtlasTypeDefGraphStore.java @@ -90,6 +90,27 @@ public AtlasTypeRegistry getTypeRegistry() { return typeRegistry; } + /** + * Registers a TypeDefChangeListener to receive notifications of type definition changes. + * @param listener the listener to register + */ + public void registerTypeDefChangeListener(TypeDefChangeListener listener) { + if (listener != null) { + typeDefChangeListeners.add(listener); + LOG.info("Registered TypeDefChangeListener: {}", listener.getClass().getSimpleName()); + } + } + + /** + * Unregisters a TypeDefChangeListener from receiving notifications. + * @param listener the listener to unregister + */ + public void unregisterTypeDefChangeListener(TypeDefChangeListener listener) { + if (listener != null && typeDefChangeListeners.remove(listener)) { + LOG.info("Unregistered TypeDefChangeListener: {}", listener.getClass().getSimpleName()); + } + } + @Override public void init() throws AtlasBaseException { LOG.info("==> AtlasTypeDefGraphStore.init()"); diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/SoftDeleteHandlerV1.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/SoftDeleteHandlerV1.java index 520b7aea203..9ee596927d3 100644 --- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/SoftDeleteHandlerV1.java +++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v1/SoftDeleteHandlerV1.java @@ -21,6 +21,7 @@ import org.apache.atlas.RequestContext; import org.apache.atlas.exception.AtlasBaseException; import org.apache.atlas.model.instance.AtlasEntity.Status; +import org.apache.atlas.repository.Constants; import org.apache.atlas.repository.graph.GraphHelper; import org.apache.atlas.repository.graphdb.AtlasEdge; import org.apache.atlas.repository.graphdb.AtlasGraph; @@ -57,6 +58,14 @@ protected void _deleteVertex(AtlasVertex instanceVertex, boolean force) { AtlasGraphUtilsV2.setEncodedProperty(instanceVertex, STATE_PROPERTY_KEY, DELETED.name()); AtlasGraphUtilsV2.setEncodedProperty(instanceVertex, MODIFICATION_TIMESTAMP_PROPERTY_KEY, RequestContext.get().getRequestTime()); AtlasGraphUtilsV2.setEncodedProperty(instanceVertex, MODIFIED_BY_KEY, RequestContext.get().getUser()); + + // Store DELETE event Kafka message creation time for accurate lineage correlation + // This is the actual time when DELETE event was published in Kafka (not when Atlas processed it) + Long deleteEventMsgCreationTime = RequestContext.get().getDeleteEventMsgTime(); + if (deleteEventMsgCreationTime != null) { + AtlasGraphUtilsV2.setEncodedProperty(instanceVertex, Constants.ENTITY_DELETE_EVENT_TIME_PROPERTY_KEY, deleteEventMsgCreationTime); + LOG.debug("Stored DELETE event Kafka message creation time {} for deleted entity {}", deleteEventMsgCreationTime, GraphHelper.getGuid(instanceVertex)); + } } } } diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/AtlasGraphUtilsV2.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/AtlasGraphUtilsV2.java index fe953486bef..2bf3e2dff09 100644 --- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/AtlasGraphUtilsV2.java +++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/AtlasGraphUtilsV2.java @@ -85,6 +85,7 @@ public class AtlasGraphUtilsV2 { private static final Logger LOG = LoggerFactory.getLogger(AtlasGraphUtilsV2.class); + public static final String ATTRIBUTE_QUALIFIED_NAME = "qualifiedName"; public static final String PROPERTY_PREFIX = Constants.INTERNAL_PROPERTY_KEY_PREFIX + "type."; public static final String SUPERTYPE_EDGE_LABEL = PROPERTY_PREFIX + ".supertype"; public static final String ENTITYTYPE_EDGE_LABEL = PROPERTY_PREFIX + ".entitytype"; @@ -395,6 +396,89 @@ public static String findFirstDeletedDuringSpooledByQualifiedName(AtlasGraph gra return ret; } + /** + * Find entity by unique attributes that was active at the given timestamp. + * + * This method performs temporal entity resolution by checking: + * 1. Active entities with creation time <= asOfTimestamp + * 2. Deleted entities where asOfTimestamp falls between creation and deletion + * + * @param graph the graph instance + * @param entityType the entity type + * @param attrValues map of unique attribute names to values (e.g., qualifiedName) + * @param asOfTimestamp the timestamp to check entity validity against + * @return AtlasVertex of the entity that was active at the given asOfTimestamp, or null if not found + */ + public static AtlasVertex findActiveByUniqueAttributesAsOf(AtlasGraph graph, AtlasEntityType entityType, + Map attrValues, Long asOfTimestamp) { + MetricRecorder metric = RequestContext.get().startMetricRecord("findActiveByUniqueAttributesAsOf"); + AtlasVertex activeVertexAsOf = null; + final String threadName = Thread.currentThread().getName(); + + // If active entity exists, check if it was created before or at the given timestamp + AtlasVertex activeVertex = findByUniqueAttributes(graph, entityType, attrValues); + if (activeVertex != null) { + Long activeEntityCreateTime = activeVertex.getProperty(Constants.ENTITY_CREATE_EVENT_TIME_PROPERTY_KEY, Long.class); + + if (activeEntityCreateTime != null && asOfTimestamp >= activeEntityCreateTime) { + LOG.debug("Parallel Processing-{}: Found ACTIVE entity {} with createTime={} valid at asOfTimestamp={}", threadName, GraphHelper.getGuid(activeVertex), activeEntityCreateTime, asOfTimestamp); + activeVertexAsOf = activeVertex; + } + } + + // Query for deleted entity where asOfTimestamp falls between creation and deletion times + // OPTIMIZATION: Add unique attributes BEFORE time range filters to leverage enhanced composite index + // Index: (entityType, state, qualifiedName, createTime, deleteTime) + if (activeVertexAsOf == null) { + AtlasVertex deletedVertex = queryDeletedEntity(graph, entityType, attrValues, asOfTimestamp); + if (deletedVertex != null) { + LOG.debug("Parallel Processing-{}: Found DELETED entity {} valid at asOfTimestamp={}", threadName, GraphHelper.getGuid(deletedVertex), asOfTimestamp); + activeVertexAsOf = deletedVertex; + } + } + + // Fallback: return active vertex if it exists (even if created after asOfTimestamp) + if (activeVertexAsOf == null && activeVertex != null) { + LOG.debug("Parallel Processing-{}: FALLBACK - Returning active entity {} (no timestamp property or valid timestamp)", threadName, GraphHelper.getGuid(activeVertex)); + activeVertexAsOf = activeVertex; + } + + RequestContext.get().endMetricRecord(metric); + + return activeVertexAsOf; + } + + private static AtlasVertex queryDeletedEntity(AtlasGraph graph, AtlasEntityType entityType, Map attrValues, Long asOfTimestamp) { + final String typeName = entityType.getTypeName(); + + // Build query with optimal filter order + AtlasGraphQuery query = graph.query() + .has(Constants.ENTITY_TYPE_PROPERTY_KEY, typeName) + .has(STATE_PROPERTY_KEY, Status.DELETED.name()); + + // Add unique attribute filters (most selective, especially qualifiedName) + // This allows the enhanced composite index to be fully utilized + final Map uniqueAttributes = entityType.getUniqAttributes(); + final Map attrNameValues = populateAttributesMap(uniqueAttributes, attrValues); + + for (Map.Entry entry : attrNameValues.entrySet()) { + String attrName = entry.getKey(); + Object attrValue = entry.getValue(); + + if (attrName != null && attrValue != null) { + query.has(attrName, attrValue); + LOG.debug("Parallel Processing-{}: Finding DELETED entity of type- {} with {}:{} valid at asOfTimestamp={}", Thread.currentThread().getName(), typeName, attrName, attrValue, asOfTimestamp); + } + } + + // Add time range filters AFTER unique attributes for optimal index usage + query.has(Constants.ENTITY_CREATE_EVENT_TIME_PROPERTY_KEY, AtlasGraphQuery.ComparisionOperator.LESS_THAN_EQUAL, asOfTimestamp) + .has(Constants.ENTITY_DELETE_EVENT_TIME_PROPERTY_KEY, AtlasGraphQuery.ComparisionOperator.GREATER_THAN, asOfTimestamp); + + Iterator results = query.vertices().iterator(); + return results.hasNext() ? results.next() : null; + } + public static AtlasVertex findByGuid(String guid) { return findByGuid(getGraphInstance(), guid); } @@ -844,7 +928,7 @@ public static void removeItemFromListProperty(AtlasEdge edge, String property, S } } - private static Map populateUniqueAttributesMap(Map uniqueAttributes, Map attrValues) { + protected static Map populateUniqueAttributesMap(Map uniqueAttributes, Map attrValues) { return populateAttributesMap(uniqueAttributes, attrValues, true); } diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/EntityGraphMapper.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/EntityGraphMapper.java index 418887918c1..caa7e8f5d9d 100644 --- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/EntityGraphMapper.java +++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/EntityGraphMapper.java @@ -402,6 +402,11 @@ public EntityMutationResponse mapAttributesAndClassifications(EntityMutationCont AtlasVertex vertex = context.getVertex(guid); AtlasEntityType entityType = context.getType(guid); + // Store CREATE event Kafka message creation time for accurate lineage correlation + // This is set here (not in createStructVertex) to ensure only actual CREATE entities + // get the timestamp, not shell entities created by lineage messages + setCreateEventMsgTime(vertex); + mapRelationshipAttributes(createdEntity, entityType, vertex, CREATE, context); mapAttributes(createdEntity, entityType, vertex, CREATE, context); @@ -424,6 +429,12 @@ public EntityMutationResponse mapAttributesAndClassifications(EntityMutationCont AtlasVertex vertex = context.getVertex(guid); AtlasEntityType entityType = context.getType(guid); + // Store CREATE event Kafka message creation time for shell entities which got updated + Long createTime = vertex.getProperty(Constants.ENTITY_CREATE_EVENT_TIME_PROPERTY_KEY, Long.class); + if (createTime == null) { + setCreateEventMsgTime(vertex); + } + mapRelationshipAttributes(updatedEntity, entityType, vertex, UPDATE, context); mapAttributes(updatedEntity, entityType, vertex, updateType, context); @@ -2866,6 +2877,24 @@ private Set getRelatedEntitiesGuids(AtlasEntity entity) { return relGuidsSet; } + /** + * Sets the CREATE event Kafka message creation time on the entity vertex. + * This timestamp is used for accurate lineage correlation and temporal entity resolution. + * + * This method is called only when processing actual CREATE messages (not shell entities), + * ensuring that entities get stamped with their actual creation time from metadata messages, + * not from lineage messages that may have created shell entities earlier and for entities which + * created as SHELL entities and then got updated + * + * @param vertex the entity vertex to set the timestamp on + */ + private void setCreateEventMsgTime(AtlasVertex vertex) { + Long createEventMsgTime = RequestContext.get().getCreateEventMsgTime(); + if (createEventMsgTime != null) { + AtlasGraphUtilsV2.setEncodedProperty(vertex, Constants.ENTITY_CREATE_EVENT_TIME_PROPERTY_KEY, createEventMsgTime); + } + } + private List updateClassificationText(AtlasClassification classification, Collection propagatedVertices) throws AtlasBaseException { List propagatedEntities = new ArrayList<>(); diff --git a/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/UniqAttrBasedEntityResolver.java b/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/UniqAttrBasedEntityResolver.java index 6009f2fcf9d..46030dfacb6 100644 --- a/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/UniqAttrBasedEntityResolver.java +++ b/repository/src/main/java/org/apache/atlas/repository/store/graph/v2/UniqAttrBasedEntityResolver.java @@ -17,6 +17,7 @@ */ package org.apache.atlas.repository.store.graph.v2; +import org.apache.atlas.AtlasConfiguration; import org.apache.atlas.AtlasErrorCode; import org.apache.atlas.RequestContext; import org.apache.atlas.exception.AtlasBaseException; @@ -28,12 +29,17 @@ import org.apache.atlas.repository.store.graph.EntityResolver; import org.apache.atlas.type.AtlasEntityType; import org.apache.atlas.type.AtlasTypeRegistry; +import org.apache.commons.collections.MapUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; +import java.util.Map; public class UniqAttrBasedEntityResolver implements EntityResolver { - private final AtlasGraph graph; + private static final Logger LOG = LoggerFactory.getLogger(UniqAttrBasedEntityResolver.class); + private final AtlasGraph graph; private final AtlasTypeRegistry typeRegistry; private final EntityGraphMapper entityGraphMapper; @@ -51,16 +57,26 @@ public EntityGraphDiscoveryContext resolveEntityReferences(EntityGraphDiscoveryC //Resolve attribute references List resolvedReferences = new ArrayList<>(); + AtlasVertex vertex; for (AtlasObjectId objId : context.getReferencedByUniqAttribs()) { - //query in graph repo that given unique attribute - check for deleted also? AtlasEntityType entityType = typeRegistry.getEntityTypeByName(objId.getTypeName()); if (entityType == null) { throw new AtlasBaseException(AtlasErrorCode.TYPE_NAME_INVALID, TypeCategory.ENTITY.name(), objId.getTypeName()); } - AtlasVertex vertex = AtlasGraphUtilsV2.findByUniqueAttributes(this.graph, entityType, objId.getUniqueAttributes()); + final Long msgTimestamp = RequestContext.get().getCreateEventMsgTime(); + final String typeName = entityType.getTypeName(); + final Map uniqAttributes = objId.getUniqueAttributes(); + + if (isParallelProcessingAllowed(uniqAttributes, msgTimestamp)) { + LOG.debug("Parallel Processing-{}: Using temporal lookup (findActiveByUniqueAttributesAsOf) with entityType={}, attrValues={}, asOfTimestamp={}", + Thread.currentThread().getName(), typeName, uniqAttributes, msgTimestamp); + vertex = AtlasGraphUtilsV2.findActiveByUniqueAttributesAsOf(this.graph, entityType, uniqAttributes, msgTimestamp); + } else { + vertex = AtlasGraphUtilsV2.findByUniqueAttributes(this.graph, entityType, uniqAttributes); + } if (vertex == null && RequestContext.get().isCreateShellEntityForNonExistingReference()) { vertex = entityGraphMapper.createShellEntityVertex(objId, context); @@ -76,4 +92,8 @@ public EntityGraphDiscoveryContext resolveEntityReferences(EntityGraphDiscoveryC return context; } + + private boolean isParallelProcessingAllowed(Map uniqAttributes, Long msgTimestamp) { + return AtlasConfiguration.ATLAS_PARALLEL_PROCESSING_ENABLED.getBoolean() && MapUtils.isNotEmpty(uniqAttributes) && msgTimestamp != null; + } } diff --git a/repository/src/main/java/org/apache/atlas/util/AtlasMetricsUtil.java b/repository/src/main/java/org/apache/atlas/util/AtlasMetricsUtil.java index 4172abe9ad1..2cc32c3365c 100644 --- a/repository/src/main/java/org/apache/atlas/util/AtlasMetricsUtil.java +++ b/repository/src/main/java/org/apache/atlas/util/AtlasMetricsUtil.java @@ -31,8 +31,11 @@ import java.time.Clock; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -41,6 +44,12 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; +import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_AVG_PROCESSING_TIME_TOTAL; +import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_GLOBAL_ENTITY_TYPE_COUNTS_TOTAL; +import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_GLOBAL_FAILED_ENTITY_TYPE_COUNTS_TOTAL; +import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_INPUT_TOPIC_STATS; +import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_MESSAGES_FAILED_BY_TOPIC; +import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_MESSAGES_PRODUCED_BY_TOPIC; import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_NOTIFY_AVG_TIME_CURR_DAY; import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_NOTIFY_AVG_TIME_CURR_HOUR; import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_NOTIFY_AVG_TIME_PREV_DAY; @@ -75,11 +84,15 @@ import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_NOTIFY_UPDATES_COUNT_PREV_DAY; import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_NOTIFY_UPDATES_COUNT_PREV_HOUR; import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_NOTIFY_UPDATES_COUNT_TOTAL; +import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_OUTPUT_TOPIC_STATS; import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_SERVER_ACTIVE_TIMESTAMP; import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_SERVER_START_TIMESTAMP; import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_SERVER_STATUS_BACKEND_STORE; import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_SERVER_STATUS_INDEX_STORE; import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_SERVER_UP_TIME; +import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_TOTAL_MESSAGES_CONSUMED_TOTAL; +import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_TOTAL_MESSAGES_FAILED_TOTAL; +import static org.apache.atlas.model.metrics.AtlasMetrics.STAT_TOTAL_MESSAGES_PROCESSED_TOTAL; import static org.apache.atlas.repository.Constants.TYPE_NAME_INTERNAL; import static org.apache.atlas.repository.Constants.TYPE_NAME_PROPERTY_KEY; import static org.apache.atlas.util.AtlasMetricsCounter.Period.ALL; @@ -106,8 +119,27 @@ public class AtlasMetricsUtil { private final AtlasMetricsCounter entityCreates = new AtlasMetricsCounter("entityCreates"); private final AtlasMetricsCounter entityUpdates = new AtlasMetricsCounter("entityUpdates"); private final AtlasMetricsCounter entityDeletes = new AtlasMetricsCounter("entityDeletes"); - private long serverStartTime; - private long serverActiveTime; + + private final AtlasMetricsCounter npMessagesConsumed = new AtlasMetricsCounter("notificationProcessor.messagesConsumed"); + private final AtlasMetricsCounter npMessagesProcessed = new AtlasMetricsCounter("notificationProcessor.messagesProcessed"); + private final AtlasMetricsCounter npMessagesFailed = new AtlasMetricsCounter("notificationProcessor.messagesFailed"); + private final AtlasMetricsCounter npMessagesRouted = new AtlasMetricsCounter("notificationProcessor.messagesRouted"); + + private final AtlasMetricsCounter npEntitiesRouted = new AtlasMetricsCounter("notificationProcessor.entitiesRouted"); + private final AtlasMetricsCounter npEntitiesFailed = new AtlasMetricsCounter("notificationProcessor.entitiesFailed"); + private final AtlasMetricsCounter npPublishFailed = new AtlasMetricsCounter("notificationProcessor.publishFailed"); + + // Stats grouped by input topic (consumer2 input) + private final Map npInputTopicStats = new ConcurrentHashMap<>(); + // Stats grouped by output/routed topic (consumer2 output) + private final Map npOutputTopicStats = new ConcurrentHashMap<>(); + + private final Map globalEntityTypeCountsNP = new ConcurrentHashMap<>(); + private final Map globalFailedEntityTypeCountsNP = new ConcurrentHashMap<>(); + private final Map totalMessagesPublishedByTopicNP = new ConcurrentHashMap<>(); + private final Map totalFailedByTopicNP = new ConcurrentHashMap<>(); + private long serverStartTime; + private long serverActiveTime; @Inject public AtlasMetricsUtil(AtlasGraph graph) { @@ -167,14 +199,107 @@ public void onNotificationProcessingComplete(String topicName, int partition, lo partitionStat.setLastMessageProcessedTime(messagesProcessed.getLastIncrTime().toEpochMilli()); } + public void onNotificationProcessorComplete(String inputTopic, int partition, long msgOffset, NotificationProcessorStats stats) { + // 1) Input topic stats + TopicStats inStats = npInputTopicStats.computeIfAbsent(inputTopic, TopicStats::new); + + TopicPartitionStat inPart = inStats.getPartitionStats().computeIfAbsent( + partition, p -> new TopicPartitionStat(inputTopic, p, msgOffset, msgOffset)); + + inPart.updateOnSuccess(msgOffset, stats.getProcessingTimeMs()); + if (stats.isFailed()) { + inPart.updateOnFailure(msgOffset); + } + + stats.getEntityTypeCounts().forEach(inStats::incrEntityType); + stats.getRoutedTopicCounts().forEach(inStats::incrRouted); + stats.getFailedTopicCounts().forEach(inStats::incrFailedRouting); + + // failed entity types (map + counter totals) + long failedEntitiesTotal = 0; + for (Map.Entry e : stats.getFailedEntityTypeCounts().entrySet()) { + String type = e.getKey(); + long count = e.getValue() == null ? 0 : e.getValue(); + + globalFailedEntityTypeCountsNP.merge(type, count, Long::sum); + inStats.incrFailedRouting("entityType:" + type, count); + + failedEntitiesTotal += count; + } + if (failedEntitiesTotal > 0) { + npEntitiesFailed.incrBy(failedEntitiesTotal); + } + + // 2) Output topic stats (all routed topics) + Set allRoutedTopics = new HashSet<>(); + allRoutedTopics.addAll(stats.getRoutedTopicCounts().keySet()); + allRoutedTopics.addAll(stats.getFailedTopicCounts().keySet()); + + for (String routedTopic : allRoutedTopics) { + TopicStats outStats = npOutputTopicStats.computeIfAbsent(routedTopic, TopicStats::new); + + TopicPartitionStat outPart = outStats.getPartitionStats().computeIfAbsent( + partition, p -> new TopicPartitionStat(routedTopic, p, msgOffset, msgOffset)); + + outPart.updateOnSuccess(msgOffset, 0); // no processing time for output topics + outStats.incrReceivedFrom(inputTopic, 1); + } + + // 3) Global totals + counters + npMessagesConsumed.incrWithMeasure(stats.getProcessingTimeMs()); + npMessagesProcessed.incrWithMeasure(stats.getProcessingTimeMs()); + + if (stats.isFailed()) { + npMessagesFailed.incr(); + } + + // entity-type totals (map + counter totals) + long routedEntitiesTotal = 0; + for (Map.Entry e : stats.getEntityTypeCounts().entrySet()) { + String type = e.getKey(); + long cnt = e.getValue() == null ? 0 : e.getValue(); + + globalEntityTypeCountsNP.merge(type, cnt, Long::sum); + routedEntitiesTotal += cnt; + } + if (routedEntitiesTotal > 0) { + npEntitiesRouted.incrBy(routedEntitiesTotal); + } + + // publish success totals + for (Map.Entry e : stats.getRoutedTopicCounts().entrySet()) { + String outTopic = e.getKey(); + long cnt = e.getValue() == null ? 0 : e.getValue(); + + totalMessagesPublishedByTopicNP.merge(outTopic, cnt, Long::sum); + npMessagesRouted.incrBy(cnt); + } + + // publish failure totals (map + counter totals) + long publishFailedTotal = 0; + for (Map.Entry e : stats.getFailedTopicCounts().entrySet()) { + String outTopic = e.getKey(); + long cnt = e.getValue() == null ? 0 : e.getValue(); + + totalFailedByTopicNP.merge(outTopic, cnt, Long::sum); + publishFailedTotal += cnt; + } + if (publishFailedTotal > 0) { + npPublishFailed.incrBy(publishFailedTotal); + } + } + public Map getStats() { Map ret = new HashMap<>(); - StatsReport messagesProcessed = this.messagesProcessed.report(); - StatsReport messagesFailed = this.messagesFailed.report(); - StatsReport entityCreates = this.entityCreates.report(); - StatsReport entityUpdates = this.entityUpdates.report(); - StatsReport entityDeletes = this.entityDeletes.report(); + StatsReport messagesProcessed = this.messagesProcessed.report(); + StatsReport messagesFailed = this.messagesFailed.report(); + StatsReport entityCreates = this.entityCreates.report(); + StatsReport entityUpdates = this.entityUpdates.report(); + StatsReport entityDeletes = this.entityDeletes.report(); + StatsReport npMessagesProcessed = this.npMessagesProcessed.report(); + StatsReport npMessagesConsumed = this.npMessagesConsumed.report(); + StatsReport npMessagesFailed = this.npMessagesFailed.report(); ret.put(STAT_SERVER_START_TIMESTAMP, serverStartTime); ret.put(STAT_SERVER_ACTIVE_TIMESTAMP, serverActiveTime); @@ -243,6 +368,19 @@ public Map getStats() { ret.put(STAT_NOTIFY_UPDATES_COUNT_PREV_DAY, entityUpdates.getCount(PREV_DAY)); ret.put(STAT_NOTIFY_DELETES_COUNT_PREV_DAY, entityDeletes.getCount(PREV_DAY)); + // Notification Processor stats + ret.put(STAT_TOTAL_MESSAGES_CONSUMED_TOTAL, npMessagesConsumed.getCount(ALL)); + ret.put(STAT_TOTAL_MESSAGES_PROCESSED_TOTAL, npMessagesProcessed.getCount(ALL)); + ret.put(STAT_TOTAL_MESSAGES_FAILED_TOTAL, npMessagesFailed.getCount(ALL)); + ret.put(STAT_AVG_PROCESSING_TIME_TOTAL, npMessagesProcessed.getMeasureAvg(ALL)); + + ret.put(STAT_GLOBAL_ENTITY_TYPE_COUNTS_TOTAL, new HashMap<>(globalEntityTypeCountsNP)); + ret.put(STAT_GLOBAL_FAILED_ENTITY_TYPE_COUNTS_TOTAL, new HashMap<>(globalFailedEntityTypeCountsNP)); + ret.put(STAT_MESSAGES_PRODUCED_BY_TOPIC, new HashMap<>(totalMessagesPublishedByTopicNP)); + ret.put(STAT_MESSAGES_FAILED_BY_TOPIC, new HashMap<>(totalFailedByTopicNP)); + ret.put(STAT_INPUT_TOPIC_STATS, new HashMap<>(npInputTopicStats)); + ret.put(STAT_OUTPUT_TOPIC_STATS, new HashMap<>(npOutputTopicStats)); + return ret; } @@ -396,9 +534,91 @@ private int getSize(Collection collection) { } } - static class TopicStats { - private final String topicName; - private final Map partitionStats = new HashMap<>(); + public static class NotificationProcessorStats { + private boolean failed; + private long processingTimeMs; + private long offset; + private int partition; + + private final Map entityTypeCounts = new HashMap<>(); + private final Map routedTopicCounts = new HashMap<>(); + private final Map failedTopicCounts = new HashMap<>(); + private final Map failedEntityTypeCounts = new HashMap<>(); + + public boolean isFailed() { + return failed; + } + + public void setFailed(boolean failed) { + this.failed = failed; + } + + public long getProcessingTimeMs() { + return processingTimeMs; + } + + public void setProcessingTimeMs(long processingTimeMs) { + this.processingTimeMs = processingTimeMs; + } + + public long getOffset() { + return offset; + } + + public void setOffset(long offset) { + this.offset = offset; + } + + public int getPartition() { + return partition; + } + + public void setPartition(int partition) { + this.partition = partition; + } + + public Map getEntityTypeCounts() { + return entityTypeCounts; + } + + public Map getRoutedTopicCounts() { + return routedTopicCounts; + } + + public Map getFailedTopicCounts() { + return failedTopicCounts; + } + + public Map getFailedEntityTypeCounts() { + return failedEntityTypeCounts; + } + + public void incrEntityType(String type) { + entityTypeCounts.merge(type, 1L, Long::sum); + } + + public void incrRoutedTopic(String topic) { + routedTopicCounts.merge(topic, 1L, Long::sum); + } + + public void incrFailedTopic(String topic) { + failedTopicCounts.merge(topic, 1L, Long::sum); + } + + public void incrFailedEntityType(String type) { + failedEntityTypeCounts.merge(type, 1L, Long::sum); + } + } + + public static class TopicStats { + private final String topicName; + private final Map partitionStats = new HashMap<>(); + + // processor-side maps + private final Map entityTypeCounts = new HashMap<>(); + private final Map routedMessagesPerOutputTopic = new HashMap<>(); + private final Map failedRoutingPerOutputTopic = new HashMap<>(); + private final Map messagesFromInputTopic = new HashMap<>(); public TopicStats(String topicName) { this.topicName = topicName; @@ -419,9 +639,29 @@ public TopicPartitionStat get(Integer partition) { public void set(Integer partition, TopicPartitionStat partitionStat) { partitionStats.put(partition, partitionStat); } + + public void incrEntityType(String type, long cnt) { + entityTypeCounts.merge(type, cnt, Long::sum); + } + + public void incrRouted(String outputTopic, long cnt) { + routedMessagesPerOutputTopic.merge(outputTopic, cnt, Long::sum); + } + + public void incrFailedRouting(String outputTopic, long cnt) { + failedRoutingPerOutputTopic.merge(outputTopic, cnt, Long::sum); + } + + public void incrReceivedFrom(String inputTopic, long cnt) { + messagesFromInputTopic.merge(inputTopic, cnt, Long::sum); + } } - static class TopicPartitionStat { + // --------------------------- + // MERGED TopicPartitionStat (consumer + processor) + // NOTE: existing field names kept to preserve old getStats() logic + // --------------------------- + public static class TopicPartitionStat { private final String topicName; private final int partition; private final long startOffset; @@ -429,6 +669,8 @@ static class TopicPartitionStat { private long lastMessageProcessedTime; private final AtomicLong failedMessageCount = new AtomicLong(); private final AtomicLong processedMessageCount = new AtomicLong(); + // processor additions + private long lastFailedTime; private final AtomicLong totalProcessingTimeMs = new AtomicLong(); public TopicPartitionStat(String topicName, int partition, long startOffset, long currentOffset) { @@ -488,5 +730,19 @@ public long getAvgProcessingTime() { return processedMessageCount == 0 ? 0 : (totalProcessingTimeMs.get() / processedMessageCount); } + + // processor wrappers + public void updateOnSuccess(long offset, long processingTimeMs) { + this.currentOffset = offset; + this.processedMessageCount.incrementAndGet(); + this.totalProcessingTimeMs.addAndGet(processingTimeMs); + this.lastMessageProcessedTime = System.currentTimeMillis(); + } + + public void updateOnFailure(long offset) { + this.currentOffset = offset; + this.failedMessageCount.incrementAndGet(); + this.lastFailedTime = System.currentTimeMillis(); + } } } diff --git a/server-api/src/main/java/org/apache/atlas/RequestContext.java b/server-api/src/main/java/org/apache/atlas/RequestContext.java index f21e971c5a9..5e35d623fbb 100644 --- a/server-api/src/main/java/org/apache/atlas/RequestContext.java +++ b/server-api/src/main/java/org/apache/atlas/RequestContext.java @@ -78,6 +78,8 @@ public class RequestContext { private String currentTypePatchAction = ""; private int maxAttempts = 1; private int attemptCount = 1; + private Long createEventMsgTime; + private Long deleteEventMsgTime; private RequestContext() { } @@ -428,6 +430,22 @@ public long getRequestTime() { return requestTime; } + public Long getCreateEventMsgTime() { + return createEventMsgTime; + } + + public void setCreateEventMsgTime(Long createEventMsgTime) { + this.createEventMsgTime = createEventMsgTime; + } + + public Long getDeleteEventMsgTime() { + return deleteEventMsgTime; + } + + public void setDeleteEventMsgTime(Long deleteEventMsgTime) { + this.deleteEventMsgTime = deleteEventMsgTime; + } + public boolean isUpdatedEntity(String guid) { return updatedEntities.containsKey(guid); } diff --git a/webapp/src/main/java/org/apache/atlas/notification/NotificationHookConsumer.java b/webapp/src/main/java/org/apache/atlas/notification/NotificationHookConsumer.java index ec3f69bbc4c..43a2aed81c4 100644 --- a/webapp/src/main/java/org/apache/atlas/notification/NotificationHookConsumer.java +++ b/webapp/src/main/java/org/apache/atlas/notification/NotificationHookConsumer.java @@ -30,9 +30,11 @@ import org.apache.atlas.listener.ActiveStateChangeHandler; import org.apache.atlas.model.notification.HookNotification; import org.apache.atlas.notification.NotificationInterface.NotificationType; +import org.apache.atlas.notification.preprocessor.NotificationPreProcessor; import org.apache.atlas.repository.converters.AtlasInstanceConverter; import org.apache.atlas.repository.impexp.AsyncImporter; import org.apache.atlas.repository.store.graph.AtlasEntityStore; +import org.apache.atlas.repository.store.graph.AtlasTypeDefGraphStore; import org.apache.atlas.repository.store.graph.EntityCorrelationStore; import org.apache.atlas.service.Service; import org.apache.atlas.type.AtlasTypeRegistry; @@ -114,11 +116,12 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl public static final String CONSUMER_AUTHORIZE_AUTHN_CACHE_TTL_SECONDS = "atlas.notification.authorize.authn.cache.ttl.seconds"; public static final int SERVER_READY_WAIT_TIME_MS = 1000; - private static final int KAFKA_CONSUMER_SHUTDOWN_WAIT = 30000; - private static final String ATLAS_HOOK_CONSUMER_THREAD_NAME = "atlas-hook-consumer-thread"; - private static final String ATLAS_HOOK_UNSORTED_CONSUMER_THREAD_NAME = "atlas-hook-unsorted-consumer-thread"; - private static final String ATLAS_IMPORT_CONSUMER_THREAD_PREFIX = "atlas-import-consumer-thread-"; - private static final String THREADNAME_PREFIX = NotificationHookConsumer.class.getSimpleName(); + private static final int KAFKA_CONSUMER_SHUTDOWN_WAIT = 30000; + private static final String ATLAS_HOOK_CONSUMER_THREAD_NAME = "atlas-hook-consumer-thread"; + private static final String ATLAS_HOOK_UNSORTED_CONSUMER_THREAD_NAME = "atlas-hook-unsorted-consumer-thread"; + private static final String ATLAS_IMPORT_CONSUMER_THREAD_PREFIX = "atlas-import-consumer-thread-"; + private static final String ATLAS_HOOK_PRE_PROCESSOR_CONSUMER_THREAD_NAME = "atlas-hook-preprocessor-consumer-thread"; + private static final String THREADNAME_PREFIX = NotificationHookConsumer.class.getSimpleName(); private final AtlasEntityStore atlasEntityStore; private final ServiceState serviceState; @@ -138,6 +141,8 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl private final long consumerMsgBufferingIntervalMS; private final int consumerMsgBufferingBatchSize; private final AsyncImporter asyncImporter; + private final boolean parallelProcessingEnabled; + private final AtlasTypeDefGraphStore typeDefStore; private ExecutorService executors; @@ -148,7 +153,7 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl List consumers; @Inject - public NotificationHookConsumer(NotificationInterface notificationInterface, AtlasEntityStore atlasEntityStore, ServiceState serviceState, AtlasInstanceConverter instanceConverter, AtlasTypeRegistry typeRegistry, AtlasMetricsUtil metricsUtil, EntityCorrelationStore entityCorrelationStore, @Lazy AsyncImporter asyncImporter) throws AtlasException { + public NotificationHookConsumer(NotificationInterface notificationInterface, AtlasEntityStore atlasEntityStore, ServiceState serviceState, AtlasInstanceConverter instanceConverter, AtlasTypeRegistry typeRegistry, AtlasMetricsUtil metricsUtil, EntityCorrelationStore entityCorrelationStore, @Lazy AsyncImporter asyncImporter, AtlasTypeDefGraphStore typeDefStore) throws AtlasException { this.notificationInterface = notificationInterface; this.atlasEntityStore = atlasEntityStore; this.serviceState = serviceState; @@ -158,6 +163,7 @@ public NotificationHookConsumer(NotificationInterface notificationInterface, Atl this.metricsUtil = metricsUtil; this.lastCommittedPartitionOffset = new HashMap<>(); this.asyncImporter = asyncImporter; + this.typeDefStore = typeDefStore; consumerRetryInterval = applicationProperties.getInt(CONSUMER_RETRY_INTERVAL, 500); minWaitDuration = applicationProperties.getInt(CONSUMER_MIN_RETRY_INTERVAL, consumerRetryInterval); // 500 ms by default @@ -167,6 +173,7 @@ public NotificationHookConsumer(NotificationInterface notificationInterface, Atl authorizeUsingMessageUser = applicationProperties.getBoolean(CONSUMER_AUTHORIZE_USING_MESSAGE_USER, false); consumerMsgBufferingIntervalMS = AtlasConfiguration.NOTIFICATION_HOOK_CONSUMER_BUFFERING_INTERVAL.getInt() * 1000L; consumerMsgBufferingBatchSize = AtlasConfiguration.NOTIFICATION_HOOK_CONSUMER_BUFFERING_BATCH_SIZE.getInt(); + parallelProcessingEnabled = AtlasConfiguration.ATLAS_PARALLEL_PROCESSING_ENABLED.getBoolean(); int authnCacheTtlSeconds = applicationProperties.getInt(CONSUMER_AUTHORIZE_AUTHN_CACHE_TTL_SECONDS, 300); @@ -334,6 +341,9 @@ void startHookConsumers() { hookConsumers.add(hookConsumer); } + if (parallelProcessingEnabled) { + hookConsumers.addAll(getPreprocessorHookConsumers()); + } startConsumers(hookConsumers); } @@ -365,6 +375,28 @@ protected ExecutorService createExecutor() { new ThreadFactoryBuilder().setNameFormat(THREADNAME_PREFIX + " thread-%d").build()); } + List getPreprocessorHookConsumers() { + List> notificationConsumers = notificationInterface.createConsumers(NotificationType.HOOK_PREPROCESS, 1); + List hookConsumers = new ArrayList<>(); + + for (final NotificationConsumer consumer : notificationConsumers) { + NotificationPreProcessor processor = new NotificationPreProcessor(applicationProperties, metricsUtil, typeRegistry, FAILED_LOG); + + // Register the processor directly with the type definition store to receive updates + if (typeDefStore != null) { + typeDefStore.registerTypeDefChangeListener(processor); + LOG.info("Registered NotificationPreProcessor with AtlasTypeDefStore for type registry updates"); + } else { + LOG.warn("AtlasTypeDefStore is null, NotificationPreProcessor will not receive type registry updates"); + } + + HookConsumer hookConsumer = new HookConsumer(ATLAS_HOOK_PRE_PROCESSOR_CONSUMER_THREAD_NAME, consumer, processor); + + hookConsumers.add(hookConsumer); + } + return hookConsumers; + } + private void startConsumers(List hookConsumers) { if (consumers == null) { consumers = new ArrayList<>(); @@ -426,6 +458,15 @@ public HookConsumer(String consumerThreadName, NotificationConsumer consumer, NotificationEntityProcessor entityProcessor) { + super(consumerThreadName); + + this.consumer = consumer; + this.entityProcessor = entityProcessor; + + LOG.info("entityProcessor: {}", entityProcessor.getClass().getSimpleName()); + } + @Override public void run() { LOG.info("==> HookConsumer run()"); diff --git a/webapp/src/main/java/org/apache/atlas/notification/QualifiedNameRouter.java b/webapp/src/main/java/org/apache/atlas/notification/QualifiedNameRouter.java new file mode 100644 index 00000000000..508ae8a64be --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/QualifiedNameRouter.java @@ -0,0 +1,103 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.atlas.notification; + +import org.apache.atlas.AtlasConfiguration; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Router class that determines which topic to use based on + * the first two parts of the qualifiedName of entities in the notification. + */ +public class QualifiedNameRouter { + private static final Logger LOG = LoggerFactory.getLogger(QualifiedNameRouter.class); + private static final String DEFAULT_TOPIC_PREFIX = AtlasConfiguration.ATLAS_METADATA_TOPIC_PREFIX.getString(); + + private final int maxTopicCount; + private final String topicPrefix; + private final ConcurrentHashMap routingCache; + private final AtomicInteger nextTopicIndex; + + public QualifiedNameRouter(int maxTopicCount) { + this(maxTopicCount, DEFAULT_TOPIC_PREFIX); + } + + public QualifiedNameRouter(int maxTopicCount, String topicPrefix) { + this.maxTopicCount = maxTopicCount > 0 ? maxTopicCount : 1; + this.topicPrefix = topicPrefix.endsWith("_") ? topicPrefix : topicPrefix + "_"; + this.routingCache = new ConcurrentHashMap<>(); + this.nextTopicIndex = new AtomicInteger(0); + } + + public String getTargetTopic(String routingKey) { + if (StringUtils.isNotBlank(routingKey)) { + int topicIndex = getTopicIndexForRoutingKey(routingKey); + String targetTopic = topicPrefix + topicIndex; + + LOG.debug("Routing message with key '{}' to topic '{}'", routingKey, targetTopic); + return targetTopic; + } else { + // Fallback to round-robin if no routing key found + int topicIndex = nextTopicIndex.getAndIncrement() % maxTopicCount; + String targetTopic = topicPrefix + topicIndex; + + LOG.debug("No routing key found, using round-robin to topic '{}'", targetTopic); + return targetTopic; + } + } + + /** + * Gets the topic index for a routing key using consistent hashing. + * + * @param routingKey The routing key (first two parts of qualifiedName) + * @return The topic index (0 to maxTopicCount-1) + */ + private int getTopicIndexForRoutingKey(String routingKey) { + return routingCache.computeIfAbsent(routingKey, key -> { + // Use consistent hashing to ensure same routing key always goes to same topic + int hash = Math.abs(key.hashCode()); + int topicIndex = hash % maxTopicCount; + + LOG.debug("Mapped routing key '{}' to topic index {}", key, topicIndex); + return topicIndex; + }); + } + + /** + * Gets statistics about the routing cache. + * + * @return A string with routing statistics + */ + public String getRoutingStats() { + return String.format("Routing cache size: %d, Max topics: %d", + routingCache.size(), maxTopicCount); + } + + /** + * Clears the routing cache. + */ + public void clearCache() { + routingCache.clear(); + LOG.info("Routing cache cleared"); + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/SerialEntityProcessor.java b/webapp/src/main/java/org/apache/atlas/notification/SerialEntityProcessor.java index fd39249d5cf..dc626e74eba 100644 --- a/webapp/src/main/java/org/apache/atlas/notification/SerialEntityProcessor.java +++ b/webapp/src/main/java/org/apache/atlas/notification/SerialEntityProcessor.java @@ -20,7 +20,6 @@ import org.apache.atlas.AtlasClient; import org.apache.atlas.AtlasClientV2; import org.apache.atlas.AtlasConfiguration; -import org.apache.atlas.AtlasErrorCode; import org.apache.atlas.RequestContext; import org.apache.atlas.exception.AtlasBaseException; import org.apache.atlas.kafka.AtlasKafkaMessage; @@ -43,6 +42,7 @@ import org.apache.atlas.notification.preprocessor.GenericEntityPreprocessor; import org.apache.atlas.notification.preprocessor.PreprocessorContext; import org.apache.atlas.repository.converters.AtlasInstanceConverter; +import org.apache.atlas.repository.graph.AtlasGraphProvider; import org.apache.atlas.repository.impexp.AsyncImporter; import org.apache.atlas.repository.store.graph.AtlasEntityStore; import org.apache.atlas.repository.store.graph.v2.AtlasEntityStream; @@ -73,8 +73,6 @@ import org.springframework.security.core.userdetails.User; import org.springframework.security.core.userdetails.UserDetails; -import javax.ws.rs.core.Response; - import java.time.Instant; import java.util.ArrayList; import java.util.Collection; @@ -86,6 +84,7 @@ import java.util.Set; import java.util.regex.Pattern; +import static org.apache.atlas.AtlasErrorCode.INSTANCE_BY_UNIQUE_ATTRIBUTE_NOT_FOUND; import static org.apache.atlas.model.instance.AtlasObjectId.KEY_GUID; import static org.apache.atlas.model.instance.AtlasObjectId.KEY_TYPENAME; import static org.apache.atlas.model.instance.AtlasObjectId.KEY_UNIQUE_ATTRIBUTES; @@ -124,9 +123,11 @@ public class SerialEntityProcessor implements NotificationEntityProcessor { private static final Logger LOG = LoggerFactory.getLogger(SerialEntityProcessor.class); private static final Logger PERF_LOG = AtlasPerfTracer.getPerfLogger(NotificationHookConsumer.class); - private static final String EXCEPTION_CLASS_NAME_JANUSGRAPH_EXCEPTION = "JanusGraphException"; - private static final String EXCEPTION_CLASS_NAME_PERMANENTLOCKING_EXCEPTION = "PermanentLockingException"; - private static final String THREADNAME_PREFIX = NotificationHookConsumer.class.getSimpleName(); + private static final String EXCEPTION_CLASS_NAME_JANUSGRAPH_EXCEPTION = "JanusGraphException"; + private static final String EXCEPTION_CLASS_NAME_PERMANENTLOCKING_EXCEPTION = "PermanentLockingException"; + private static final String EXCEPTION_CLASS_NAME_TEMPORARY_BACKEND_EXCEPTION = "TemporaryBackendException"; + private static final String EXCEPTION_CLASS_NAME_TEMPORARY_LOCKING_EXCEPTION = "TemporaryLockingException"; + private static final String THREADNAME_PREFIX = NotificationHookConsumer.class.getSimpleName(); private static final int SC_OK = 200; private static final int SC_BAD_REQUEST = 400; @@ -330,7 +331,76 @@ public TopicPartitionOffsetResult collectResults() { @Override public void shutdown() { - recordFailedMessages(failedMessages); + } + + /** + * Classifies exceptions as retryable or non-retryable for better error handling. + * + * Retryable exceptions (temporary issues, will retry with fresh transaction): + * - JanusGraphException: General graph database errors (e.g., transaction conflicts) + * - PermanentLockingException: Optimistic locking failures + * - TemporaryBackendException: Temporary storage backend issues + * - TemporaryLockingException: Temporary lock conflicts + * + * Non-retryable exceptions (fail fast, no retry): + * - AtlasSchemaViolationException: Unique constraint violations (code bug or duplicate message) + * - IllegalStateException: Invalid application state + * - NullPointerException: Programming errors + * - AtlasBaseException with INSTANCE_BY_UNIQUE_ATTRIBUTE_NOT_FOUND: Entity not found (acceptable) + * + * @param e the exception to classify + * @return true if the exception is retryable, false otherwise + */ + private boolean isRetryableException(Throwable e) { + if (e == null) { + return false; + } + + // Non-retryable: Schema violations indicate code bugs or duplicate messages + if (e instanceof org.apache.atlas.repository.graphdb.AtlasSchemaViolationException) { + LOG.warn("Non-retryable exception: SchemaViolationException indicates unique constraint violation"); + return false; + } + + // Non-retryable: Programming errors + if (e instanceof IllegalStateException || e instanceof NullPointerException) { + LOG.warn("Non-retryable exception: {} indicates programming error", e.getClass().getSimpleName()); + return false; + } + + // Non-retryable: Entity not found is acceptable in some scenarios + if (e instanceof AtlasBaseException) { + AtlasBaseException baseException = (AtlasBaseException) e; + if (baseException.getAtlasErrorCode().equals(INSTANCE_BY_UNIQUE_ATTRIBUTE_NOT_FOUND)) { + LOG.warn("Non-retryable exception: INSTANCE_BY_UNIQUE_ATTRIBUTE_NOT_FOUND"); + return false; + } + } + + // Non-retryable: InterruptedException should terminate processing + if (e instanceof InterruptedException) { + LOG.error("Non-retryable exception: InterruptedException - thread interrupted"); + return false; + } + + // Retryable: Transaction conflicts and locking issues + if (isTransactionRelatedError(e)) { + return true; + } + + // Default: Retry all other exceptions (conservative approach) + return true; + } + + private boolean isTransactionRelatedError(Throwable e) { + String exceptionClassName = e.getClass().getSimpleName(); + if (exceptionClassName.equals(EXCEPTION_CLASS_NAME_JANUSGRAPH_EXCEPTION) || + exceptionClassName.equals(EXCEPTION_CLASS_NAME_PERMANENTLOCKING_EXCEPTION) || + exceptionClassName.equals(EXCEPTION_CLASS_NAME_TEMPORARY_BACKEND_EXCEPTION) || + exceptionClassName.equals(EXCEPTION_CLASS_NAME_TEMPORARY_LOCKING_EXCEPTION)) { + return true; + } + return false; } public TopicPartitionOffsetResult handleMessage(Ticket ticket) { @@ -385,6 +455,8 @@ public TopicPartitionOffsetResult handleMessage(Ticket ticket) { return new TopicPartitionOffsetResult(kafkaMsg.getTopicPartition(), kafkaMsg.getOffset()); } + final String threadName = Thread.currentThread().getName(); + // Used for intermediate conversions during create and update String exceptionClassName = StringUtils.EMPTY; for (int numRetries = 0; numRetries < maxRetries; numRetries++) { @@ -402,11 +474,17 @@ public TopicPartitionOffsetResult handleMessage(Ticket ticket) { requestContext.setInNotificationProcessing(true); requestContext.setCreateShellEntityForNonExistingReference(createShellEntityForNonExistingReference); + LOG.debug("Parallel Processing-{}: Starting attempt={}/{} for message type={}, topic={}, offset={}, msgCreationTime={}", + threadName, numRetries + 1, maxRetries, message.getType().name(), + kafkaMsg.getTopic(), kafkaMsg.getOffset(), kafkaMsg.getMsgCreated()); + switch (message.getType()) { case ENTITY_CREATE: { final HookNotificationV1.EntityCreateRequest createRequest = (HookNotificationV1.EntityCreateRequest) message; final AtlasEntity.AtlasEntitiesWithExtInfo entities = instanceConverter.toAtlasEntities(createRequest.getEntities()); + requestContext.setCreateEventMsgTime(kafkaMsg.getMsgCreated()); + if (auditLog == null) { auditLog = new AuditFilter.AuditLog(messageUser, THREADNAME_PREFIX, AtlasClient.API_V1.CREATE_ENTITY.getMethod(), @@ -441,6 +519,8 @@ public TopicPartitionOffsetResult handleMessage(Ticket ticket) { case ENTITY_DELETE: { final HookNotificationV1.EntityDeleteRequest deleteRequest = (HookNotificationV1.EntityDeleteRequest) message; + requestContext.setDeleteEventMsgTime(kafkaMsg.getMsgCreated()); + if (auditLog == null) { auditLog = new AuditFilter.AuditLog(messageUser, THREADNAME_PREFIX, AtlasClientV2.API_V2.DELETE_ENTITY_BY_ATTRIBUTE.getMethod(), @@ -478,6 +558,8 @@ public TopicPartitionOffsetResult handleMessage(Ticket ticket) { final EntityCreateRequestV2 createRequestV2 = (EntityCreateRequestV2) message; final AtlasEntitiesWithExtInfo entities = createRequestV2.getEntities(); + requestContext.setCreateEventMsgTime(kafkaMsg.getMsgCreated()); + if (auditLog == null) { auditLog = new AuditFilter.AuditLog(messageUser, THREADNAME_PREFIX, AtlasClientV2.API_V2.CREATE_ENTITY.getMethod(), @@ -523,6 +605,8 @@ public TopicPartitionOffsetResult handleMessage(Ticket ticket) { final EntityDeleteRequestV2 deleteRequest = (EntityDeleteRequestV2) message; final List entities = deleteRequest.getEntities(); + requestContext.setDeleteEventMsgTime(kafkaMsg.getMsgCreated()); + try { for (AtlasObjectId entity : entities) { if (auditLog == null) { @@ -596,61 +680,66 @@ public TopicPartitionOffsetResult handleMessage(Ticket ticket) { RequestContext.get().resetEntityGuidUpdates(); exceptionClassName = e.getClass().getSimpleName(); - // don't retry in following conditions: - // 1. number of retry attempts reached configured count - // 2. notification processing failed due to invalid data (non-existing type, entity, ..) - boolean maxRetriesReached = numRetries == (maxRetries - 1); - AtlasErrorCode errorCode = (e instanceof AtlasBaseException) ? ((AtlasBaseException) e).getAtlasErrorCode() : null; - boolean unrecoverableFailure = errorCode != null && (Response.Status.NOT_FOUND.equals(errorCode.getHttpCode()) || Response.Status.BAD_REQUEST.equals(errorCode.getHttpCode())); + // Check if exception is retryable + boolean isRetryable = isRetryableException(e); + + if (!isRetryable) { + // Non-retryable exceptions: fail fast and move to next message + if (e instanceof InterruptedException) { + LOG.error("Interrupted!", e); + return null; + } else if (e instanceof org.apache.atlas.repository.graphdb.AtlasSchemaViolationException) { + LOG.warn("{} - {}: Non-retryable {}: Skipping message: {}", + kafkaMsg.getTopicPartition().toString(), kafkaMsg.getOffset(), exceptionClassName, e.getMessage()); + return new TopicPartitionOffsetResult(kafkaMsg.getTopicPartition(), kafkaMsg.getOffset()); + } else if (e instanceof AtlasBaseException) { + AtlasBaseException baseException = (AtlasBaseException) e; + LOG.warn("Error handling message: {}: {} - {} - {}", + ticket.getMessage().getMessage().getType(), baseException.getAtlasErrorCode(), baseException.getMessage(), ticket.getQualifiedNamesSet()); + return new TopicPartitionOffsetResult(kafkaMsg.getTopicPartition(), kafkaMsg.getOffset()); + } else { + LOG.error("Non-retryable exception: {}", exceptionClassName, e); + return new TopicPartitionOffsetResult(kafkaMsg.getTopicPartition(), kafkaMsg.getOffset()); + } + } - if (maxRetriesReached || unrecoverableFailure) { - try { - String strMessage = AbstractNotification.getMessageJson(message); + // Retryable exceptions: check if max retries exceeded + if (numRetries == (maxRetries - 1)) { + String strMessage = AbstractNotification.getMessageJson(message); - if (unrecoverableFailure) { - LOG.warn("Unrecoverable failure while processing message {}", strMessage, e); - } else { - LOG.warn("Max retries exceeded for message {}", strMessage, e); - } + LOG.warn("Offset: {}: Max retries: {} exceeded for message {}", kafkaMsg.getOffset(), maxRetries, strMessage, e); - stats.isFailedMsg = true; + stats.isFailedMsg = true; - failedMessages.add(strMessage); + failedMessages.add(strMessage); - if (failedMessages.size() >= failedMsgCacheSize) { - recordFailedMessages(failedMessages); - } - } catch (Throwable t) { - LOG.warn("error while recording failed message: type={}, topic={}, partition={}, offset={}", - message.getType(), kafkaMsg.getTopic(), kafkaMsg.getPartition(), kafkaMsg.getOffset(), t); + if (failedMessages.size() >= failedMsgCacheSize) { + recordFailedMessages(kafkaMsg.getTopic(), failedMessages); } return new TopicPartitionOffsetResult(kafkaMsg.getTopicPartition(), kafkaMsg.getOffset()); - } else if (e instanceof InterruptedException) { - LOG.error("Interrupted!", e); - return null; - } else if (e instanceof org.apache.atlas.repository.graphdb.AtlasSchemaViolationException) { - LOG.warn("{}: Continuing: {}", exceptionClassName, e.getMessage()); - return new TopicPartitionOffsetResult(kafkaMsg.getTopicPartition(), kafkaMsg.getOffset()); - } else if (exceptionClassName.equals(EXCEPTION_CLASS_NAME_JANUSGRAPH_EXCEPTION) - || exceptionClassName.equals(EXCEPTION_CLASS_NAME_PERMANENTLOCKING_EXCEPTION)) { - LOG.warn("{}: Offset: {}: Pausing & retry: Try: {}: Pause: {} ms. {}", - exceptionClassName, kafkaMsg.getOffset(), numRetries, adaptiveWaiter.getWaitDuration(), e.getMessage()); - - adaptiveWaiter.pause((Exception) e); - } else if (e instanceof java.lang.IllegalStateException || e instanceof NullPointerException) { - return null; - } else { - LOG.warn("Error handling message", e); + } - try { - LOG.info("Sleeping for {} ms before retry", consumerRetryInterval); + if (isTransactionRelatedError(e)) { + // Retryable exceptions: perform rollback, clear cache, and retry + LOG.warn("{}: Offset: {}: Retryable exception: Try: {}/{}: Pause: {} ms. {}", + exceptionClassName, kafkaMsg.getOffset(), numRetries + 1, maxRetries, adaptiveWaiter.getWaitDuration(), e.getMessage()); - Thread.sleep(consumerRetryInterval); - } catch (InterruptedException ie) { - LOG.error("Notification consumer thread sleep interrupted"); + try { + LOG.info("Parallel Processing-{}: Rolling back failed transaction (attempt {}/{}) to enable fresh retry", + threadName, numRetries + 1, maxRetries); + AtlasGraphProvider.getGraphInstance().rollback(); + } catch (Exception rollbackEx) { + LOG.warn("Parallel Processing-{}: Error during transaction rollback: {}", + threadName, rollbackEx.getMessage()); } + } else { + LOG.warn("Error handling message", e); + LOG.info("Sleeping for {} ms before retry", adaptiveWaiter.getWaitDuration()); } + + // Adaptive wait before retry + adaptiveWaiter.pause((Exception) e); } finally { RequestContext.clear(); } @@ -1160,10 +1249,10 @@ private Authentication getAuthenticationForUser(String userName) { return ret; } - private void recordFailedMessages(List failedMessages) { + private void recordFailedMessages(String topic, List failedMessages) { //logging failed messages for (String message : failedMessages) { - failedMessageLog.error("[DROPPED_NOTIFICATION] {}", message); + failedMessageLog.error("[{}-DROPPED_NOTIFICATION] {}", topic, message); } failedMessages.clear(); diff --git a/webapp/src/main/java/org/apache/atlas/notification/preprocessor/NotificationPreProcessor.java b/webapp/src/main/java/org/apache/atlas/notification/preprocessor/NotificationPreProcessor.java new file mode 100644 index 00000000000..47509da8ac7 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/preprocessor/NotificationPreProcessor.java @@ -0,0 +1,1814 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.atlas.notification.preprocessor; + +import org.apache.atlas.AtlasConfiguration; +import org.apache.atlas.AtlasException; +import org.apache.atlas.exception.AtlasBaseException; +import org.apache.atlas.kafka.AtlasKafkaMessage; +import org.apache.atlas.kafka.NotificationProvider; +import org.apache.atlas.listener.ChangedTypeDefs; +import org.apache.atlas.listener.TypeDefChangeListener; +import org.apache.atlas.model.instance.AtlasEntity; +import org.apache.atlas.model.instance.AtlasObjectId; +import org.apache.atlas.model.notification.HookNotification; +import org.apache.atlas.model.notification.HookNotification.EntityCreateRequestV2; +import org.apache.atlas.model.notification.HookNotification.EntityDeleteRequestV2; +import org.apache.atlas.model.notification.HookNotification.EntityPartialUpdateRequestV2; +import org.apache.atlas.model.notification.HookNotification.EntityUpdateRequestV2; +import org.apache.atlas.model.notification.MessageSource; +import org.apache.atlas.model.typedef.AtlasBaseTypeDef; +import org.apache.atlas.notification.AbstractNotification; +import org.apache.atlas.notification.NotificationEntityProcessor; +import org.apache.atlas.notification.NotificationException; +import org.apache.atlas.notification.NotificationInterface; +import org.apache.atlas.notification.QualifiedNameRouter; +import org.apache.atlas.notification.TopicPartitionOffsetResult; +import org.apache.atlas.notification.pc.Ticket; +import org.apache.atlas.type.AtlasTypeRegistry; +import org.apache.atlas.util.AtlasMetricsUtil; +import org.apache.atlas.util.AtlasMetricsUtil.NotificationProcessorStats; +import org.apache.atlas.utils.AtlasPerfTracer; +import org.apache.atlas.v1.model.instance.Referenceable; +import org.apache.atlas.v1.model.notification.HookNotificationV1.EntityCreateRequest; +import org.apache.atlas.v1.model.notification.HookNotificationV1.EntityDeleteRequest; +import org.apache.atlas.v1.model.notification.HookNotificationV1.EntityPartialUpdateRequest; +import org.apache.atlas.v1.model.notification.HookNotificationV1.EntityUpdateRequest; +import org.apache.commons.configuration2.Configuration; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_FAILEDCACHESIZE_PROPERTY; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_RETRIES_PROPERTY; +import static org.apache.atlas.notification.preprocessor.EntityPreprocessor.getQualifiedName; + +public class NotificationPreProcessor implements NotificationEntityProcessor, TypeDefChangeListener { + private static final Logger LOG = LoggerFactory.getLogger("NOTIFICATION_PROCESSOR"); + private static final Logger PERF_LOG = AtlasPerfTracer.getPerfLogger(NotificationPreProcessor.class); + + private final int metadataTopicCount; + private final int lineageTopicCount; + private final int maxRetries; + private final boolean lineageTopicEnabled; + private final Configuration configuration; + private final NotificationInterface notificationInterface; + private final QualifiedNameRouter metadataRouter; + private final QualifiedNameRouter lineageRouter; + private final AtlasMetricsUtil metricsUtil; + private final AtlasTypeRegistry typeRegistry; + private final List failedMessages; + private final int failedMsgCacheSize; + private final Map processTypeCache = new ConcurrentHashMap<>(); + private Set allProcessTypes; + private Logger failedMessageLog; + private static final java.util.regex.Pattern GUID_PATTERN = java.util.regex.Pattern.compile("^[a-fA-F0-9-]{36}$"); + public static final String ATTRIBUTE_GUID = "guid"; + public static final String ATTRIBUTE_NAME = "name"; + public static final String ATTRIBUTE_QUALIFIED_NAME = "qualifiedName"; + /** + * Maximum size of ThreadLocal rename cache per thread. + * Uses LRU eviction policy - when cache exceeds this size, oldest entries are automatically removed. + */ + private static final int MAX_RENAME_CACHE_SIZE = 1000; + + // ThreadLocal rename routing map - each thread maintains its own rename chain mappings + private static final ThreadLocal> renameRoutingInfoMap = + ThreadLocal.withInitial(() -> new LinkedHashMap(16, 0.75f, true) { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + return size() > MAX_RENAME_CACHE_SIZE; + } + }); + + public NotificationPreProcessor(Configuration configuration, AtlasMetricsUtil metricsUtil, + AtlasTypeRegistry typeRegistry, Logger failedMessageLogger) { + this.configuration = configuration; + this.failedMessageLog = failedMessageLogger; + this.notificationInterface = NotificationProvider.get(); + this.metadataTopicCount = this.configuration.getInt("atlas.notification.processor.metadata.topic.count", 5); + this.lineageTopicCount = this.configuration.getInt("atlas.notification.processor.lineage.topic.count", 3); + this.lineageTopicEnabled = this.configuration.getBoolean("atlas.notification.processor.lineage.topic.enabled", true); + this.maxRetries = this.configuration.getInt(CONSUMER_RETRIES_PROPERTY, 3); + this.failedMsgCacheSize = this.configuration.getInt(CONSUMER_FAILEDCACHESIZE_PROPERTY, 1); + this.metadataRouter = new QualifiedNameRouter(metadataTopicCount, AtlasConfiguration.ATLAS_METADATA_TOPIC_PREFIX.getString()); + this.lineageRouter = new QualifiedNameRouter(lineageTopicCount, AtlasConfiguration.ATLAS_LINEAGE_TOPIC_PREFIX.getString()); + this.metricsUtil = metricsUtil; + this.typeRegistry = typeRegistry; + this.failedMessages = new ArrayList<>(); + + // Initialize type registry early in constructor to avoid null pointer issues + try { + initializeTypeRegistry(); + } catch (AtlasException e) { + LOG.warn("Failed to initialize type registry during construction: {}. Will use heuristics fallback only.", e.getMessage()); + // Initialize to empty unmodifiable set to avoid null pointer exceptions + this.allProcessTypes = Collections.emptySet(); + } + } + + @Override + public TopicPartitionOffsetResult handleMessage(AtlasKafkaMessage kafkaMsg) { + return handleMessage(new Ticket(kafkaMsg)); + } + + @Override + public TopicPartitionOffsetResult collectResults() { + return null; + } + + @Override + public void shutdown() { + } + + private TopicPartitionOffsetResult handleMessage(Ticket ticket) { + AtlasPerfTracer perf = null; + AtlasKafkaMessage kafkaMsg = ticket.getMessage(); + HookNotification message = kafkaMsg.getMessage(); + long startTime = System.currentTimeMillis(); + NotificationProcessorStats stats = new NotificationProcessorStats(); + + if (AtlasPerfTracer.isPerfTraceEnabled(PERF_LOG)) { + perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, message.getType().name()); + } + + for (int numRetries = 0; numRetries < maxRetries; numRetries++) { + try { + // Extract original source from JSON before deserializing + NotificationMetadata notificationMetadata = buildNotificationMetadataFromMessage(kafkaMsg, kafkaMsg.getTopic(), kafkaMsg.getOffset()); + + routeNotification(message, notificationMetadata); + + break; + } catch (Exception e) { + LOG.error("Error processing notification: {}", e.getMessage(), e); + + if (numRetries == (maxRetries - 1)) { + String strMessage = AbstractNotification.getMessageJson(message); + + LOG.warn("Offset: {}: Max retries: {} exceeded for message {}", kafkaMsg.getOffset(), maxRetries, strMessage, e); + + stats.setFailed(true); + + failedMessages.add(strMessage); + + if (failedMessages.size() >= failedMsgCacheSize) { + recordFailedMessages(kafkaMsg.getTopic(), failedMessages); + } + + return new TopicPartitionOffsetResult(kafkaMsg.getTopicPartition(), kafkaMsg.getOffset()); + } + } finally { + AtlasPerfTracer.log(perf); + stats.setProcessingTimeMs(System.currentTimeMillis() - startTime); + metricsUtil.onNotificationProcessorComplete(kafkaMsg.getTopic(), kafkaMsg.getPartition(), kafkaMsg.getOffset(), stats); + } + } + return new TopicPartitionOffsetResult(kafkaMsg.getTopicPartition(), kafkaMsg.getOffset()); + } + + private NotificationMetadata buildNotificationMetadataFromMessage(AtlasKafkaMessage kafkaMessage, String sourceTopic, long sourceOffset) { + NotificationMetadata notificationMetadata = new NotificationMetadata(); + MessageSource source = new MessageSource(); + String sourceTopicAndOffset = String.format("%s-%s", sourceTopic, sourceOffset); + + try { + String originalSourceName = null; + if (kafkaMessage != null && kafkaMessage.getSource() != null) { + originalSourceName = kafkaMessage.getSource(); + } + if (StringUtils.isNotBlank(originalSourceName)) { + source.setSource(String.format("%s-%s", sourceTopicAndOffset, originalSourceName)); + LOG.debug("Extracted original source from JSON: {}", originalSourceName); + } else { + LOG.debug("No source found in JSON message, using fallback default source: {}", sourceTopicAndOffset); + source.setSource(sourceTopicAndOffset); + } + + if (kafkaMessage != null) { + notificationMetadata.setMsgCreationTime(kafkaMessage.getMsgCreated()); + } + } catch (Exception e) { + LOG.debug("Error parsing JSON to extract source, using fallback default source {}: {}", sourceTopicAndOffset, e.getMessage()); + source.setSource(sourceTopicAndOffset); + } + + notificationMetadata.setSource(source); + return notificationMetadata; + } + + /** + * Converts V1 notifications to V2 format for unified processing. + * This eliminates the need for separate V1/V2 processing logic. + */ + private HookNotification convertV1ToV2(HookNotification notification) { + try { + switch (notification.getType()) { + case ENTITY_CREATE: { + EntityCreateRequest v1Request = (EntityCreateRequest) notification; + // Convert V1 to V2 keeping ALL referred entities (filtering happens later by topic) + AtlasEntity.AtlasEntitiesWithExtInfo v2Entities = convertReferenceablesToAtlasEntities(v1Request.getEntities()); + return new EntityCreateRequestV2(v1Request.getUser(), v2Entities); + } + case ENTITY_FULL_UPDATE: { + EntityUpdateRequest v1Request = (EntityUpdateRequest) notification; + // Convert V1 to V2 keeping ALL referred entities (filtering happens later by topic) + AtlasEntity.AtlasEntitiesWithExtInfo v2Entities = convertReferenceablesToAtlasEntities(v1Request.getEntities()); + return new EntityUpdateRequestV2(v1Request.getUser(), v2Entities); + } + case ENTITY_PARTIAL_UPDATE: + case ENTITY_DELETE: + // These don't have bulk entity lists, so we'll keep them as-is for now + // They'll be handled separately in the routing logic + return notification; + + default: + // Already V2 or other types + return notification; + } + } catch (Exception e) { + LOG.warn("Failed to convert V1 notification to V2 format, will process as V1: {}", e.getMessage()); + return notification; + } + } + + /** + * Converts a list of V1 Referenceable objects to V2 AtlasEntitiesWithExtInfo. + * This handles negative GUID processing during conversion. + */ + private AtlasEntity.AtlasEntitiesWithExtInfo convertReferenceablesToAtlasEntities(List referenceables) { + List entities = new ArrayList<>(); + Map referredEntities = new HashMap<>(); + + if (referenceables != null) { + // First, build the GUID-to-unique-attributes map from V1 entities + Map> guidToUniqueAttrsMap = buildGuidToUniqueAttributesMapFromReferenceables(referenceables); + + for (Referenceable referenceable : referenceables) { + AtlasEntity entity = convertReferenceableToAtlasEntity(referenceable, guidToUniqueAttrsMap); + entities.add(entity); + } + } + + return new AtlasEntity.AtlasEntitiesWithExtInfo(entities, new AtlasEntity.AtlasEntityExtInfo(referredEntities)); + } + + /** + * Builds a map from GUID to uniqueAttributes for V1 Referenceable entities. + */ + private Map> buildGuidToUniqueAttributesMapFromReferenceables(List referenceables) { + Map> guidToUniqueAttrsMap = new HashMap<>(); + + for (Referenceable entity : referenceables) { + if (entity.getValues() != null) { + String guid = (String) entity.getValues().get(ATTRIBUTE_GUID); + if (guid != null && guid.startsWith("-")) { + Map uniqueAttrs = new HashMap<>(); + // Common unique attributes + Object qualifiedName = entity.getValues().get(ATTRIBUTE_QUALIFIED_NAME); + Object name = entity.getValues().get(ATTRIBUTE_NAME); + + if (qualifiedName != null) { + uniqueAttrs.put(ATTRIBUTE_QUALIFIED_NAME, qualifiedName); + } + if (name != null) { + uniqueAttrs.put(ATTRIBUTE_NAME, name); + } + + if (!uniqueAttrs.isEmpty()) { + guidToUniqueAttrsMap.put(guid, uniqueAttrs); + } + } + } + } + + return guidToUniqueAttrsMap; + } + + /** + * Converts a single V1 Referenceable to V2 AtlasEntity. + * This handles negative GUID processing during conversion. + */ + private AtlasEntity convertReferenceableToAtlasEntity(Referenceable referenceable, Map> guidToUniqueAttrsMap) { + AtlasEntity entity = new AtlasEntity(); + + entity.setTypeName(referenceable.getTypeName()); + + if (referenceable.getValues() != null) { + // Extract GUID if present + Object guidObj = referenceable.getValues().get(ATTRIBUTE_GUID); + if (guidObj != null) { + entity.setGuid(guidObj.toString()); + } + + // Convert all other attributes + Map attributes = new HashMap<>(); + Map relationshipAttributes = new HashMap<>(); + + for (Map.Entry entry : referenceable.getValues().entrySet()) { + String key = entry.getKey(); + Object value = entry.getValue(); + + if (ATTRIBUTE_GUID.equals(key)) { + // Already handled above + continue; + } + + // Simple heuristic: if the value is a Referenceable or contains entity references, + // it might be a relationship attribute. Otherwise, treat as regular attribute. + if (isLikelyRelationshipAttribute(value)) { + relationshipAttributes.put(key, convertAttributeValue(value, guidToUniqueAttrsMap, true)); // true = is relationship attribute + } else { + attributes.put(key, convertAttributeValue(value, guidToUniqueAttrsMap, false)); // false = regular attribute + } + } + + entity.setAttributes(attributes); + if (!relationshipAttributes.isEmpty()) { + entity.setRelationshipAttributes(relationshipAttributes); + } + } + + // Convert classifications if present + if (referenceable.getTraits() != null && !referenceable.getTraits().isEmpty()) { + // For now, we'll skip classification conversion to keep the implementation simple + // In a full implementation, we would need to convert V1 traits to V2 AtlasClassification objects + LOG.debug("Skipping classification conversion for entity {} - {} traits present", + referenceable.getTypeName(), referenceable.getTraits().size()); + } + + LOG.debug("Converted V1 Referenceable {} to V2 AtlasEntity", referenceable.getTypeName()); + + return entity; + } + + /** + * Heuristic to determine if an attribute is likely a relationship attribute. + */ + private boolean isLikelyRelationshipAttribute(Object value) { + if (value instanceof Referenceable) { + return true; + } + if (value instanceof List) { + List list = (List) value; + return !list.isEmpty() && list.get(0) instanceof Referenceable; + } + return false; + } + + /** + * Converts attribute values during V1 to V2 conversion. + * Handles negative GUID processing for relationship attributes. + */ + private Object convertAttributeValue(Object value, Map> guidToUniqueAttrsMap, boolean isRelationshipAttribute) { + if (value instanceof Referenceable) { + Referenceable ref = (Referenceable) value; + // Convert to AtlasObjectId + AtlasObjectId objectId = new AtlasObjectId(); + objectId.setTypeName(ref.getTypeName()); + + if (ref == null || ref.getValues() == null) { + return objectId; + } + + Object guidObj = ref.getValues().get(ATTRIBUTE_GUID); + String guid = guidObj != null ? guidObj.toString() : null; + + // Handle negative GUIDs for relationship attributes + if (isRelationshipAttribute && guid != null && guid.startsWith("-")) { + // Look up uniqueAttributes from the original V1 entities + Map uniqueAttrs = guidToUniqueAttrsMap.get(guid); + if (uniqueAttrs != null && !uniqueAttrs.isEmpty()) { + // Create AtlasObjectId with uniqueAttributes only (no GUID) + objectId.setUniqueAttributes(uniqueAttrs); + LOG.debug("Replaced negative GUID {} with uniqueAttributes {} for type {} during V1 to V2 conversion", + guid, uniqueAttrs, ref.getTypeName()); + } else { + // Fallback: extract uniqueAttributes from the referenceable itself + Map uniqueAttributes = new HashMap<>(); + Object qualifiedName = ref.getValues().get(ATTRIBUTE_QUALIFIED_NAME); + Object name = ref.getValues().get(ATTRIBUTE_NAME); + + if (qualifiedName != null) { + uniqueAttributes.put(ATTRIBUTE_QUALIFIED_NAME, qualifiedName); + } + if (name != null) { + uniqueAttributes.put(ATTRIBUTE_NAME, name); + } + + if (!uniqueAttributes.isEmpty()) { + objectId.setUniqueAttributes(uniqueAttributes); + LOG.debug("Used fallback uniqueAttributes {} for negative GUID {} during V1 to V2 conversion", + uniqueAttributes, guid); + } else { + // Last resort: keep the negative GUID + objectId.setGuid(guid); + LOG.warn("Could not resolve negative GUID {} for type {} during V1 to V2 conversion", + guid, ref.getTypeName()); + } + } + } else { + // Regular GUID or non-relationship attribute - preserve as-is + return value; + } + } else if (value instanceof List) { + List list = (List) value; + List convertedList = new ArrayList<>(); + for (Object item : list) { + convertedList.add(convertAttributeValue(item, guidToUniqueAttrsMap, isRelationshipAttribute)); + } + return convertedList; + } else if (value instanceof Map) { + Map map = (Map) value; + Map convertedMap = new HashMap<>(); + for (Map.Entry entry : map.entrySet()) { + convertedMap.put(entry.getKey(), convertAttributeValue(entry.getValue(), guidToUniqueAttrsMap, isRelationshipAttribute)); + } + return convertedMap; + } + + return value; + } + + // Initialize local type registry (primary method for type hierarchy checking) + private void initializeTypeRegistry() throws AtlasException { + if (typeRegistry != null) { + try { + Set processTypes = typeRegistry.getAllEntityTypes().stream() + .filter(x -> x.getTypeName().equals(AtlasBaseTypeDef.ATLAS_TYPE_PROCESS) || x.isSubTypeOf(AtlasBaseTypeDef.ATLAS_TYPE_PROCESS)) + .map(x -> x.getTypeName()) + .collect(Collectors.toSet()); + allProcessTypes = Collections.unmodifiableSet(processTypes); + LOG.info("Successfully initialized process types cache with {} types (unmodifiable)", allProcessTypes.size()); + } catch (Exception e) { + LOG.warn("Failed to build process types cache: {}. Initializing empty unmodifiable set.", e.getMessage()); + allProcessTypes = Collections.emptySet(); + } + } else { + LOG.warn("Type registry is null. Initializing empty unmodifiable process types set."); + allProcessTypes = Collections.emptySet(); + } + } + + /** + * Determines if an entity is a lineage entity based on its type. + * An entity is considered lineage if its type is "Process" or a subtype of "Process". + */ + private boolean isLineageEntity(Object entity) { + String entityTypeName = getEntityTypeName(entity); + + // Check cache first to avoid repeated API calls + Boolean cachedResult = processTypeCache.get(entityTypeName); + if (cachedResult != null) { + return cachedResult; + } + + boolean isProcessType = isProcessOrSubtype(entityTypeName); + + // Cache the result for future use + processTypeCache.put(entityTypeName, isProcessType); + + return isProcessType; + } + + /** + * Checks if the given type name is "Process" or a subtype of "Process" using AtlasTypeRegistry + */ + private boolean isProcessOrSubtype(String typeName) { + // Direct match for Process + if (AtlasBaseTypeDef.ATLAS_TYPE_PROCESS.equals(typeName)) { + return true; + } + + if (typeRegistry != null && allProcessTypes != null) { + try { + if (typeRegistry.isRegisteredType(typeName)) { + boolean isTypeOrSubTypeOfProcess = allProcessTypes.contains(typeName); + LOG.debug("Local type registry check for '{}': isSubTypeOfProcess = {}", typeName, isTypeOrSubTypeOfProcess); + return isTypeOrSubTypeOfProcess; + } else { + LOG.debug("Type '{}' not found in local type registry", typeName); + } + } catch (ClassCastException e) { + LOG.debug("Type '{}' is not an entity type in local registry: {}", + typeName, e.getMessage()); + } + } + + return false; + } + + /** + * Gets the type name from different entity types. + */ + private String getEntityTypeName(Object entity) { + if (entity instanceof AtlasEntity) { + return ((AtlasEntity) entity).getTypeName(); + } else if (entity instanceof Referenceable) { + return ((Referenceable) entity).getTypeName(); + } else if (entity instanceof AtlasObjectId) { + return ((AtlasObjectId) entity).getTypeName(); + } else if (entity instanceof EntityPartialUpdateRequest) { + return ((EntityPartialUpdateRequest) entity).getTypeName(); + } else if (entity instanceof EntityDeleteRequest) { + return ((EntityDeleteRequest) entity).getTypeName(); + } + return "unknown"; + } + + /** + * Extracts entity routing information from different types of HookNotifications. + * After V1 to V2 conversion, we mainly handle V2 types plus the V1 types that don't get converted. + * NOTE: This method does NOT process temporary GUIDs - that happens later per topic group. + */ + private List extractEntityRoutingInfos(HookNotification notification, NotificationMetadata notificationMetadata) { + switch (notification.getType()) { + // V1 types that don't get converted to V2 (single entity operations) + case ENTITY_PARTIAL_UPDATE: + return extractRoutingInfoFromPartialUpdateRequest((EntityPartialUpdateRequest) notification, notificationMetadata); + case ENTITY_DELETE: + return extractRoutingInfoFromDeleteRequest((EntityDeleteRequest) notification, notificationMetadata); + // V2 types (including converted V1 bulk operations) + case ENTITY_CREATE_V2: + return extractRoutingInfoFromCreateRequestV2((EntityCreateRequestV2) notification, notificationMetadata); + case ENTITY_PARTIAL_UPDATE_V2: + return extractRoutingInfoFromPartialUpdateRequestV2((EntityPartialUpdateRequestV2) notification, notificationMetadata); + case ENTITY_FULL_UPDATE_V2: + return extractRoutingInfoFromUpdateRequestV2((EntityUpdateRequestV2) notification, notificationMetadata); + case ENTITY_DELETE_V2: + return extractRoutingInfoFromDeleteRequestV2((EntityDeleteRequestV2) notification, notificationMetadata); + + // These should not occur after conversion, but handle them defensively + case ENTITY_CREATE: + case ENTITY_FULL_UPDATE: + LOG.warn("Unexpected V1 bulk notification type after conversion: {}", notification.getType()); + // Fall back to V1 processing if somehow we get here + if (notification.getType() == HookNotification.HookNotificationType.ENTITY_CREATE) { + return extractRoutingInfoFromCreateRequest((EntityCreateRequest) notification, notificationMetadata); + } else { + return extractRoutingInfoFromUpdateRequest((EntityUpdateRequest) notification, notificationMetadata); + } + + default: + LOG.warn("Unknown notification type: {}", notification.getType()); + } + + return Collections.emptyList(); + } + + /** + * Extracts routing infos for entities from EntityCreateRequest (V1). + * Note: This should rarely be called since V1 CREATE requests are converted to V2. + * Temporary GUID processing is deferred until topic grouping. + */ + private List extractRoutingInfoFromCreateRequest(EntityCreateRequest request, NotificationMetadata notificationMetadata) { + List routingInfos = new ArrayList<>(); + if (request.getEntities() != null) { + for (Referenceable entity : request.getEntities()) { + routingInfos.add(createEntityRoutingInfo(request, entity, notificationMetadata)); + } + } + return routingInfos; + } + + /** + * Extracts routing infos for entities from EntityUpdateRequest (V1). + * Note: This should rarely be called since V1 UPDATE requests are converted to V2. + * Temporary GUID processing is deferred until topic grouping. + */ + private List extractRoutingInfoFromUpdateRequest(EntityUpdateRequest request, NotificationMetadata notificationMetadata) { + List routingInfos = new ArrayList<>(); + if (request.getEntities() != null) { + for (Referenceable entity : request.getEntities()) { + routingInfos.add(createEntityRoutingInfo(request, entity, notificationMetadata)); + } + } + return routingInfos; + } + + /** + * Extracts routing infos for entity from EntityPartialUpdateRequest (V1). + */ + private List extractRoutingInfoFromPartialUpdateRequest(EntityPartialUpdateRequest request, NotificationMetadata notificationMetadata) { + // For partial updates, we only have type and attribute, create a routing key from typename + //String routingKey = request.getTypeName(); + String routingKey = getRoutingKey(request.getEntity(), notificationMetadata); + String targetTopic = metadataRouter.getTargetTopic(routingKey); + return Collections.singletonList(new EntityRoutingInfo(request, routingKey, targetTopic)); + } + + /** + * Extracts routing infos for entity from EntityDeleteRequest (V1). + */ + private List extractRoutingInfoFromDeleteRequest(EntityDeleteRequest request, NotificationMetadata notificationMetadata) { + // For deletes, create routing key from typename and attribute value + String routingKey = request.getTypeName() + ":" + request.getAttributeValue(); + String targetTopic = metadataRouter.getTargetTopic(routingKey); + return Collections.singletonList(new EntityRoutingInfo(request, routingKey, targetTopic)); + } + + /** + * Extracts routing infos for entities from EntityCreateRequestV2. + * Temporary GUID processing is deferred until topic grouping. + */ + private List extractRoutingInfoFromCreateRequestV2(EntityCreateRequestV2 request, NotificationMetadata notificationMetadata) { + List routingInfos = new ArrayList<>(); + if (request.getEntities() == null) { + return routingInfos; + } + if (request.getEntities().getEntities() != null) { + for (AtlasEntity entity : request.getEntities().getEntities()) { + routingInfos.add(createEntityRoutingInfo(request, entity, notificationMetadata)); + } + } + + return routingInfos; + } + + /** + * Extracts routing infos for entities from EntityUpdateRequestV2. + * Temporary GUID processing is deferred until topic grouping. + */ + private List extractRoutingInfoFromUpdateRequestV2(EntityUpdateRequestV2 request, NotificationMetadata notificationMetadata) { + List routingInfos = new ArrayList<>(); + if (request.getEntities() == null) { + return routingInfos; + } + + if (request.getEntities().getEntities() != null) { + for (AtlasEntity entity : request.getEntities().getEntities()) { + routingInfos.add(createEntityRoutingInfo(request, entity, notificationMetadata)); + } + } + + return routingInfos; + } + + /** + * Extracts routing infos for entity from EntityPartialUpdateRequestV2. + * Detects and records rename events in ThreadLocal rename map. + * Temporary GUID processing is deferred until topic grouping. + */ + private List extractRoutingInfoFromPartialUpdateRequestV2(EntityPartialUpdateRequestV2 request, NotificationMetadata notificationMetadata) { + AtlasObjectId entityId = request.getEntityId(); + + // Detect and record rename events + if (entityId != null && request.getEntity() != null && request.getEntity().getEntity() != null) { + String oldEntityName = getRoutingQualifiedName(entityId); + String renamedEntityName = getRoutingQualifiedName(request.getEntity().getEntity()); + + // Validate that this is actually a rename (qualifiedName changed) + if (oldEntityName != null && renamedEntityName != null && !oldEntityName.equals(renamedEntityName)) { + long renamedEventTimestamp = notificationMetadata.getMsgCreationTime(); + Map renameMap = renameRoutingInfoMap.get(); + + // Record rename (with chain resolution) + if (renameMap.containsKey(oldEntityName)) { + // Chain: tb1 → tb1_new → tb1_latest + renameMap.put(renamedEntityName, renameMap.get(oldEntityName)); + LOG.debug("Detected rename chain: {} → {} (original: {}, timestamp: {})", + oldEntityName, renamedEntityName, renameMap.get(oldEntityName).getOriginalEntityName(), renamedEventTimestamp); + } else { + // First rename: tb1 → tb1_new + renameMap.put(renamedEntityName, new RenameRoutingInfo(oldEntityName, renamedEventTimestamp)); + LOG.debug("Detected rename: {} → {} (timestamp: {})", oldEntityName, renamedEntityName, renamedEventTimestamp); + } + } + } + + // Extract entity for routing + List routingInfos = new ArrayList<>(); + if (request.getEntity() != null && request.getEntity().getEntity() != null) { + AtlasEntity entity = request.getEntity().getEntity(); + routingInfos.add(createEntityRoutingInfo(request, entity, notificationMetadata)); + } + return routingInfos; + } + + /** + * Extracts routing infos for entities from EntityDeleteRequestV2. + */ + private List extractRoutingInfoFromDeleteRequestV2(EntityDeleteRequestV2 request, NotificationMetadata notificationMetadata) { + List routingInfos = new ArrayList<>(); + if (request.getEntities() != null) { + for (AtlasObjectId entity : request.getEntities()) { + String routingKey = getRoutingKey(entity, notificationMetadata); + String targetTopic = metadataRouter.getTargetTopic(routingKey); + routingInfos.add(new EntityRoutingInfo(entity, routingKey, targetTopic)); + } + } + return routingInfos; + } + + private EntityRoutingInfo createEntityRoutingInfo(HookNotification request, Object entity, NotificationMetadata notificationMetadata) { + Set referencedGUIDs = Collections.emptySet(); + // Extract referenced GUIDs BEFORE processing relationship attributes + if (request instanceof EntityCreateRequest || request instanceof EntityUpdateRequest) { + referencedGUIDs = extractReferencedGUIDsFromReferenceable((Referenceable) entity); + } else if (request instanceof EntityCreateRequestV2 || request instanceof EntityUpdateRequestV2 || request instanceof EntityPartialUpdateRequestV2) { + referencedGUIDs = extractReferencedGUIDsFromEntity((AtlasEntity) entity); + } + + // Skip temporary GUID processing here - will be done per topic group later + String routingKey = getRoutingKey(entity, notificationMetadata); + String targetTopic = metadataRouter.getTargetTopic(routingKey); + + return new EntityRoutingInfo(entity, routingKey, targetTopic, referencedGUIDs); + } + + private String getRoutingQualifiedName(Object obj) { + String qualifiedName = getQualifiedName(obj); + if (StringUtils.isNotBlank(qualifiedName)) { + // Skip timestamp or anything after metadata namespace (part after ':' if any) + String qualifiedNameToRoute = qualifiedName; + int atIndex = qualifiedName.indexOf(':'); + if (atIndex != -1) { + qualifiedNameToRoute = qualifiedName.substring(0, atIndex); + } + return qualifiedNameToRoute; + } + return null; + } + + /** + * Processes relationship attributes for entities to handle negative GUIDs. + * For V2 entities (AtlasEntity). Only strips temporary GUIDs if the referred entity is not present in the message. + */ + private void processEntityRelationshipAttributes(AtlasEntity entity, Map> guidToUniqueAttrsMap, Set availableEntityGUIDs) { + if (entity == null) { + return; + } + + // Process ONLY relationship attributes - not regular attributes + // Only replace negative GUIDs in relationshipAttributes if the referred entity is not present in the message + if (entity.getRelationshipAttributes() != null) { + for (Map.Entry entry : entity.getRelationshipAttributes().entrySet()) { + Object value = entry.getValue(); + Object processedValue = processAttributeValue(value, guidToUniqueAttrsMap, availableEntityGUIDs); + if (processedValue != value) { + entry.setValue(processedValue); + } + } + } + + // DO NOT process regular attributes - negative GUIDs in regular attributes are intentional + // for newly created entities that haven't been assigned permanent GUIDs yet + } + + /** + * Recursively processes attribute values to handle negative GUIDs. + * Only strips temporary GUIDs if isRelationshipAttribute is true AND the referred entity is not present in the message. + */ + private Object processAttributeValue(Object value, Map> guidToUniqueAttrsMap, Set availableEntityGUIDs) { + if (value == null) { + return null; + } + + if (value instanceof AtlasObjectId) { + return processAtlasObjectId((AtlasObjectId) value, guidToUniqueAttrsMap, availableEntityGUIDs); + } else if (value instanceof Referenceable) { + return processReferenceable((Referenceable) value, guidToUniqueAttrsMap, availableEntityGUIDs); + } else if (value instanceof List) { + return processListValue((List) value, guidToUniqueAttrsMap, availableEntityGUIDs); + } else if (value instanceof Map) { + return processMapValue((Map) value, guidToUniqueAttrsMap, availableEntityGUIDs); + } + + return value; + } + + /** + * Processes AtlasObjectId to handle negative GUIDs. + * Only strips temporary GUIDs if isRelationshipAttribute is true AND the referred entity is not present in the message. + */ + private Object processAtlasObjectId(AtlasObjectId objectId, Map> guidToUniqueAttrsMap, Set availableEntityGUIDs) { + String guid = objectId.getGuid(); + + // Only process negative GUIDs if we're in relationship attributes + if (guid != null && guid.startsWith("-")) { + // NEW LOGIC: Check if the referred entity with this GUID exists in the current message + if (availableEntityGUIDs.contains(guid)) { + // Referred entity is present in the message - keep the temporary GUID as-is + LOG.debug("Keeping temporary GUID {} as-is because referred entity is present in message for type {}", + guid, objectId.getTypeName()); + return objectId; + } + + // Referred entity is NOT present - strip the GUID and replace with unique attributes + // Check if uniqueAttributes are already provided + if (objectId.getUniqueAttributes() != null && !objectId.getUniqueAttributes().isEmpty()) { + // Remove the negative GUID and keep only uniqueAttributes + AtlasObjectId newObjectId = new AtlasObjectId(objectId.getTypeName(), objectId.getUniqueAttributes()); + LOG.debug("Stripped temporary GUID {} and kept existing uniqueAttributes for type {} (referred entity not in message)", + guid, objectId.getTypeName()); + return newObjectId; + } else { + // Look up uniqueAttributes from the original message + Map uniqueAttrs = guidToUniqueAttrsMap.get(guid); + if (uniqueAttrs != null && !uniqueAttrs.isEmpty()) { + AtlasObjectId newObjectId = new AtlasObjectId(objectId.getTypeName(), uniqueAttrs); + LOG.debug("Replaced temporary GUID {} with uniqueAttributes {} for type {} (referred entity not in message)", + guid, uniqueAttrs, objectId.getTypeName()); + return newObjectId; + } else { + LOG.warn("Could not find uniqueAttributes for temporary GUID {} of type {} (referred entity not in message)", + guid, objectId.getTypeName()); + } + } + } + + // For regular attributes or non-negative GUIDs, return as-is + return objectId; + } + + /** + * Processes Referenceable (V1) to handle negative GUIDs. + * Only strips temporary GUIDs if isRelationshipAttribute is true AND the referred entity is not present in the message. + */ + private Object processReferenceable(Referenceable referenceable, Map> guidToUniqueAttrsMap, Set availableEntityGUIDs) { + String guid = (String) referenceable.getValues().get(ATTRIBUTE_GUID); + + // Only process negative GUIDs if we're in relationship attributes + if (guid != null && guid.startsWith("-")) { + // NEW LOGIC: Check if the referred entity with this GUID exists in the current message + if (availableEntityGUIDs.contains(guid)) { + // Referred entity is present in the message - keep the temporary GUID as-is + LOG.debug("Keeping temporary GUID {} as-is because referred entity is present in message for V1 type {}", + guid, referenceable.getTypeName()); + return referenceable; + } + + // Referred entity is NOT present - strip the GUID and replace with unique attributes + // Look up uniqueAttributes from the original message + Map uniqueAttrs = guidToUniqueAttrsMap.get(guid); + if (uniqueAttrs != null && !uniqueAttrs.isEmpty()) { + // Create a new Referenceable without the negative GUID + Referenceable newRef = new Referenceable(referenceable.getTypeName()); + // Copy all attributes except guid + for (Map.Entry entry : referenceable.getValues().entrySet()) { + if (!ATTRIBUTE_GUID.equals(entry.getKey())) { + newRef.set(entry.getKey(), entry.getValue()); + } + } + // Add uniqueAttributes + for (Map.Entry entry : uniqueAttrs.entrySet()) { + newRef.set(entry.getKey(), entry.getValue()); + } + LOG.debug("Replaced temporary GUID {} with uniqueAttributes for V1 entity type {} (referred entity not in message)", + guid, referenceable.getTypeName()); + return newRef; + } else { + LOG.warn("Could not find uniqueAttributes for temporary GUID {} of V1 entity type {} (referred entity not in message)", + guid, referenceable.getTypeName()); + } + } + + // For regular attributes or non-negative GUIDs, return as-is + return referenceable; + } + + /** + * Processes List values recursively. + */ + private Object processListValue(List list, Map> guidToUniqueAttrsMap, Set availableEntityGUIDs) { + List newList = new ArrayList<>(); + boolean changed = false; + + for (Object item : list) { + Object processedItem = processAttributeValue(item, guidToUniqueAttrsMap, availableEntityGUIDs); + newList.add(processedItem); + if (processedItem != item) { + changed = true; + } + } + + return changed ? newList : list; + } + + /** + * Processes Map values recursively. + * Also detects Map-based entity references (with guid, typeName, etc.) and processes them like AtlasObjectId. + */ + private Object processMapValue(Map map, Map> guidToUniqueAttrsMap, Set availableEntityGUIDs) { + // Check if this Map represents an entity reference structure + if (isEntityReferenceMap(map)) { + return processMapAsEntityReference(map, guidToUniqueAttrsMap, availableEntityGUIDs); + } + + // Regular Map processing - recursively process each value + Map newMap = new HashMap<>(); + boolean changed = false; + + for (Map.Entry entry : map.entrySet()) { + Object processedValue = processAttributeValue(entry.getValue(), guidToUniqueAttrsMap, availableEntityGUIDs); + newMap.put(entry.getKey(), processedValue); + if (processedValue != entry.getValue()) { + changed = true; + } + } + + return changed ? newMap : map; + } + + /** + * Checks if a Map represents an entity reference (contains guid, typeName, etc.). + */ + private boolean isEntityReferenceMap(Map map) { + if (map == null) { + return false; + } + + // Check for common entity reference keys + boolean hasGuid = map.containsKey(ATTRIBUTE_GUID); + boolean hasTypeName = map.containsKey("typeName"); + + // A Map is considered an entity reference if it has both guid and typeName + // (this matches the structure shown in the user's example) + return hasGuid && hasTypeName; + } + + /** + * Processes a Map that represents an entity reference, similar to AtlasObjectId processing. + * Only strips temporary GUIDs if the referred entity is not present in the message. + */ + private Object processMapAsEntityReference(Map map, Map> guidToUniqueAttrsMap, Set availableEntityGUIDs) { + Object guidObj = map.get(ATTRIBUTE_GUID); + String guid = guidObj != null ? guidObj.toString() : null; + + // Only process negative GUIDs if we're in relationship attributes + if (guid != null && guid.startsWith("-")) { + // NEW LOGIC: Check if the referred entity with this GUID exists in the current message + if (availableEntityGUIDs.contains(guid)) { + // Referred entity is present in the message - keep the temporary GUID as-is + LOG.debug("Keeping temporary GUID {} as-is because referred entity is present in message for Map type {}", + guid, map.get("typeName")); + return map; + } + + // Referred entity is NOT present - strip the GUID and replace with unique attributes + // Check if uniqueAttributes are already provided + Object uniqueAttrsObj = map.get("uniqueAttributes"); + if (uniqueAttrsObj instanceof Map && !((Map) uniqueAttrsObj).isEmpty()) { + // Remove the negative GUID and keep only uniqueAttributes + Map newMap = new HashMap<>(); + for (Map.Entry entry : map.entrySet()) { + if (!ATTRIBUTE_GUID.equals(entry.getKey())) { + newMap.put(entry.getKey(), entry.getValue()); + } + } + LOG.debug("Stripped temporary GUID {} and kept existing uniqueAttributes for Map type {} (referred entity not in message)", + guid, map.get("typeName")); + return newMap; + } else { + // Look up uniqueAttributes from the original message + Map uniqueAttrs = guidToUniqueAttrsMap.get(guid); + if (uniqueAttrs != null && !uniqueAttrs.isEmpty()) { + Map newMap = new HashMap<>(); + for (Map.Entry entry : map.entrySet()) { + if (!ATTRIBUTE_GUID.equals(entry.getKey())) { + newMap.put(entry.getKey(), entry.getValue()); + } + } + // Add or replace uniqueAttributes + newMap.put("uniqueAttributes", uniqueAttrs); + LOG.debug("Replaced temporary GUID {} with uniqueAttributes {} for Map type {} (referred entity not in message)", + guid, uniqueAttrs, map.get("typeName")); + return newMap; + } else { + LOG.warn("Could not find uniqueAttributes for temporary GUID {} of Map type {} (referred entity not in message)", + guid, map.get("typeName")); + } + } + } + + // For regular attributes or non-negative GUIDs, return as-is + return map; + } + + /** + * Gets all V2 entities from the original request. + */ + private List getAllV2EntitiesFromRequest(Object request) { + List allEntities = new ArrayList<>(); + + if (request instanceof EntityCreateRequestV2) { + EntityCreateRequestV2 createRequest = (EntityCreateRequestV2) request; + if (createRequest.getEntities() != null) { + if (createRequest.getEntities().getEntities() != null) { + allEntities.addAll(createRequest.getEntities().getEntities()); + } + if (createRequest.getEntities().getReferredEntities() != null) { + allEntities.addAll(createRequest.getEntities().getReferredEntities().values()); + } + } + } else if (request instanceof EntityUpdateRequestV2) { + EntityUpdateRequestV2 updateRequest = (EntityUpdateRequestV2) request; + if (updateRequest.getEntities() != null) { + if (updateRequest.getEntities().getEntities() != null) { + allEntities.addAll(updateRequest.getEntities().getEntities()); + } + if (updateRequest.getEntities().getReferredEntities() != null) { + allEntities.addAll(updateRequest.getEntities().getReferredEntities().values()); + } + } + } else if (request instanceof EntityPartialUpdateRequestV2) { + EntityPartialUpdateRequestV2 partialRequest = (EntityPartialUpdateRequestV2) request; + if (partialRequest.getEntity() != null && partialRequest.getEntity().getEntity() != null) { + allEntities.add(partialRequest.getEntity().getEntity()); + if (partialRequest.getEntity().getReferredEntities() != null) { + allEntities.addAll(partialRequest.getEntity().getReferredEntities().values()); + } + } + } + + return allEntities; + } + + /** + * Builds a map from GUID to uniqueAttributes for V2 entities. + */ + private Map> buildGuidToUniqueAttributesMapV2(List entities) { + Map> guidToUniqueAttrsMap = new HashMap<>(); + + for (AtlasEntity entity : entities) { + String guid = entity.getGuid(); + if (guid != null && guid.startsWith("-") && entity.getAttributes() != null) { + Map uniqueAttrs = new HashMap<>(); + // Common unique attributes + Object qualifiedName = entity.getAttributes().get(ATTRIBUTE_QUALIFIED_NAME); + Object name = entity.getAttributes().get(ATTRIBUTE_NAME); + + if (qualifiedName != null) { + uniqueAttrs.put(ATTRIBUTE_QUALIFIED_NAME, qualifiedName); + } + if (name != null) { + uniqueAttrs.put(ATTRIBUTE_NAME, name); + } + + if (!uniqueAttrs.isEmpty()) { + guidToUniqueAttrsMap.put(guid, uniqueAttrs); + } + } + } + + return guidToUniqueAttrsMap; + } + + /** + * Extracts referenced GUIDs from an AtlasEntity's relationship attributes BEFORE temp GUID processing. + * This preserves GUID information that would otherwise be lost during processEntityRelationshipAttributes. + */ + private Set extractReferencedGUIDsFromEntity(AtlasEntity entity) { + Set referencedGUIDs = new HashSet<>(); + + if (entity == null) { + return referencedGUIDs; + } + + // Extract from relationship attributes only (this is where referenced entities are) + if (entity.getRelationshipAttributes() != null) { + for (Object value : entity.getRelationshipAttributes().values()) { + extractReferencesFromRelationshipAttr(value, referencedGUIDs); + } + } + + LOG.debug("Extracted {} referenced GUIDs from entity {} relationship attributes", + referencedGUIDs.size(), entity.getTypeName()); + + return referencedGUIDs; + } + + /** + * Extracts referenced GUIDs from a V1 Referenceable entity's attributes BEFORE conversion. + */ + private Set extractReferencedGUIDsFromReferenceable(Referenceable entity) { + Set referencedGUIDs = new HashSet<>(); + + if (entity != null) { + extractReferencesFromRelationshipAttr(entity, referencedGUIDs); + } + + return referencedGUIDs; + } + + /** + * Recursively extracts referenced GUIDs from any value type. + */ + private void extractReferencesFromRelationshipAttr(Object value, Set referencedGUIDs) { + if (value == null) { + return; + } + + if (value instanceof AtlasObjectId) { + AtlasObjectId objectId = (AtlasObjectId) value; + if (objectId.getGuid() != null) { + referencedGUIDs.add(objectId.getGuid()); + } + } else if (value instanceof Referenceable) { + Referenceable ref = (Referenceable) value; + Map values = ref.getValues(); + + if (values != null) { + Object guid = values.get(ATTRIBUTE_GUID); + if (guid != null) { + referencedGUIDs.add(guid.toString()); + } + // Recurse through all attributes of the Referenceable + for (Object attrValue : values.values()) { + extractReferencesFromRelationshipAttr(attrValue, referencedGUIDs); + } + } + } else if (value instanceof List) { + List list = (List) value; + for (Object item : list) { + extractReferencesFromRelationshipAttr(item, referencedGUIDs); + } + } else if (value instanceof Map) { + Map map = (Map) value; + for (Object mapValue : map.values()) { + extractReferencesFromRelationshipAttr(mapValue, referencedGUIDs); + } + } else if (value instanceof String) { + String strValue = (String) value; + if (isPotentialGuid(strValue)) { + referencedGUIDs.add(strValue); + } + } + } + + /** + * Filters referredEntities to only include those referenced by the given entities. + */ + private Map filterReferredEntities(Map allReferredEntities, + Set referencedGUIDs) { + if (allReferredEntities == null || allReferredEntities.isEmpty() || referencedGUIDs == null || referencedGUIDs.isEmpty()) { + return Collections.emptyMap(); + } + + Map filteredReferredEntities = new HashMap<>(); + Set processedGUIDs = new HashSet<>(); + + // First pass: directly referenced entities + for (String referencedGUID : referencedGUIDs) { + AtlasEntity referredEntity = allReferredEntities.get(referencedGUID); + if (referredEntity != null) { + filteredReferredEntities.put(referencedGUID, referredEntity); + processedGUIDs.add(referencedGUID); + } + } + + LOG.debug("Filtered referredEntities from {} to {} based on references", + allReferredEntities.size(), filteredReferredEntities.size()); + + return filteredReferredEntities; + } + + /** + * Processes entities for a specific topic group by applying intelligent temporary GUID handling. + * Only strips temporary GUIDs if the referred entity is not available in this topic group. + */ + private List processEntitiesForTopic(List entities, Set entityGuidsForTopic, HookNotification originalRequest) { + List processedEntities = new ArrayList<>(); + + // Build unique attributes map for GUID resolution + Map> guidToUniqueAttrsMap = buildGuidToUniqueAttributesMapFromOriginal(originalRequest); + + for (AtlasEntity entity : entities) { + // Create a deep copy to avoid modifying the original entity + AtlasEntity processedEntity = createEntityCopy(entity); + + // Apply temporary GUID processing + processEntityRelationshipAttributes(processedEntity, guidToUniqueAttrsMap, entityGuidsForTopic); + + processedEntities.add(processedEntity); + } + + return processedEntities; + } + + /** + * Builds GUID-to-unique-attributes map from the original notification request. + */ + private Map> buildGuidToUniqueAttributesMapFromOriginal(HookNotification originalRequest) { + List allEntities = getAllV2EntitiesFromRequest(originalRequest); + return buildGuidToUniqueAttributesMapV2(allEntities); + } + + /** + * Creates a deep copy of an AtlasEntity for safe modification. + */ + private AtlasEntity createEntityCopy(AtlasEntity original) { + // For simplicity, we'll create a new entity with the same data + // In a production environment, you might want to use a proper deep cloning mechanism + AtlasEntity copy = new AtlasEntity(); + copy.setGuid(original.getGuid()); + copy.setTypeName(original.getTypeName()); + copy.setStatus(original.getStatus()); + copy.setVersion(original.getVersion()); + copy.setCreatedBy(original.getCreatedBy()); + copy.setUpdatedBy(original.getUpdatedBy()); + copy.setCreateTime(original.getCreateTime()); + copy.setUpdateTime(original.getUpdateTime()); + + // Deep copy attributes + if (original.getAttributes() != null) { + copy.setAttributes(new HashMap<>(original.getAttributes())); + } + + // Deep copy relationship attributes + if (original.getRelationshipAttributes() != null) { + copy.setRelationshipAttributes(new HashMap<>(original.getRelationshipAttributes())); + } + + // Copy other fields as needed + if (original.getClassifications() != null) { + copy.setClassifications(new ArrayList<>(original.getClassifications())); + } + + return copy; + } + + /** + * Determines the routing key for an entity object. + * Extracts qualifiedName, name, and typeName from the entity and delegates to the main routing logic. + * + * @param entity the entity object (AtlasEntity, AtlasObjectId, or Referenceable) + * @param notificationMetadata metadata containing message creation time + * @return routing key for topic assignment + */ + private String getRoutingKey(Object entity, NotificationMetadata notificationMetadata) { + String qualifiedName = getRoutingQualifiedName(entity); + Object name = null; + String typeName = null; + String routingKey = null; + + // Extract name and typeName based on entity type + if (entity instanceof AtlasEntity) { + name = ((AtlasEntity) entity).getAttributes().get(ATTRIBUTE_NAME); + typeName = ((AtlasEntity) entity).getTypeName(); + } else if (entity instanceof AtlasObjectId) { + name = ((AtlasObjectId) entity).getUniqueAttributes().get(ATTRIBUTE_NAME); + typeName = ((AtlasObjectId) entity).getTypeName(); + } else if (entity instanceof Referenceable) { + name = ((Referenceable) entity).getValues().get(ATTRIBUTE_NAME); + typeName = ((Referenceable) entity).getTypeName(); + } + + switch (typeName) { + // Future: Add custom routing logic per entity type here + // case "hive_table": + // case "hive_process": + default: + // Use default logic for all entity types + routingKey = getRoutingKey(qualifiedName, typeName, name, notificationMetadata); + } + return routingKey; + } + + /** + * Determines the routing key for an entity based on its qualifiedName, with rename resolution. + * + * @param qualifiedName the entity's qualifiedName + * @param typeName the entity's type name + * @param name the entity's name attribute + * @param notificationMetadata metadata containing message creation time + * @return routing key for topic assignment + */ + private String getRoutingKey(String qualifiedName, String typeName, Object name, + NotificationMetadata notificationMetadata) { + if (StringUtils.isNotBlank(qualifiedName)) { + // Resolve through rename chain using ThreadLocal map + Map renameMap = renameRoutingInfoMap.get(); + + if (renameMap.containsKey(qualifiedName)) { + RenameRoutingInfo routingInfo = renameMap.get(qualifiedName); + + if (notificationMetadata.getMsgCreationTime() >= routingInfo.getRenamedEventTimestamp()) { + String originalName = routingInfo.getOriginalEntityName(); + LOG.debug("Routing renamed entity '{}' using original name '{}' (msgTime={}, renameTime={})", + qualifiedName, originalName, notificationMetadata.getMsgCreationTime(), routingInfo.getRenamedEventTimestamp()); + qualifiedName = originalName; + } else { + LOG.debug("Message for '{}' predates rename (msgTime={}, renameTime={}), using current name", + qualifiedName, notificationMetadata.getMsgCreationTime(), routingInfo.getRenamedEventTimestamp()); + } + } + + int namespaceIndex = qualifiedName.indexOf('@'); + String qNameWithoutNameSpace; + if (namespaceIndex != -1) { + qNameWithoutNameSpace = qualifiedName.substring(0, namespaceIndex); + } else { + qNameWithoutNameSpace = qualifiedName; + } + + // Extract routing key from qualifiedName (first two parts separated by '.') + String[] parts = qNameWithoutNameSpace.split("\\."); + + if (parts.length >= 2) { + return parts[0] + "." + parts[1]; + } else if (parts.length == 1) { + return parts[0]; + } + + // Fallback to original qualifiedName if empty after processing + return StringUtils.isNotBlank(qNameWithoutNameSpace) ? qNameWithoutNameSpace : qualifiedName; + } + + // Fallback to name or typeName if qualifiedName is not available + return name != null ? name.toString() : typeName; + } + + /** + * Creates a new grouped notification containing only entities for a specific topic. + */ + private HookNotification createGroupedNotification(HookNotification originalNotification, List entitiesForTopic) { + // For simplicity, we'll create a new notification with the same type but filtered entities + // This is a simplified approach - in practice, you might want more sophisticated grouping logic + + switch (originalNotification.getType()) { + case ENTITY_CREATE: + case ENTITY_FULL_UPDATE: + return createGroupedEntityCreateOrUpdateRequest(originalNotification, entitiesForTopic); + case ENTITY_CREATE_V2: + case ENTITY_FULL_UPDATE_V2: + return createGroupedEntityCreateOrUpdateRequestV2(originalNotification, entitiesForTopic); + case ENTITY_PARTIAL_UPDATE_V2: + // For partial updates, we might still need to filter referredEntities + return createGroupedEntityPartialUpdateRequestV2((EntityPartialUpdateRequestV2) originalNotification, entitiesForTopic); + case ENTITY_PARTIAL_UPDATE: + case ENTITY_DELETE: + case ENTITY_DELETE_V2: + // For single-entity operations, return the original notification + // since there should only be one entity per topic anyway + return originalNotification; + default: + LOG.warn("Cannot create grouped notification for type: {}", originalNotification.getType()); + return originalNotification; + } + } + + /** + * Creates grouped EntityCreateRequest or EntityUpdateRequest. + */ + private HookNotification createGroupedEntityCreateOrUpdateRequest(HookNotification originalNotification, List entitiesForTopic) { + String user = originalNotification.getUser(); + List filteredEntities = entitiesForTopic.stream() + .filter(info -> info.getEntity() instanceof Referenceable) + .map(info -> (Referenceable) info.getEntity()) + .collect(Collectors.toList()); + + return originalNotification instanceof EntityCreateRequest ? new EntityCreateRequest(user, filteredEntities) : new EntityUpdateRequest(user, filteredEntities); + } + + /** + * Creates grouped EntityCreateRequestV2 and EntityUpdateRequestV2. + * Filters referred entities and applies temporary GUID processing based on topic group availability. + */ + private HookNotification createGroupedEntityCreateOrUpdateRequestV2(HookNotification originalV2Request, List entitiesForTopic) { + AtlasEntity.AtlasEntitiesWithExtInfo originalEntities = null; + String user = originalV2Request.getUser(); + + if (originalV2Request instanceof EntityCreateRequestV2) { + EntityCreateRequestV2 original = (EntityCreateRequestV2) originalV2Request; + originalEntities = original.getEntities(); + } else if (originalV2Request instanceof EntityUpdateRequestV2) { + EntityUpdateRequestV2 original = (EntityUpdateRequestV2) originalV2Request; + originalEntities = original.getEntities(); + } + + if (originalEntities == null) { + LOG.debug("Original V2 entities are null. Returning original request."); + return originalV2Request; + } + + Set originalGUIDs = Collections.emptySet(); + if (originalEntities != null && originalEntities.getEntities() != null) { + originalGUIDs = originalEntities.getEntities().stream().map(AtlasEntity::getGuid).filter(StringUtils::isNotBlank).collect(Collectors.toSet()); + } + + List mainEntities = new ArrayList<>(); + Set allReferencedGUIDs = new HashSet<>(); + + // Filter main entities and aggregate referenced GUIDs for dependency filtering + for (EntityRoutingInfo info : entitiesForTopic) { + if (info.getEntity() instanceof AtlasEntity) { + AtlasEntity entity = (AtlasEntity) info.getEntity(); + + // Efficient check to ensure this entity was a "main" entity in the original request + if (originalGUIDs.contains(entity.getGuid())) { + mainEntities.add(entity); + allReferencedGUIDs.addAll(info.getReferencedGUIDs()); + } + } + } + + // Filter referred entities to only include those referenced by entities in this topic + Map originalReferredEntities = originalEntities.getReferredEntities(); + Map filteredReferredEntities = filterReferredEntities(originalReferredEntities, allReferencedGUIDs); + + // Create set of available entity GUIDs for this specific topic group + Set entityGuidsForTopic = new HashSet<>(); + + // Add main entity GUIDs + for (AtlasEntity entity : mainEntities) { + if (entity.getGuid() != null) { + entityGuidsForTopic.add(entity.getGuid()); + } + } + + // Add referred entity GUIDs + entityGuidsForTopic.addAll(filteredReferredEntities.keySet()); + + // Process temporary GUIDs (strip if referred entity is not in this Kafka topic) + List processedMainEntities = processEntitiesForTopic(mainEntities, entityGuidsForTopic, originalV2Request); + // Referred entities are kept as-is (filtering already happened) + Map processedReferredEntities = filteredReferredEntities; + + LOG.debug("Created grouped {} : {} main entities, {} referred entities (filtered from {}) with {} available GUIDs for topic group", + originalV2Request instanceof EntityCreateRequestV2 ? "EntityCreateRequestV2" : "EntityUpdateRequestV2", processedMainEntities.size(), processedReferredEntities.size(), + originalReferredEntities != null ? originalReferredEntities.size() : 0, entityGuidsForTopic.size()); + + AtlasEntity.AtlasEntitiesWithExtInfo createOrUpdateRequestWithGroupedData = new AtlasEntity.AtlasEntitiesWithExtInfo(processedMainEntities, new AtlasEntity.AtlasEntityExtInfo(processedReferredEntities)); + + return originalV2Request instanceof EntityCreateRequestV2 ? new EntityCreateRequestV2(user, createOrUpdateRequestWithGroupedData) : new EntityUpdateRequestV2(user, createOrUpdateRequestWithGroupedData); + } + + /** + * Creates grouped EntityPartialUpdateRequestV2. + * Filters referred entities and applies intelligent temporary GUID processing based on topic group availability. + */ + private EntityPartialUpdateRequestV2 createGroupedEntityPartialUpdateRequestV2(EntityPartialUpdateRequestV2 original, List entitiesForTopic) { + if (original.getEntity() == null || original.getEntity().getEntity() == null) { + return original; + } + + // Collect referenced GUIDs from the entity being updated + Set allReferencedGUIDs = new HashSet<>(); + for (EntityRoutingInfo info : entitiesForTopic) { + allReferencedGUIDs.addAll(info.getReferencedGUIDs()); + } + + // Filter referred entities to only include those referenced by the entity being updated + Map originalReferredEntities = original.getEntity().getReferredEntities(); + Map filteredReferredEntities = filterReferredEntities(originalReferredEntities, allReferencedGUIDs); + + // Create set of available entity GUIDs for this specific topic group + Set topicAvailableGUIDs = new HashSet<>(); + + // Add main entity GUID + AtlasEntity mainEntity = original.getEntity().getEntity(); + if (mainEntity.getGuid() != null) { + topicAvailableGUIDs.add(mainEntity.getGuid()); + } + + // Add referred entity GUIDs + topicAvailableGUIDs.addAll(filteredReferredEntities.keySet()); + + // Apply intelligent temporary GUID processing + AtlasEntity processedMainEntity = createEntityCopy(mainEntity); + Map> guidToUniqueAttrsMap = buildGuidToUniqueAttributesMapFromOriginal(original); + processEntityRelationshipAttributes(processedMainEntity, guidToUniqueAttrsMap, topicAvailableGUIDs); + Map processedReferredEntities = filteredReferredEntities; + + LOG.debug("Created grouped EntityPartialUpdateRequestV2: {} referred entities (filtered from {}) with {} available GUIDs for topic group", + processedReferredEntities.size(), + originalReferredEntities != null ? originalReferredEntities.size() : 0, + topicAvailableGUIDs.size()); + + // Create new AtlasEntityWithExtInfo with processed entities + AtlasEntity.AtlasEntityWithExtInfo newEntityWithExtInfo = new AtlasEntity.AtlasEntityWithExtInfo( + processedMainEntity, + new AtlasEntity.AtlasEntityExtInfo(processedReferredEntities)); + + // Original entityId should be passed as it'll have uniqueAttributes reference of table which is renamed + return new EntityPartialUpdateRequestV2(original.getUser(), original.getEntityId(), newEntityWithExtInfo); + } + + /** + * Processes a parsed notification with original metadata context and routes entities to appropriate topics. + * First converts V1 notifications to V2 format, then separates lineage entities from metadata entities, + * groups each by routing key, and sends lineage entities to ATLAS_LINEAGE topics and metadata entities to ATLAS_METADATA topics. + * + * @param notification The notification to process + * @param notificationMetadata The original metadata (source, timestamp) + */ + private void routeNotification(HookNotification notification, NotificationMetadata notificationMetadata) { + try { + // First, convert V1 notifications to V2 format for unified processing + HookNotification v2Notification = convertV1ToV2(notification); + + // Extract all entities from the notification + List entityRoutingInfos = extractEntityRoutingInfos(v2Notification, notificationMetadata); + + if (entityRoutingInfos.isEmpty()) { + LOG.debug("No entities found in notification, skipping routing"); + return; + } + + // Separate lineage entities from metadata entities + List lineageEntities = new ArrayList<>(); + List metadataEntities = new ArrayList<>(); + + for (EntityRoutingInfo routingInfo : entityRoutingInfos) { + if (lineageTopicEnabled && isLineageEntity(routingInfo.getEntity())) { + // Re-route lineage entities using lineage router + String lineageRoutingKey = routingInfo.getRoutingKey(); + String lineageTargetTopic = lineageRouter.getTargetTopic(lineageRoutingKey); + lineageEntities.add(new EntityRoutingInfo(routingInfo.getEntity(), lineageRoutingKey, lineageTargetTopic, routingInfo.getReferencedGUIDs())); + } else { + metadataEntities.add(routingInfo); + } + } + + LOG.debug("Notification contains {} total entities: {} lineage, {} metadata from source: {}", + entityRoutingInfos.size(), lineageEntities.size(), metadataEntities.size(), notificationMetadata.getSource().getSource()); + + // Process lineage entities with original notification's metadata + if (!lineageEntities.isEmpty()) { + routeEntitiesByTopic(v2Notification, lineageEntities, "lineage", notificationMetadata); + } + + // Process metadata entities with original notification's metadata + if (!metadataEntities.isEmpty()) { + routeEntitiesByTopic(v2Notification, metadataEntities, "metadata", notificationMetadata); + } + } catch (Exception e) { + LOG.error("Error processing notification: {}", e.getMessage(), e); + throw new RuntimeException("Failed to process notification", e); + } + } + + /** + * Processes entities by grouping them by target topic and sending separate notifications with original metadata. + */ + private void routeEntitiesByTopic(HookNotification originalNotification, + List entitiesWithRoutingInfo, + String entityType, + NotificationMetadata notificationMetadata) { + // Group entities by their target topics + Map> entitiesGroupedByTopic = entitiesWithRoutingInfo.stream() + .collect(Collectors.groupingBy(EntityRoutingInfo::getTargetTopic)); + + LOG.debug("Processing {} {} entities grouped into {} topics with original source: {}", + entitiesWithRoutingInfo.size(), entityType, entitiesGroupedByTopic.size(), notificationMetadata.getSource().getSource()); + + // Create and send separate notifications for each topic group + for (Map.Entry> entry : entitiesGroupedByTopic.entrySet()) { + String targetTopic = entry.getKey(); + List entitiesForTopic = entry.getValue(); + + // Create a new notification containing only entities for this topic + HookNotification groupedNotification = createGroupedNotification(originalNotification, entitiesForTopic); + + if (groupedNotification != null) { + sendToTopic(targetTopic, groupedNotification, notificationMetadata); + LOG.debug("Sent {} {} entities to topic {} with original source: {}", + entitiesForTopic.size(), entityType, targetTopic, notificationMetadata.getSource().getSource()); + } + } + } + + /** + * Send notification to topic with original source - preserves the original message source context. + */ + private void sendToTopic(String topic, HookNotification notification, NotificationMetadata notificationMetadata) { + try { + notificationInterface.send(topic, Collections.singletonList(notification), notificationMetadata.getSource(), notificationMetadata.getMsgCreationTime()); + LOG.debug("Successfully sent notification to topic {} with original source: {}", topic, notificationMetadata.getSource().getSource()); + } catch (NotificationException e) { + LOG.error("Failed to send notification to topic {} with original source {}: {}", + topic, notificationMetadata.getSource().getSource(), e.getMessage()); + throw new RuntimeException("Failed to send notification to topic " + topic + + " with original source " + notificationMetadata.getSource().getSource(), e); + } + } + + private boolean isPotentialGuid(String value) { + return value != null && (value.startsWith("-") || GUID_PATTERN.matcher(value).matches()); + } + + private void recordFailedMessages(String topic, List failedMessages) { + //logging failed messages + for (String message : failedMessages) { + failedMessageLog.error("[{}-DROPPED_NOTIFICATION] {}", topic, message); + } + + failedMessages.clear(); + } + + /** + * Enhanced entity routing information that includes referenced GUIDs from relationship attributes. + * The referenced GUIDs are collected BEFORE temp GUID resolution to preserve filtering context. + */ + private static class EntityRoutingInfo { + private final Object entity; + private final String routingKey; + private final String targetTopic; + private final Set referencedGUIDs; + + public EntityRoutingInfo(Object entity, String routingKey, String targetTopic) { + this(entity, routingKey, targetTopic, new HashSet<>()); + } + + public EntityRoutingInfo(Object entity, String routingKey, String targetTopic, Set referencedGUIDs) { + this.entity = entity; + this.routingKey = routingKey; + this.targetTopic = targetTopic; + this.referencedGUIDs = referencedGUIDs != null ? referencedGUIDs : new HashSet<>(); + } + + public Object getEntity() { + return entity; + } + + public String getRoutingKey() { + return routingKey; + } + + public String getTargetTopic() { + return targetTopic; + } + + public Set getReferencedGUIDs() { + return referencedGUIDs; + } + } + + private static class NotificationMetadata { + private MessageSource source; + private long msgCreationTime; + + public NotificationMetadata() { + } + + public MessageSource getSource() { + return source; + } + + public void setSource(MessageSource source) { + this.source = source; + } + + public long getMsgCreationTime() { + return msgCreationTime; + } + + public void setMsgCreationTime(long msgCreationTime) { + this.msgCreationTime = msgCreationTime; + } + } + + private class RenameRoutingInfo { + private String originalEntityName; + private long renamedEventTimestamp; + + public RenameRoutingInfo(String originalEntityName, long renamedEventTimestamp) { + this.originalEntityName = originalEntityName; + this.renamedEventTimestamp = renamedEventTimestamp; + } + + public String getOriginalEntityName() { + return originalEntityName; + } + + public void setOriginalEntityName(String originalEntityName) { + this.originalEntityName = originalEntityName; + } + + public long getRenamedEventTimestamp() { + return renamedEventTimestamp; + } + + public void setRenamedEventTimestamp(long renamedEventTimestamp) { + this.renamedEventTimestamp = renamedEventTimestamp; + } + } + + /** + * Handles type definition changes by refreshing the process types cache. + * This ensures that the NotificationPreProcessor stays up-to-date with any + * type changes that occur after Atlas startup. + * + * @param changedTypeDefs the changed type definitions + * @throws AtlasBaseException if refresh fails + */ + @Override + public void onChange(ChangedTypeDefs changedTypeDefs) throws AtlasBaseException { + if (changedTypeDefs == null) { + return; + } + + boolean needsRefresh = false; + + // Check if any entity types were added, updated, or deleted + if (changedTypeDefs.getCreatedTypeDefs() != null && !changedTypeDefs.getCreatedTypeDefs().isEmpty()) { + LOG.info("Type registry change detected: {} new type definitions created", changedTypeDefs.getCreatedTypeDefs().size()); + needsRefresh = true; + } + + if (changedTypeDefs.getUpdatedTypeDefs() != null && !changedTypeDefs.getUpdatedTypeDefs().isEmpty()) { + LOG.info("Type registry change detected: {} type definitions updated", changedTypeDefs.getUpdatedTypeDefs().size()); + needsRefresh = true; + } + + if (changedTypeDefs.getDeletedTypeDefs() != null && !changedTypeDefs.getDeletedTypeDefs().isEmpty()) { + LOG.info("Type registry change detected: {} type definitions deleted", changedTypeDefs.getDeletedTypeDefs().size()); + needsRefresh = true; + } + + if (needsRefresh) { + refreshTypeRegistry(); + } + } + + /** + * Handles type registry load completion. + * This is called when the type registry finishes loading all type definitions. + * + * @throws AtlasBaseException if refresh fails + */ + @Override + public void onLoadCompletion() throws AtlasBaseException { + LOG.info("Type registry load completion detected, refreshing process types cache"); + refreshTypeRegistry(); + } + + /** + * Refreshes the process types cache with the latest type definitions from the registry. + * This method is thread-safe and handles errors gracefully. + */ + private void refreshTypeRegistry() { + try { + LOG.info("Refreshing type registry cache for NotificationPreProcessor"); + + if (typeRegistry != null) { + Set newProcessTypes = typeRegistry.getAllEntityTypes().stream() + .filter(x -> x.getTypeName().equals(AtlasBaseTypeDef.ATLAS_TYPE_PROCESS) || x.isSubTypeOf(AtlasBaseTypeDef.ATLAS_TYPE_PROCESS)) + .map(x -> x.getTypeName()) + .collect(Collectors.toSet()); + + // Create unmodifiable set for thread safety + Set oldProcessTypes = this.allProcessTypes; + this.allProcessTypes = Collections.unmodifiableSet(newProcessTypes); + + // Clear the process type cache to force re-evaluation + processTypeCache.clear(); + + LOG.info("Successfully refreshed process types cache: {} types (was {} types)", + newProcessTypes.size(), oldProcessTypes != null ? oldProcessTypes.size() : 0); + + if (LOG.isDebugEnabled()) { + Set addedTypes = new HashSet<>(newProcessTypes); + if (oldProcessTypes != null) { + addedTypes.removeAll(oldProcessTypes); + } + + Set removedTypes = new HashSet<>(); + if (oldProcessTypes != null) { + removedTypes.addAll(oldProcessTypes); + removedTypes.removeAll(newProcessTypes); + } + + if (!addedTypes.isEmpty()) { + LOG.debug("Added process types: {}", addedTypes); + } + if (!removedTypes.isEmpty()) { + LOG.debug("Removed process types: {}", removedTypes); + } + } + } else { + LOG.warn("Type registry is null during refresh, keeping existing process types cache"); + } + } catch (Exception e) { + LOG.error("Failed to refresh type registry cache: {}. Process type detection may be outdated.", e.getMessage(), e); + // Don't throw exception - continue with existing cache + } + } +} diff --git a/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerKafkaTest.java b/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerKafkaTest.java index 3f2a83eef2e..a9758694384 100644 --- a/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerKafkaTest.java +++ b/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerKafkaTest.java @@ -120,7 +120,7 @@ public void testConsumerConsumesNewMessageWithAutoCommitDisabled() throws AtlasE produceMessage(null, NotificationInterface.NotificationType.HOOK, new HookNotificationV1.EntityCreateRequest("test_user1", createEntity())); NotificationConsumer consumer = createNewConsumer(NotificationInterface.NotificationType.HOOK, kafkaNotification, false); - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.new HookConsumer(consumer); consumeOneMessage(consumer, hookConsumer); @@ -138,7 +138,7 @@ public void testConsumerConsumesNewMessageWithAutoCommitDisabled() throws AtlasE @Test(enabled = false) public void consumerConsumesNewMessageButCommitThrowsAnException_MessageOffsetIsRecorded() throws AtlasException { ExceptionThrowingCommitConsumer consumer = createNewConsumerThatThrowsExceptionInCommit(kafkaNotification, true); - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.new HookConsumer(consumer); produceMessage(null, NotificationInterface.NotificationType.HOOK, new HookNotificationV1.EntityCreateRequest("test_user2", createEntity())); @@ -168,7 +168,7 @@ public void testConsumerRemainsAtSameMessageWithAutoCommitEnabled() throws Excep assertNotNull(consumer); - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.new HookConsumer(consumer); consumeOneMessage(consumer, hookConsumer); @@ -192,7 +192,7 @@ public void testImportMessagesArePublishedToDynamicTopicAndConsumedAndProcessed( addTopicToNotification(NotificationInterface.NotificationType.ASYNC_IMPORT, kafkaNotification, topic); NotificationConsumer consumer = createNewConsumer(NotificationInterface.NotificationType.ASYNC_IMPORT, kafkaNotification, false); - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.new HookConsumer(consumer); consumeOneMessage(consumer, hookConsumer); diff --git a/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerTest.java b/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerTest.java index 4bbfe410956..7529850533b 100644 --- a/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerTest.java +++ b/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerTest.java @@ -158,7 +158,7 @@ public void setup() throws AtlasBaseException { @Test public void testConsumerCanProceedIfServerIsReady() throws Exception { - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.new HookConsumer(mock(NotificationConsumer.class)); NotificationHookConsumer.Timer timer = mock(NotificationHookConsumer.Timer.class); @@ -171,7 +171,7 @@ public void testConsumerCanProceedIfServerIsReady() throws Exception { @Test public void testConsumerWaitsNTimesIfServerIsNotReadyNTimes() throws Exception { - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.new HookConsumer(mock(NotificationConsumer.class)); NotificationHookConsumer.Timer timer = mock(NotificationHookConsumer.Timer.class); @@ -188,7 +188,7 @@ public void testConsumerWaitsNTimesIfServerIsNotReadyNTimes() throws Exception { @Test public void testCommitIsCalledWhenMessageIsProcessed() throws AtlasServiceException, AtlasException { - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); NotificationConsumer consumer = mock(NotificationConsumer.class); NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.new HookConsumer(consumer); EntityCreateRequest message = mock(EntityCreateRequest.class); @@ -208,7 +208,7 @@ public void testCommitIsNotCalledEvenWhenMessageProcessingFails() throws Excepti Configuration config = buildFailedMsgCacheConfig(10); when(config.getInt(NotificationHookConsumer.CONSUMER_RETRIES_PROPERTY, 3)).thenReturn(1); - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); NotificationConsumer consumer = mock(NotificationConsumer.class); NotificationHookConsumer.HookConsumer hookConsumer = (NotificationHookConsumer.HookConsumer) createHookConsumerWithEntityProcessor( notificationHookConsumer, consumer, createEntityProcessor(config)); @@ -225,7 +225,7 @@ public void testCommitIsNotCalledEvenWhenMessageProcessingFails() throws Excepti @Test public void testConsumerProceedsWithFalseIfInterrupted() throws Exception { - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.new HookConsumer(mock(NotificationConsumer.class)); NotificationHookConsumer.Timer timer = mock(NotificationHookConsumer.Timer.class); @@ -245,7 +245,7 @@ public void testConsumersStartedIfHAIsDisabled() throws Exception { when(configuration.getBoolean(HAConfiguration.ATLAS_SERVER_HA_ENABLED_KEY, false)).thenReturn(false); when(configuration.getInt(NotificationHookConsumer.CONSUMER_THREADS_PROPERTY, 1)).thenReturn(1); when(notificationInterface.createConsumers(NotificationType.HOOK, 1)).thenReturn(consumers); - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); notificationHookConsumer.startInternal(configuration, executorService); verify(notificationInterface).createConsumers(NotificationType.HOOK, 1); @@ -263,7 +263,7 @@ public void testConsumersAreNotStartedIfHAIsEnabled() throws Exception { when(configuration.getBoolean(HAConfiguration.ATLAS_SERVER_HA_ENABLED_KEY)).thenReturn(true); when(configuration.getInt(NotificationHookConsumer.CONSUMER_THREADS_PROPERTY, 1)).thenReturn(1); when(notificationInterface.createConsumers(NotificationType.HOOK, 1)).thenReturn(consumers); - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); notificationHookConsumer.startInternal(configuration, executorService); @@ -282,7 +282,7 @@ public void testConsumersAreStartedWhenInstanceBecomesActive() throws Exception when(configuration.getInt(NotificationHookConsumer.CONSUMER_THREADS_PROPERTY, 1)).thenReturn(1); when(notificationInterface.createConsumers(NotificationType.HOOK, 1)).thenReturn(consumers); - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); notificationHookConsumer.startInternal(configuration, executorService); notificationHookConsumer.instanceIsActive(); @@ -302,7 +302,7 @@ public void testConsumersAreStoppedWhenInstanceBecomesPassive() throws Exception when(configuration.getBoolean(HAConfiguration.ATLAS_SERVER_HA_ENABLED_KEY, false)).thenReturn(true); when(configuration.getInt(NotificationHookConsumer.CONSUMER_THREADS_PROPERTY, 1)).thenReturn(1); when(notificationInterface.createConsumers(NotificationType.HOOK, 1)).thenReturn(consumers); - final NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + final NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); doAnswer(new Answer() { @Override @@ -333,7 +333,7 @@ public void consumersStoppedBeforeStarting() throws Exception { when(configuration.getBoolean(HAConfiguration.ATLAS_SERVER_HA_ENABLED_KEY, false)).thenReturn(true); when(configuration.getInt(NotificationHookConsumer.CONSUMER_THREADS_PROPERTY, 1)).thenReturn(1); when(notificationInterface.createConsumers(NotificationType.HOOK, 1)).thenReturn(consumers); - final NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + final NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); notificationHookConsumer.startInternal(configuration, executorService); notificationHookConsumer.instanceIsPassive(); @@ -405,7 +405,7 @@ public void onCloseImportConsumerShutdownConsumerAndDeletesTopic() throws Except when(notificationInterface.createConsumers(ASYNC_IMPORT, 1)).thenReturn(consumers); doNothing().when(notificationInterface).deleteTopic(ASYNC_IMPORT, AtlasConfiguration.ASYNC_IMPORT_TOPIC_PREFIX.getString() + importId); - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); // setting this just so this test would not create hook consumers Field consumerDisabledField = NotificationHookConsumer.class.getDeclaredField("consumerDisabled"); @@ -514,7 +514,7 @@ public void testHookConsumersNotStartedWhenConsumersAreDisabled() throws Excepti when(configuration.getInt(NotificationHookConsumer.CONSUMER_THREADS_PROPERTY, 1)).thenReturn(1); // TestableNotificationHookConsumer with override that sets consumerDisabled = true - NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter) { + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null) { @Override protected ExecutorService createExecutor() { return mock(ExecutorService.class); @@ -550,7 +550,7 @@ private NotificationHookConsumer setupNotificationHookConsumer() throws AtlasExc when(notificationConsumerMock.receive()).thenThrow(new IllegalStateException()); when(notificationInterface.createConsumers(NotificationType.HOOK, 1)).thenReturn(consumers); - return new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + return new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); } @Test @@ -615,7 +615,7 @@ public void testConstructorWithAllPreprocessingEnabled() throws Exception { NotificationHookConsumer consumer = new NotificationHookConsumer( notificationInterface, atlasEntityStore, serviceState, instanceConverter, - typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter); + typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter, null); // Verify complex configuration was applied assertNotNull(consumer); @@ -653,7 +653,7 @@ public void testConstructorWithInvalidPatterns() throws Exception { // Should still create consumer despite invalid patterns (they get logged and ignored) NotificationHookConsumer consumer = new NotificationHookConsumer( notificationInterface, atlasEntityStore, serviceState, instanceConverter, - typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter); + typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter, null); assertNotNull(consumer); } @@ -1114,7 +1114,7 @@ private NotificationHookConsumer createTestConsumerWithBatching(int batchSize) t appProps.when(ApplicationProperties::get).thenReturn(batchConfig); return new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, - instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter); + instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter, null); } } @@ -1454,7 +1454,7 @@ private NotificationHookConsumer createTestConsumerWithFailedMsgCache(int cacheS appProps.when(ApplicationProperties::get).thenReturn(cacheConfig); return new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, - instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter); + instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter, null); } } @@ -1475,7 +1475,7 @@ private NotificationHookConsumer createTestConsumerWithLargeMessageThreshold(int appProps.when(ApplicationProperties::get).thenReturn(thresholdConfig); return new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, - instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter); + instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter, null); } } @@ -1497,7 +1497,7 @@ private NotificationHookConsumer createTestConsumerWithEntityIgnorePatterns() th appProps.when(ApplicationProperties::get).thenReturn(ignoreConfig); return new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, - instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter); + instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter, null); } } @@ -1519,7 +1519,7 @@ private NotificationHookConsumer createTestConsumerWithHiveLineageSkip() throws appProps.when(ApplicationProperties::get).thenReturn(skipConfig); return new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, - instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter); + instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter, null); } } @@ -1883,7 +1883,7 @@ public void testCloseImportConsumerWithValidImportId() throws Exception { NotificationHookConsumer consumer = new NotificationHookConsumer( notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, - entityCorrelationStore, asyncImporter); + entityCorrelationStore, asyncImporter, null); // ✅ Manually initialize consumers list (otherwise it's null) Field consumersField = NotificationHookConsumer.class.getDeclaredField("consumers"); @@ -2496,7 +2496,7 @@ public void testHookConsumerShutdownAndRestart() throws Exception { } private NotificationHookConsumer createTestConsumer() throws AtlasException { - return new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter); + return new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, entityCorrelationStore, asyncImporter, null); } // ==================== HELPER METHODS ==================== @@ -2521,7 +2521,8 @@ private NotificationHookConsumer createTestConsumerWithPreprocessing() throws Ex typeRegistry, metricsUtil, entityCorrelationStore, - asyncImporter); + asyncImporter, + null); } } @@ -2552,7 +2553,8 @@ private NotificationHookConsumer createTestConsumerWithAuthorization() throws Ex typeRegistry, metricsUtil, entityCorrelationStore, - asyncImporter); + asyncImporter, + null); } } @@ -2677,7 +2679,7 @@ class TestableNotificationHookConsumer extends NotificationHookConsumer { int executorCreationCount; TestableNotificationHookConsumer() throws AtlasException { - super(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter); + super(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, null); } @Override diff --git a/webapp/src/test/java/org/apache/atlas/notification/NotificationHookParallelConsumerTest.java b/webapp/src/test/java/org/apache/atlas/notification/NotificationHookParallelConsumerTest.java new file mode 100644 index 00000000000..52ccf818197 --- /dev/null +++ b/webapp/src/test/java/org/apache/atlas/notification/NotificationHookParallelConsumerTest.java @@ -0,0 +1,833 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.atlas.notification; + +import org.apache.atlas.ApplicationProperties; +import org.apache.atlas.AtlasConfiguration; +import org.apache.atlas.AtlasException; +import org.apache.atlas.exception.AtlasBaseException; +import org.apache.atlas.kafka.AtlasKafkaMessage; +import org.apache.atlas.kafka.NotificationProvider; +import org.apache.atlas.model.instance.AtlasEntity; +import org.apache.atlas.model.instance.AtlasEntity.AtlasEntitiesWithExtInfo; +import org.apache.atlas.model.instance.EntityMutationResponse; +import org.apache.atlas.model.notification.HookNotification; +import org.apache.atlas.model.notification.HookNotification.EntityCreateRequestV2; +import org.apache.atlas.model.notification.MessageSource; +import org.apache.atlas.notification.preprocessor.NotificationPreProcessor; +import org.apache.atlas.repository.converters.AtlasInstanceConverter; +import org.apache.atlas.repository.impexp.AsyncImporter; +import org.apache.atlas.repository.store.graph.AtlasEntityStore; +import org.apache.atlas.repository.store.graph.AtlasTypeDefGraphStore; +import org.apache.atlas.repository.store.graph.v2.EntityStream; +import org.apache.atlas.type.AtlasType; +import org.apache.atlas.type.AtlasTypeRegistry; +import org.apache.atlas.util.AtlasMetricsUtil; +import org.apache.atlas.v1.model.instance.Referenceable; +import org.apache.atlas.v1.model.notification.HookNotificationV1; +import org.apache.atlas.web.service.ServiceState; +import org.apache.commons.configuration2.Configuration; +import org.apache.commons.configuration2.PropertiesConfiguration; +import org.apache.commons.lang.StringUtils; +import org.apache.kafka.common.TopicPartition; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.MockitoAnnotations; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutorService; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; + +public class NotificationHookParallelConsumerTest { + private static final String QUALIFIED_NAME = "qualifiedName"; + private static final String NAME = "name"; + private static final String DESCRIPTION = "description"; + + private static final Logger FAILED_LOG = LoggerFactory.getLogger("FAILED"); + + @SuppressWarnings("unchecked") + private static NotificationConsumer mockHookNotificationConsumer() { + return (NotificationConsumer) mock(NotificationConsumer.class); + } + + private static void mockCreateConsumers( + NotificationInterface notificationInterface, + NotificationInterface.NotificationType notificationType, + int numConsumers, + List> consumers) { + doReturn(consumers).when(notificationInterface).createConsumers(notificationType, numConsumers); + } + + @FunctionalInterface + private interface Throwing { + void run() throws Exception; + } + + /** + * {@link NotificationPreProcessor} resolves {@link NotificationProvider#get()} in its constructor; + * stub before {@link NotificationHookConsumer#getPreprocessorHookConsumers()}. + */ + private void withMockedNotificationProvider(Throwing runnable) throws Exception { + try (MockedStatic np = mockStatic(NotificationProvider.class)) { + np.when(NotificationProvider::get).thenReturn(notificationInterface); + doNothing().when(notificationInterface).send(anyString(), anyList(), any(MessageSource.class), anyLong()); + runnable.run(); + } + } + + @Mock + private NotificationInterface notificationInterface; + + @Mock + private AtlasEntityStore atlasEntityStore; + + @Mock + private ServiceState serviceState; + + @Mock + private AtlasInstanceConverter instanceConverter; + + @Mock + private AtlasTypeRegistry typeRegistry; + + @Mock + private AtlasTypeDefGraphStore typeDefStore; + + @Mock + private AtlasMetricsUtil metricsUtil; + + @Mock + private AsyncImporter asyncImporter; + + @Mock + private NotificationInterface notificationInterfaceForPreProcessor; + + private AutoCloseable mocks; + + @BeforeMethod + public void setup() throws AtlasBaseException { + mocks = MockitoAnnotations.openMocks(this); + + AtlasType mockType = mock(AtlasType.class); + AtlasEntitiesWithExtInfo mockEntity = new AtlasEntitiesWithExtInfo(mock(AtlasEntity.class)); + + when(typeRegistry.getType(anyString())).thenReturn(mockType); + when(instanceConverter.toAtlasEntities(anyList())).thenReturn(mockEntity); + + EntityMutationResponse mutationResponse = mock(EntityMutationResponse.class); + when(atlasEntityStore.createOrUpdate(any(EntityStream.class), anyBoolean())).thenReturn(mutationResponse); + + when(serviceState.getState()).thenReturn(ServiceState.ServiceStateValue.ACTIVE); + + doNothing().when(metricsUtil).onNotificationProcessorComplete(anyString(), anyInt(), anyLong(), any()); + } + + @AfterMethod + public void tearDown() throws Exception { + resetParallelProcessingEnabled(); + if (mocks != null) { + mocks.close(); + } + } + + private void mockParallelProcessingEnabled(boolean enabled) throws AtlasException { + setParallelProcessingEnabled(enabled); + } + + private HookNotificationV1.EntityCreateRequest createTestMessage() { + Referenceable entity = new Referenceable("test_type"); + entity.set(NAME, "testEntity"); + entity.set(DESCRIPTION, "Test entity for preprocessing"); + entity.set(QUALIFIED_NAME, "test.entity@cluster"); + return new HookNotificationV1.EntityCreateRequest("test_user", Collections.singletonList(entity)); + } + + @Test + public void testQualifiedNameRouterSameRoutingKeyMapsToSameTopic() { + QualifiedNameRouter router = new QualifiedNameRouter(5, "ATLAS_METADATA_"); + String t1 = router.getTargetTopic("db1.schema1"); + String t2 = router.getTargetTopic("db1.schema1"); + assertEquals(t1, t2); + } + + @Test + public void testQualifiedNameRouterDifferentKeysMayMapToDifferentTopics() { + QualifiedNameRouter router = new QualifiedNameRouter(5, "ATLAS_METADATA_"); + String t1 = router.getTargetTopic("db1.schema1"); + String t2 = router.getTargetTopic("db9.schema9"); + assertNotEquals(t1, t2); + } + + @Test + public void testQualifiedNameRouterBlankKeyUsesFallbackPrefix() { + QualifiedNameRouter router = new QualifiedNameRouter(2, "ATLAS_METADATA_"); + String t1 = router.getTargetTopic(null); + assertTrue(t1.startsWith("ATLAS_METADATA_")); + } + + @Test + public void testNotificationPreProcessorRoutesCreateV2ToMetadataTopic() throws Exception { + try (MockedStatic np = mockStatic(NotificationProvider.class)) { + np.when(NotificationProvider::get).thenReturn(notificationInterfaceForPreProcessor); + doNothing().when(notificationInterfaceForPreProcessor).send(anyString(), anyList(), any(MessageSource.class), anyLong()); + + Configuration config = defaultPreprocessorConfiguration(); + AtlasEntity entity = hiveTableEntity("db1.table1@cl1"); + EntityCreateRequestV2 request = entityCreateRequestV2("u1", entity); + + NotificationPreProcessor processor = new NotificationPreProcessor(config, metricsUtil, null, FAILED_LOG); + AtlasKafkaMessage kafkaMsg = preprocessKafkaMessage(request); + + assertNotNull(processor.handleMessage(kafkaMsg)); + + ArgumentCaptor topicCaptor = ArgumentCaptor.forClass(String.class); + verify(notificationInterfaceForPreProcessor).send(topicCaptor.capture(), anyList(), any(MessageSource.class), anyLong()); + assertTrue(topicCaptor.getValue().startsWith("ATLAS_METADATA_")); + } + } + + @Test + public void testNotificationPreProcessorDistinctRoutingKeysTwoSends() throws Exception { + try (MockedStatic np = mockStatic(NotificationProvider.class)) { + np.when(NotificationProvider::get).thenReturn(notificationInterfaceForPreProcessor); + doNothing().when(notificationInterfaceForPreProcessor).send(anyString(), anyList(), any(MessageSource.class), anyLong()); + + Configuration config = defaultPreprocessorConfiguration(); + AtlasEntity e1 = hiveTableEntity("alpha.beta1@cl"); + AtlasEntity e2 = hiveTableEntity("gamma.delta1@cl"); + EntityCreateRequestV2 request = entityCreateRequestV2("user", e1, e2); + + NotificationPreProcessor processor = new NotificationPreProcessor(config, metricsUtil, null, FAILED_LOG); + processor.handleMessage(preprocessKafkaMessage(request)); + + verify(notificationInterfaceForPreProcessor, times(2)).send(anyString(), anyList(), any(MessageSource.class), anyLong()); + } + } + + @Test + public void testNotificationPreProcessorCollectResultsNull() throws Exception { + try (MockedStatic np = mockStatic(NotificationProvider.class)) { + np.when(NotificationProvider::get).thenReturn(notificationInterfaceForPreProcessor); + doNothing().when(notificationInterfaceForPreProcessor).send(anyString(), anyList(), any(MessageSource.class), anyLong()); + NotificationPreProcessor processor = new NotificationPreProcessor( + defaultPreprocessorConfiguration(), metricsUtil, null, FAILED_LOG); + assertNull(processor.collectResults()); + } + } + + @Test + public void testPreprocessorConsumersCreatedWhenParallelEnabledViaApplicationProperties() throws Exception { + withMockedNotificationProvider(() -> { + setParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer( + notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, + null, asyncImporter, typeDefStore); + + List preprocessorConsumers = notificationHookConsumer.getPreprocessorHookConsumers(); + + assertNotNull(preprocessorConsumers); + assertEquals(preprocessorConsumers.size(), 1); + verify(notificationInterface).createConsumers(NotificationInterface.NotificationType.HOOK_PREPROCESS, 1); + verify(typeDefStore).registerTypeDefChangeListener(any(NotificationPreProcessor.class)); + }); + } + + @Test + public void testStartHookConsumersDoesNotCreatePreprocessWhenParallelDisabledViaApplicationProperties() throws Exception { + setParallelProcessingEnabled(false); + + NotificationConsumer mockHookConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK, 1, + Collections.singletonList(mockHookConsumer)); + + NotificationHookConsumer consumer = new NotificationHookConsumer( + notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, + null, asyncImporter, typeDefStore); + + Field executorsField = NotificationHookConsumer.class.getDeclaredField("executors"); + executorsField.setAccessible(true); + executorsField.set(consumer, mock(ExecutorService.class)); + + Method startHookConsumers = NotificationHookConsumer.class.getDeclaredMethod("startHookConsumers"); + startHookConsumers.setAccessible(true); + startHookConsumers.invoke(consumer); + + verify(notificationInterface, never()).createConsumers(NotificationInterface.NotificationType.HOOK_PREPROCESS, 1); + } + + @Test + public void testPreprocessorSkipsTypeDefRegistrationWhenTypeDefStoreNull() throws Exception { + withMockedNotificationProvider(() -> { + setParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer( + notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, + null, asyncImporter, null); + + notificationHookConsumer.getPreprocessorHookConsumers(); + + verify(typeDefStore, never()).registerTypeDefChangeListener(any()); + }); + } + + @Test + public void testRoutingKeyFromFullQualifiedName_matrix_likePreprocessor() { + String[][] cases = { + {"db.table@cluster", "db.table"}, + {"db.table.column@cluster", "db.table"}, + {"db.table.column.nested@cluster", "db.table"}, + {"singlelevel@cluster", "singlelevel"}, + {"db.table:timestamp@cluster", "db.table"}, + }; + for (String[] c : cases) { + assertEquals(routingKeyFromFullQualifiedNameByPreprocessor(c[0]), c[1], c[0]); + } + } + + @Test + public void testRoutingKey_tableAndColumnEntities_shareSameRoutingKey() { + String tableKey = routingKeyFromFullQualifiedNameByPreprocessor("db1.table1@cluster"); + String colKey = routingKeyFromFullQualifiedNameByPreprocessor("db1.table1.col1@cluster"); + assertEquals(tableKey, "db1.table1"); + assertEquals(colKey, tableKey); + } + + @Test + public void testRoutingKey_differentDatabases_distinctKeys() { + String k1 = routingKeyFromFullQualifiedNameByPreprocessor("db1.table1@cluster"); + String k2 = routingKeyFromFullQualifiedNameByPreprocessor("db2.table2@cluster"); + assertNotEquals(k1, k2); + } + + @Test + public void testQualifiedNameRouter_samePreprocessorRoutingKey_mapsToStableTopic() { + QualifiedNameRouter router = new QualifiedNameRouter(5, "ATLAS_METADATA_"); + String key = routingKeyFromFullQualifiedNameByPreprocessor("db1.table1@cluster"); + assertEquals(router.getTargetTopic(key), router.getTargetTopic(key)); + assertTrue(router.getTargetTopic(key).startsWith("ATLAS_METADATA_")); + } + + @Test + public void testRoutingKey_threePartQualifiedName_usesFirstTwoDotSegments() { + assertEquals( + routingKeyFromFullQualifiedNameByPreprocessor("namespace.db.table@cluster"), + "namespace.db"); + } + + @Test + public void testRoutingKey_blankQualifiedName_noRoutingKey() { + assertNull(routingKeyFromFullQualifiedNameByPreprocessor("")); + assertNull(routingKeyFromFullQualifiedNameByPreprocessor(null)); + } + + @Test + public void testRoutingKey_specialCharactersAndRouterTopics() { + QualifiedNameRouter router = new QualifiedNameRouter(5, "ATLAS_METADATA_"); + String kHyphen = routingKeyFromFullQualifiedNameByPreprocessor("db-name.table-name@cluster"); + String kNs = routingKeyFromFullQualifiedNameByPreprocessor("namespace.db.table@cluster"); + assertEquals(kHyphen, "db-name.table-name"); + assertEquals(kNs, "namespace.db"); + assertTrue(router.getTargetTopic(kHyphen).startsWith("ATLAS_METADATA_")); + assertTrue(router.getTargetTopic(kNs).startsWith("ATLAS_METADATA_")); + } + + @Test + public void testPreprocessorConsumersAreCreatedWhenParallelProcessingEnabled() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationConsumer mockHookConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK, 1, + Collections.singletonList(mockHookConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + List preprocessorConsumers = notificationHookConsumer.getPreprocessorHookConsumers(); + + assertNotNull(preprocessorConsumers); + assertFalse(preprocessorConsumers.isEmpty()); + assertEquals(preprocessorConsumers.size(), 1); + + verify(notificationInterface).createConsumers(NotificationInterface.NotificationType.HOOK_PREPROCESS, 1); + }); + } + + @Test + public void testPreprocessorConsumersAreNotCreatedWhenParallelProcessingDisabled() throws Exception { + mockParallelProcessingEnabled(false); + + NotificationConsumer mockHookConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK, 1, + Collections.singletonList(mockHookConsumer)); + + new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + verify(notificationInterface, never()).createConsumers(NotificationInterface.NotificationType.HOOK_PREPROCESS, 1); + } + + @Test + public void testPreprocessorConsumerUsesNotificationPreProcessorEntityProcessor() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + List preprocessorConsumers = notificationHookConsumer.getPreprocessorHookConsumers(); + + assertNotNull(preprocessorConsumers); + assertEquals(preprocessorConsumers.size(), 1); + + NotificationHookConsumer.HookConsumer hookConsumer = preprocessorConsumers.get(0); + + Field entityProcessorField = NotificationHookConsumer.HookConsumer.class.getDeclaredField("entityProcessor"); + entityProcessorField.setAccessible(true); + Object entityProcessor = entityProcessorField.get(hookConsumer); + + assertNotNull(entityProcessor); + assertTrue(entityProcessor instanceof NotificationPreProcessor, + "EntityProcessor should be instance of NotificationPreProcessor"); + }); + } + + @Test + public void testPreprocessorConsumerHandleMessageCommitsKafkaOffset() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.getPreprocessorHookConsumers().get(0); + + HookNotificationV1.EntityCreateRequest message = createTestMessage(); + AtlasKafkaMessage kafkaMessage = new AtlasKafkaMessage<>(message, 0, "ATLAS_HOOK", 0); + + doNothing().when(mockPreprocessConsumer).commit(any(TopicPartition.class), anyLong()); + + hookConsumer.handleMessage(kafkaMessage); + + verify(mockPreprocessConsumer, times(1)).commit(any(TopicPartition.class), anyLong()); + }); + } + + @Test + public void testMultiplePreprocessorConsumersAreCreated() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockConsumer1 = mockHookNotificationConsumer(); + NotificationConsumer mockConsumer2 = mockHookNotificationConsumer(); + + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Arrays.asList(mockConsumer1, mockConsumer2)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + List preprocessorConsumers = notificationHookConsumer.getPreprocessorHookConsumers(); + + assertNotNull(preprocessorConsumers); + assertEquals(preprocessorConsumers.size(), 2); + + for (NotificationHookConsumer.HookConsumer consumer : preprocessorConsumers) { + Field entityProcessorField = NotificationHookConsumer.HookConsumer.class.getDeclaredField("entityProcessor"); + entityProcessorField.setAccessible(true); + Object entityProcessor = entityProcessorField.get(consumer); + + assertTrue(entityProcessor instanceof NotificationPreProcessor); + } + }); + } + + @Test + public void testPreprocessorConsumerThreadName() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.getPreprocessorHookConsumers().get(0); + String threadName = hookConsumer.getName(); + + assertNotNull(threadName); + assertTrue(threadName.contains("preprocessor-consumer-thread"), + "Expected ATLAS_HOOK_PRE_PROCESSOR thread name (e.g. atlas-hook-preprocessor-consumer-thread)"); + }); + } + + @Test + public void testGetPreprocessorHookConsumersReturnsEmptyListWhenNoConsumersCreated() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, Collections.emptyList()); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + List preprocessorConsumers = notificationHookConsumer.getPreprocessorHookConsumers(); + + assertNotNull(preprocessorConsumers); + assertTrue(preprocessorConsumers.isEmpty()); + }); + } + + @Test + public void testConsumerFieldIsProperlyInitializedInPreprocessorHookConsumer() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.getPreprocessorHookConsumers().get(0); + + Field consumerField = NotificationHookConsumer.HookConsumer.class.getDeclaredField("consumer"); + consumerField.setAccessible(true); + Object consumer = consumerField.get(hookConsumer); + + assertNotNull(consumer, "Consumer field should be initialized"); + assertEquals(consumer, mockPreprocessConsumer, "Consumer field should be the mock consumer"); + }); + } + + @Test + public void testPreprocessorConsumerEntityProcessorIsNotSerialEntityProcessor() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.getPreprocessorHookConsumers().get(0); + + Field entityProcessorField = NotificationHookConsumer.HookConsumer.class.getDeclaredField("entityProcessor"); + entityProcessorField.setAccessible(true); + Object entityProcessor = entityProcessorField.get(hookConsumer); + + assertTrue(entityProcessor instanceof NotificationPreProcessor); + assertFalse(entityProcessor instanceof SerialEntityProcessor, + "Preprocessor consumer should NOT use SerialEntityProcessor"); + }); + } + + @Test + public void testPreprocessorConsumersUseCorrectNotificationType() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + notificationHookConsumer.getPreprocessorHookConsumers(); + + verify(notificationInterface, times(1)).createConsumers(NotificationInterface.NotificationType.HOOK_PREPROCESS, 1); + + verify(notificationInterface, never()).createConsumers( + NotificationInterface.NotificationType.HOOK_PREPROCESS, 8); + }); + } + + @Test + public void testGetPreprocessorHookConsumersCanBeCalledMultipleTimes() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + List consumers1 = notificationHookConsumer.getPreprocessorHookConsumers(); + List consumers2 = notificationHookConsumer.getPreprocessorHookConsumers(); + + assertNotNull(consumers1); + assertNotNull(consumers2); + + verify(notificationInterface, times(2)).createConsumers(NotificationInterface.NotificationType.HOOK_PREPROCESS, 1); + }); + } + + @Test + public void testPreprocessorConsumerReceivesMultipleMessagesCommitsEach() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.getPreprocessorHookConsumers().get(0); + + HookNotificationV1.EntityCreateRequest message1 = createTestMessage(); + HookNotificationV1.EntityCreateRequest message2 = createTestMessage(); + + AtlasKafkaMessage kafkaMessage1 = new AtlasKafkaMessage<>(message1, 0, "ATLAS_HOOK", 0); + AtlasKafkaMessage kafkaMessage2 = new AtlasKafkaMessage<>(message2, 1, "ATLAS_HOOK", 0); + + doNothing().when(mockPreprocessConsumer).commit(any(TopicPartition.class), anyLong()); + + hookConsumer.handleMessage(kafkaMessage1); + hookConsumer.handleMessage(kafkaMessage2); + + verify(mockPreprocessConsumer, times(2)).commit(any(TopicPartition.class), anyLong()); + }); + } + + @Test + public void testPreprocessorErrorHandling_EmptyMessage() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.getPreprocessorHookConsumers().get(0); + + HookNotificationV1.EntityCreateRequest emptyMessage = new HookNotificationV1.EntityCreateRequest("test_user", Collections.emptyList()); + + AtlasKafkaMessage kafkaMessage = new AtlasKafkaMessage<>(emptyMessage, 0, "ATLAS_HOOK", 0); + + doNothing().when(mockPreprocessConsumer).commit(any(TopicPartition.class), anyLong()); + + hookConsumer.handleMessage(kafkaMessage); + + verify(mockPreprocessConsumer, times(1)).commit(any(TopicPartition.class), anyLong()); + }); + } + + @Test + public void testPreprocessorErrorHandling_InvalidEntityType() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + when(typeRegistry.getType(anyString())).thenReturn(null); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.getPreprocessorHookConsumers().get(0); + + Referenceable entity = new Referenceable("invalid_type_that_does_not_exist"); + entity.set(QUALIFIED_NAME, "test.entity@cluster"); + HookNotificationV1.EntityCreateRequest message = new HookNotificationV1.EntityCreateRequest("test_user", Collections.singletonList(entity)); + + AtlasKafkaMessage kafkaMessage = new AtlasKafkaMessage<>(message, 0, "ATLAS_HOOK", 0); + + doNothing().when(mockPreprocessConsumer).commit(any(TopicPartition.class), anyLong()); + + hookConsumer.handleMessage(kafkaMessage); + + verify(mockPreprocessConsumer, times(1)).commit(any(TopicPartition.class), anyLong()); + }); + } + + @Test + public void testPreprocessorErrorHandling_ProcessingException() throws Exception { + withMockedNotificationProvider(() -> { + mockParallelProcessingEnabled(true); + + NotificationConsumer mockPreprocessConsumer = mockHookNotificationConsumer(); + mockCreateConsumers(notificationInterface, NotificationInterface.NotificationType.HOOK_PREPROCESS, 1, + Collections.singletonList(mockPreprocessConsumer)); + + NotificationHookConsumer notificationHookConsumer = new NotificationHookConsumer(notificationInterface, atlasEntityStore, serviceState, instanceConverter, typeRegistry, metricsUtil, null, asyncImporter, typeDefStore); + + NotificationHookConsumer.HookConsumer hookConsumer = notificationHookConsumer.getPreprocessorHookConsumers().get(0); + + HookNotificationV1.EntityCreateRequest message = createTestMessage(); + AtlasKafkaMessage kafkaMessage = new AtlasKafkaMessage<>(message, 0, "ATLAS_HOOK", 0); + + doThrow(new RuntimeException("Kafka commit failed")).when(mockPreprocessConsumer).commit(any(TopicPartition.class), anyLong()); + + try { + hookConsumer.handleMessage(kafkaMessage); + } catch (Exception e) { + // optional: exception may surface from handleMessage + } + + verify(mockPreprocessConsumer, times(1)).commit(any(TopicPartition.class), anyLong()); + }); + } + + //Helper methods + /** + * Sets {@code atlas.notification.parallel.processing.enabled} on the live {@link ApplicationProperties} + * configuration so {@link AtlasConfiguration#ATLAS_PARALLEL_PROCESSING_ENABLED#getBoolean()} reflects it. + */ + public static void setParallelProcessingEnabled(boolean enabled) throws AtlasException { + ApplicationProperties.get().setProperty( + AtlasConfiguration.ATLAS_PARALLEL_PROCESSING_ENABLED.getPropertyName(), enabled); + } + + /** + * Resets parallel processing flag to {@code false} (matches enum default). + */ + public static void resetParallelProcessingEnabled() throws AtlasException { + setParallelProcessingEnabled(false); + } + + /** + * Mirrors {@link org.apache.atlas.notification.preprocessor.NotificationPreProcessor::getRoutingQualifiedName(Object)} + * for a raw qualified-name string: drop the first {@code ':'} suffix (metadata/timestamp), same as production. + */ + public static String stripQualifiedNameForRouting(String fullQualifiedNameFromEntity) { + if (StringUtils.isBlank(fullQualifiedNameFromEntity)) { + return null; + } + int colon = fullQualifiedNameFromEntity.indexOf(':'); + if (colon != -1) { + return fullQualifiedNameFromEntity.substring(0, colon); + } + return fullQualifiedNameFromEntity; + } + + /** + * Mirrors {@code NotificationPreProcessor#getRoutingKey(String, String, Object, NotificationMetadata)} for the + * qualifiedName branch (no rename): strip {@code @cluster}, then first two dot-separated components. + */ + public static String routingKeyFromStrippedQualifiedName(String routingQualifiedName) { + if (StringUtils.isBlank(routingQualifiedName)) { + return null; + } + int at = routingQualifiedName.indexOf('@'); + String qNameWithoutCluster = at != -1 ? routingQualifiedName.substring(0, at) : routingQualifiedName; + String[] parts = qNameWithoutCluster.split("\\."); + if (parts.length >= 2) { + return parts[0] + "." + parts[1]; + } + if (parts.length == 1) { + return parts[0]; + } + return StringUtils.isNotBlank(qNameWithoutCluster) ? qNameWithoutCluster : routingQualifiedName; + } + + /** + * End-to-end: entity {@code qualifiedName} attribute value → routing key passed to {@link org.apache.atlas.notification.QualifiedNameRouter#getTargetTopic(String)}. + */ + public static String routingKeyFromFullQualifiedNameByPreprocessor(String fullQualifiedNameFromEntity) { + String stripped = stripQualifiedNameForRouting(fullQualifiedNameFromEntity); + if (stripped == null) { + return null; + } + return routingKeyFromStrippedQualifiedName(stripped); + } + + public static Configuration defaultPreprocessorConfiguration() { + Configuration config = new PropertiesConfiguration(); + config.setProperty("atlas.notification.processor.metadata.topic.count", 5); + config.setProperty("atlas.notification.processor.lineage.topic.count", 3); + config.setProperty("atlas.notification.processor.lineage.topic.enabled", true); + config.setProperty("atlas.notification.hook.maxretries", 3); + config.setProperty("atlas.notification.hook.failedcachesize", 10); + return config; + } + + public static AtlasEntity hiveTableEntity(String qualifiedName) { + AtlasEntity entity = new AtlasEntity(); + entity.setTypeName("hive_table"); + entity.setGuid(java.util.UUID.randomUUID().toString()); + entity.setAttribute("qualifiedName", qualifiedName); + entity.setAttribute("name", "t"); + return entity; + } + + public static EntityCreateRequestV2 entityCreateRequestV2(String user, AtlasEntity... entities) { + List list = new ArrayList<>(); + Collections.addAll(list, entities); + AtlasEntitiesWithExtInfo extInfo = new AtlasEntitiesWithExtInfo(list, null); + return new EntityCreateRequestV2(user, extInfo); + } + + public static AtlasKafkaMessage preprocessKafkaMessage(HookNotification notification) { + long now = System.currentTimeMillis(); + return new AtlasKafkaMessage<>(notification, 0L, "ATLAS_HOOK", 0, now, false); + } +} From 419a392c76263832f88289fd732b3ec30893e6ab Mon Sep 17 00:00:00 2001 From: Radhika Kundam Date: Wed, 17 Jun 2026 16:14:59 -0700 Subject: [PATCH 2/3] ATLAS-5320: Distributed Notification Processing - fixing failed tests --- test-tools/src/main/resources/solr/core-template/solrconfig.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test-tools/src/main/resources/solr/core-template/solrconfig.xml b/test-tools/src/main/resources/solr/core-template/solrconfig.xml index 71b26edb19d..52f6db5b980 100644 --- a/test-tools/src/main/resources/solr/core-template/solrconfig.xml +++ b/test-tools/src/main/resources/solr/core-template/solrconfig.xml @@ -445,7 +445,7 @@ --> edismax - 35x_t 5j9_t 7wl_t a9x_t but_t dfp_l f0l_t i6d_l iyt_l jr9_t kjp_s lc5_t m4l_s mx1_t ohx_t xz9_i 1151_t 12px_t 14at_l 15vp_t 1891_t 19tx_t 1bet_t 1czp_t 1ekl_t 1gxx_t 1iit_l 1k3p_t 1lol_t 1o1x_t 1qf9_t 1ssl_t 1v5x_t 1wqt_t 1z45_t 20p1_t 4xs5_t 5af9_s 58ud_s 56h1_s 54w5_s 581x_t 5fyd_t 5c05_t 5dl1_l 5edh_t 5kp1_t 5j45_t 5m9x_t 5tdx_t 5yx1_t 63np_t 61ad_t 622t_t 60hx_t 69z9_t 696t_t 6h39_l 6io5_l 6arp_t 6gat_t 6rd1_l 6sxx_l 6ps5_t 6o79_t 6uit_t 6k91_t 6l1h_t 6mmd_t 74sl_t 75l1_l 71mt_t 6yh1_l 701x_l 6vb9_t 737p_t 7f2d_t 7fut_t 7hfp_t 775x_t 7j0l_t 7e9x_t 7gn9_t 7i85_t 7cp1_t a8zp_t a139_t aakl_t a6md_t a9s5_t a879_t ak1x_t ac5h_t amf9_t ahol_t aj9h_t almt_t avwl_t ao05_t atj9_t av45_t f951_t f211_t f6rp_t f8cl_t f56t_t f0g5_l f3lx_t f5z9_t f18l_l fapx_l ffgl_t fg91_t f9xh_l fbid_t fcat_i feo5_t fjet_t fimd_t fk79_t fls5_i fpqd_t frb9_t fv9h_t fw1x_i ftol_t fxmt_t fwud_t g2dh_t g4qt_t g005_t g35x_t g1l1_t fyf9_l g3yd_t fz7p_l htl1_l hqf9_t hs05_l hxj9_l hybp_t i1hh_i i3ut_t hzwl_t j6kl_t jif9_l jk05_i jcw5_t jksl_i jll1_i jmdh_f jsp1_d jrwl_l krgl_t kwzp_l l05h_t l3b9_l l4w5_t l1qd_t l6h1_l l81x_t + 35x_t 5j9_t 7wl_t a9x_t but_t dfp_l f0l_t i6d_l iyt_l jr9_t kjp_s lc5_t m4l_s mx1_t ohx_t xz9_i 1151_t 12px_t 14at_l 15vp_t 1891_t 19tx_t 1bet_t 1czp_t 1ekl_t 1gxx_t 1iit_l 1k3p_t 1lol_t 1o1x_t 1qf9_t 1ssl_t 1v5x_t 1wqt_t 1z45_t 20p1_t 2rk5_l 2t51_l 50xx_t 5dl1_s 5c05_s 59mt_s 581x_s 5b7p_t 5j45_t 5f5x_t 5gqt_l 5hj9_t 5nut_t 5m9x_t 5pfp_t 5wjp_t 622t_t 66th_t 64g5_t 658l_t 63np_t 6d51_t 6ccl_t 6k91_l 6ltx_l 6dxh_t 6jgl_t 6uit_l 6w3p_l 6sxx_t 6rd1_t 6xol_t 6net_t 6o79_t 6ps5_t 77yd_t 78qt_l 74sl_t 71mt_l 737p_l 6yh1_t 76dh_t 7i85_t 7j0l_t 7klh_t 7abp_t 7m6d_t 7hfp_t 7jt1_t 7ldx_t 7fut_t ac5h_t a491_t adqd_t a9s5_t acxx_t abd1_t an7p_t afb9_t apl1_t akud_t amf9_t aosl_t az2d_t ar5x_t awp1_t ay9x_t fcat_t f56t_t f9xh_t fbid_t f8cl_t f3lx_l f6rp_t f951_t f4ed_l fdvp_l fimd_t fjet_t fd39_l feo5_t ffgl_i fhtx_t fmkl_t fls5_t fnd1_t foxx_i fsw5_t fuh1_t fyf9_t fz7p_i fwud_t g0sl_t g005_t g5j9_t g7wl_t g35x_t g6bp_t g4qt_t g1l1_l g745_t g2dh_l hwqt_l htl1_t hv5x_l i0p1_l i1hh_t i4n9_i i70l_t i32d_t j9qd_t jll1_l jn5x_i jg1x_t jnyd_i joqt_i jpj9_f jvut_d jv2d_l kumd_t l05h_l l3b9_t l6h1_l l81x_t l4w5_t l9mt_l lb7p_t * true true From b79f3044814079b5c784b8600d9a2d7a869b4021 Mon Sep 17 00:00:00 2001 From: Radhika Kundam Date: Thu, 18 Jun 2026 10:23:38 -0700 Subject: [PATCH 3/3] ATLAS-5320: Distributed Notification Processing - fixing failed tests NotificationHookConsumerTest --- .../atlas/notification/NotificationHookConsumerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerTest.java b/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerTest.java index 7529850533b..2d9bb24a06f 100644 --- a/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerTest.java +++ b/webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerTest.java @@ -1844,7 +1844,7 @@ public void testHookConsumerRecordFailedMessages() throws Exception { Object hookConsumer = createHookConsumer(consumer, notificationConsumer); Object entityProcessor = getEntityProcessor(hookConsumer); - Method recordFailedMethod = SerialEntityProcessor.class.getDeclaredMethod("recordFailedMessages", List.class); + Method recordFailedMethod = SerialEntityProcessor.class.getDeclaredMethod("recordFailedMessages", String.class, List.class); recordFailedMethod.setAccessible(true); // Add some failed messages @@ -1855,7 +1855,7 @@ public void testHookConsumerRecordFailedMessages() throws Exception { failedMessages.add("failed message 1"); failedMessages.add("failed message 2"); - recordFailedMethod.invoke(entityProcessor, failedMessages); + recordFailedMethod.invoke(entityProcessor, "test-topic", failedMessages); assertTrue(failedMessages.isEmpty()); // Should be cleared after recording }