From 952701be9d720c3d9a93913d4f4b52da834ad4eb Mon Sep 17 00:00:00 2001 From: Radhika Kundam Date: Mon, 1 Jun 2026 16:00:32 -0700 Subject: [PATCH] ATLAS-4464: Concurrent ingest implementation --- .../org/apache/atlas/pc/StatusReporter.java | 12 +- .../org/apache/atlas/pc/WorkItemManager.java | 20 +- .../janus/AtlasJanusGraphDatabase.java | 3 + intg/pom.xml | 11 +- .../org/apache/atlas/AtlasConfiguration.java | 1 + .../atlas/kafka/AtlasKafkaConsumer.java | 28 +- .../notification/NotificationConsumer.java | 2 + .../apache/atlas/kafka/KafkaConsumerTest.java | 7 +- .../AbstractNotificationConsumerTest.java | 5 + repository/pom.xml | 5 + .../ConcurrentEntityProcessor.java | 102 ++ .../NotificationEntityProcessor.java | 41 + .../NotificationHookConsumer.java | 1194 +--------------- .../notification/SerialEntityProcessor.java | 1206 +++++++++++++++++ .../TopicPartitionOffsetResult.java | 61 + .../atlas/notification/pc/Consumer.java | 86 ++ .../notification/pc/ConsumerBuilder.java | 73 + .../notification/pc/DependentsTracking.java | 73 + .../apache/atlas/notification/pc/Manager.java | 83 ++ .../atlas/notification/pc/MiscUtils.java | 43 + .../notification/pc/ReferenceKeeper.java | 118 ++ .../notification/pc/ResultsCollector.java | 50 + .../apache/atlas/notification/pc/Ticket.java | 329 +++++ .../preprocessor/EntityPreprocessor.java | 4 +- .../org/apache/atlas/util/AdaptiveWaiter.java | 79 ++ .../notification/AdaptiveWaiterTest.java | 5 +- .../notification/ReferenceFinderTest.java | 174 +++ 27 files changed, 2653 insertions(+), 1162 deletions(-) create mode 100644 webapp/src/main/java/org/apache/atlas/notification/ConcurrentEntityProcessor.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/NotificationEntityProcessor.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/SerialEntityProcessor.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/TopicPartitionOffsetResult.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/pc/Consumer.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/pc/ConsumerBuilder.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/pc/DependentsTracking.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/pc/Manager.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/pc/MiscUtils.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/pc/ReferenceKeeper.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/pc/ResultsCollector.java create mode 100644 webapp/src/main/java/org/apache/atlas/notification/pc/Ticket.java create mode 100644 webapp/src/main/java/org/apache/atlas/util/AdaptiveWaiter.java create mode 100644 webapp/src/test/java/org/apache/atlas/notification/ReferenceFinderTest.java diff --git a/common/src/main/java/org/apache/atlas/pc/StatusReporter.java b/common/src/main/java/org/apache/atlas/pc/StatusReporter.java index 147fd4e13ca..39b52b6942c 100644 --- a/common/src/main/java/org/apache/atlas/pc/StatusReporter.java +++ b/common/src/main/java/org/apache/atlas/pc/StatusReporter.java @@ -22,16 +22,17 @@ import org.slf4j.LoggerFactory; import java.util.Arrays; -import java.util.HashSet; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentSkipListSet; public class StatusReporter { private static final Logger LOG = LoggerFactory.getLogger(StatusReporter.class); - private final Map producedItems = new LinkedHashMap<>(); - private final Set processedSet = new HashSet<>(); + private final Map producedItems = Collections.synchronizedMap(new LinkedHashMap<>()); + private final Set processedSet = new ConcurrentSkipListSet<>(); private final long timeoutDuration; private long lastAck; @@ -51,6 +52,11 @@ public void processed(T item) { this.processedSet.add(item); } + public void processed(T item, U index) { + this.processedSet.add(item); + this.producedItems.put(item, index); + } + public void processed(T[] index) { this.processedSet.addAll(Arrays.asList(index)); } diff --git a/common/src/main/java/org/apache/atlas/pc/WorkItemManager.java b/common/src/main/java/org/apache/atlas/pc/WorkItemManager.java index c1f79e417ec..a77941e0ee7 100644 --- a/common/src/main/java/org/apache/atlas/pc/WorkItemManager.java +++ b/common/src/main/java/org/apache/atlas/pc/WorkItemManager.java @@ -18,7 +18,7 @@ package org.apache.atlas.pc; -import org.apache.curator.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.commons.lang3.concurrent.BasicThreadFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,7 +46,7 @@ public class WorkItemManager> { public WorkItemManager(WorkItemBuilder builder, String namePrefix, int batchSize, int numWorkers, boolean collectResults) { this.numWorkers = numWorkers; this.workQueue = new LinkedBlockingQueue<>(batchSize * numWorkers); - this.service = Executors.newFixedThreadPool(numWorkers, new ThreadFactoryBuilder().setNameFormat(namePrefix + "-%d").build()); + this.service = Executors.newFixedThreadPool(numWorkers, new BasicThreadFactory.Builder().namingPattern(namePrefix + "-%d").build()); createConsumers(builder, numWorkers, collectResults); @@ -147,4 +147,20 @@ private int getAvgCommitTimeSeconds() { return (int) ((commitTimeSeconds / consumers.size()) / 1000); } + + protected List getTypedResults() { + if (getResults().isEmpty()) { + return null; + } + + List ret = new ArrayList<>(); + Object result; + while (((result = getResults().poll())) != null) { + T res = (T) result; + + ret.add(res); + } + + return ret; + } } diff --git a/graphdb/janus/src/main/java/org/apache/atlas/repository/graphdb/janus/AtlasJanusGraphDatabase.java b/graphdb/janus/src/main/java/org/apache/atlas/repository/graphdb/janus/AtlasJanusGraphDatabase.java index 186b817d4fa..cddf6180f35 100644 --- a/graphdb/janus/src/main/java/org/apache/atlas/repository/graphdb/janus/AtlasJanusGraphDatabase.java +++ b/graphdb/janus/src/main/java/org/apache/atlas/repository/graphdb/janus/AtlasJanusGraphDatabase.java @@ -235,6 +235,9 @@ static JanusGraph initJanusGraph(Configuration config) { } else { throw new RuntimeException(e); } + } catch (Exception e) { + LOG.error("Error: Error initializing graph"); + throw e; } } diff --git a/intg/pom.xml b/intg/pom.xml index ab99e9d067f..29b9093f14b 100644 --- a/intg/pom.xml +++ b/intg/pom.xml @@ -109,11 +109,6 @@ - - org.mockito - mockito-all - - org.springframework spring-context @@ -127,6 +122,12 @@ test + + org.mockito + mockito-all + test + + org.testng testng diff --git a/intg/src/main/java/org/apache/atlas/AtlasConfiguration.java b/intg/src/main/java/org/apache/atlas/AtlasConfiguration.java index 994945a6743..e30b80995b9 100644 --- a/intg/src/main/java/org/apache/atlas/AtlasConfiguration.java +++ b/intg/src/main/java/org/apache/atlas/AtlasConfiguration.java @@ -92,6 +92,7 @@ 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 SOLR_INDEX_TX_LOG_TTL_CONF("write.ahead.log.ttl.in.hours", 240), //10 days default diff --git a/notification/src/main/java/org/apache/atlas/kafka/AtlasKafkaConsumer.java b/notification/src/main/java/org/apache/atlas/kafka/AtlasKafkaConsumer.java index 239691caaca..a4394e9c4b2 100644 --- a/notification/src/main/java/org/apache/atlas/kafka/AtlasKafkaConsumer.java +++ b/notification/src/main/java/org/apache/atlas/kafka/AtlasKafkaConsumer.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -46,6 +47,7 @@ public class AtlasKafkaConsumer extends AbstractNotificationConsumer { private final KafkaConsumer kafkaConsumer; private final boolean autoCommitEnabled; private final long pollTimeoutMilliSeconds; + private final Duration duration; public AtlasKafkaConsumer(NotificationInterface.NotificationType notificationType, KafkaConsumer kafkaConsumer, boolean autoCommitEnabled, long pollTimeoutMilliSeconds) { this(notificationType.getDeserializer(), kafkaConsumer, autoCommitEnabled, pollTimeoutMilliSeconds); @@ -57,6 +59,7 @@ public AtlasKafkaConsumer(AtlasNotificationMessageDeserializer deserializer, this.autoCommitEnabled = autoCommitEnabled; this.kafkaConsumer = kafkaConsumer; this.pollTimeoutMilliSeconds = pollTimeoutMilliSeconds; + this.duration = Duration.ofMillis(pollTimeoutMilliSeconds); } @Override @@ -72,7 +75,7 @@ public Set subscription() { @Override public void commit(TopicPartition partition, long offset) { if (!autoCommitEnabled) { - LOG.debug(" commiting the offset ==>> {}", offset); + LOG.debug(" committing the offset ==>> {}", offset); kafkaConsumer.commitSync(Collections.singletonMap(partition, new OffsetAndMetadata(offset))); } @@ -92,37 +95,42 @@ public void wakeup() { } } + @Override + public void poll() { + this.kafkaConsumer.poll(this.duration); + } + public List> receive() { return this.receive(this.pollTimeoutMilliSeconds); } @Override public List> receive(long timeoutMilliSeconds) { - return receive(this.pollTimeoutMilliSeconds, null); + return receive(this.duration, null); } @Override public List> receiveWithCheckedCommit(Map lastCommittedPartitionOffset) { - return receive(this.pollTimeoutMilliSeconds, lastCommittedPartitionOffset); + return receive(this.duration, lastCommittedPartitionOffset); } @Override public List> receiveRawRecordsWithCheckedCommit(Map lastCommittedPartitionOffset) { - return receiveRawRecords(this.pollTimeoutMilliSeconds, lastCommittedPartitionOffset); + return receiveRawRecords(this.duration, lastCommittedPartitionOffset); } - private List> receiveRawRecords(long timeoutMilliSeconds, Map lastCommittedPartitionOffset) { - return receive(timeoutMilliSeconds, lastCommittedPartitionOffset, true); + private List> receiveRawRecords(Duration duration, Map lastCommittedPartitionOffset) { + return receive(duration, lastCommittedPartitionOffset, true); } - private List> receive(long timeoutMilliSeconds, Map lastCommittedPartitionOffset) { - return receive(timeoutMilliSeconds, lastCommittedPartitionOffset, false); + private List> receive(Duration duration, Map lastCommittedPartitionOffset) { + return receive(duration, lastCommittedPartitionOffset, false); } - private List> receive(long timeoutMilliSeconds, Map lastCommittedPartitionOffset, boolean isRawDataRequired) { + private List> receive(Duration duration, Map lastCommittedPartitionOffset, boolean isRawDataRequired) { List> messages = new ArrayList<>(); - ConsumerRecords records = kafkaConsumer != null ? kafkaConsumer.poll(timeoutMilliSeconds) : null; + ConsumerRecords records = kafkaConsumer != null ? kafkaConsumer.poll(duration) : null; if (records != null) { for (ConsumerRecord record : records) { diff --git a/notification/src/main/java/org/apache/atlas/notification/NotificationConsumer.java b/notification/src/main/java/org/apache/atlas/notification/NotificationConsumer.java index 2348b322bd8..981a5fcdee7 100644 --- a/notification/src/main/java/org/apache/atlas/notification/NotificationConsumer.java +++ b/notification/src/main/java/org/apache/atlas/notification/NotificationConsumer.java @@ -70,6 +70,8 @@ public interface NotificationConsumer { */ List> receiveRawRecordsWithCheckedCommit(Map lastCommittedPartitionOffset); + void poll(); + Set getTopicPartition(); Set subscription(); diff --git a/notification/src/test/java/org/apache/atlas/kafka/KafkaConsumerTest.java b/notification/src/test/java/org/apache/atlas/kafka/KafkaConsumerTest.java index 733bc0f2fe7..32bee136d63 100644 --- a/notification/src/test/java/org/apache/atlas/kafka/KafkaConsumerTest.java +++ b/notification/src/test/java/org/apache/atlas/kafka/KafkaConsumerTest.java @@ -40,6 +40,7 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import java.time.Duration; import java.util.Collections; import java.util.List; @@ -98,7 +99,7 @@ public void testNextVersionMismatch() { kafkaConsumer.assign(Collections.singletonList(tp)); - when(kafkaConsumer.poll(100L)).thenReturn(records); + when(kafkaConsumer.poll(Duration.ofMillis(100L))).thenReturn(records); AtlasKafkaConsumer consumer = new AtlasKafkaConsumer<>(NotificationType.HOOK, kafkaConsumer, false, 100L); @@ -149,7 +150,7 @@ public void checkCrossCombatMessageVersionTest() { kafkaConsumer.assign(Collections.singletonList(tp)); - when(kafkaConsumer.poll(100L)).thenReturn(records); + when(kafkaConsumer.poll(Duration.ofMillis(100L))).thenReturn(records); AtlasKafkaConsumer consumer = new AtlasKafkaConsumer<>(NotificationType.HOOK, kafkaConsumer, false, 100L); @@ -166,7 +167,7 @@ private List> testReceiveHelper(EntityUpdate List> klist = Collections.singletonList(new ConsumerRecord<>(topic, 0, 0L, "mykey", json)); ConsumerRecords records = new ConsumerRecords<>(Collections.singletonMap(tp, klist)); - when(kafkaConsumer.poll(100)).thenReturn(records); + when(kafkaConsumer.poll(Duration.ofMillis(100L))).thenReturn(records); kafkaConsumer.assign(Collections.singletonList(tp)); diff --git a/notification/src/test/java/org/apache/atlas/notification/AbstractNotificationConsumerTest.java b/notification/src/test/java/org/apache/atlas/notification/AbstractNotificationConsumerTest.java index 8917e9f31a6..bc5f590ae3b 100644 --- a/notification/src/test/java/org/apache/atlas/notification/AbstractNotificationConsumerTest.java +++ b/notification/src/test/java/org/apache/atlas/notification/AbstractNotificationConsumerTest.java @@ -221,6 +221,11 @@ public List> receiveWithCheckedCommit(Map> receiveRawRecordsWithCheckedCommit(Map lastCommittedPartitionOffset) { return null; } + + @Override + public void poll() { + // Do nothing + } } public static class TestMessageDeserializer extends AbstractMessageDeserializer { diff --git a/repository/pom.xml b/repository/pom.xml index 000c5197ac2..6715e1eabdf 100644 --- a/repository/pom.xml +++ b/repository/pom.xml @@ -163,6 +163,11 @@ commons-lang3 + + org.apache.hbase + hbase-client + + org.apache.poi poi diff --git a/webapp/src/main/java/org/apache/atlas/notification/ConcurrentEntityProcessor.java b/webapp/src/main/java/org/apache/atlas/notification/ConcurrentEntityProcessor.java new file mode 100644 index 00000000000..0576ce4db51 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/ConcurrentEntityProcessor.java @@ -0,0 +1,102 @@ +/** + * 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.kafka.AtlasKafkaMessage; +import org.apache.atlas.model.notification.HookNotification; +import org.apache.atlas.notification.pc.ConsumerBuilder; +import org.apache.atlas.notification.pc.Manager; +import org.apache.atlas.notification.pc.MiscUtils; +import org.apache.atlas.notification.pc.ReferenceKeeper; +import org.apache.atlas.notification.pc.Ticket; +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.type.AtlasTypeRegistry; +import org.apache.atlas.util.AtlasMetricsUtil; +import org.apache.commons.configuration2.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.core.Authentication; + +import java.util.Map; + +public class ConcurrentEntityProcessor implements NotificationEntityProcessor { + private static final Logger LOG = LoggerFactory.getLogger(ConcurrentEntityProcessor.class); + + private static final String PROPERTY_CONSUMER_NUM_WORKERS = "atlas.notification.consumer.numWorkers"; + private static final String PROPERTY_CONSUMER_BATCH_SIZE = "atlas.notification.consumer.batchSize"; + private static final int CONSUMER_NUM_WORKERS_DEFAULT = 10; + private static final int CONSUMER_BATCH_SIZE_DEFAULT = 100; + + private static int consumerNumWorkers = CONSUMER_NUM_WORKERS_DEFAULT; + private static int consumerBatchSize = CONSUMER_BATCH_SIZE_DEFAULT; + private final Manager manager; + private final ReferenceKeeper referenceKeeper; + + public ConcurrentEntityProcessor(Configuration applicationProperties, AtlasMetricsUtil metricsUtil, Map authnCache, + AtlasEntityStore entityStore, AtlasInstanceConverter instanceConverter, EntityCorrelationManager entityCorrelationManager, + AtlasTypeRegistry typeRegistry, Logger failedLog, Logger largeMessagesLog, + AsyncImporter asyncImporter) { + this.referenceKeeper = new ReferenceKeeper(); + + ConsumerBuilder consumerBuilder = new ConsumerBuilder(this.referenceKeeper, applicationProperties, metricsUtil, authnCache, entityStore, + instanceConverter, entityCorrelationManager, typeRegistry, failedLog, largeMessagesLog, asyncImporter); + + this.manager = new Manager(this.referenceKeeper, consumerBatchSize, consumerNumWorkers, consumerBuilder); + MiscUtils.extractAllProcessTypeNames(Ticket.getProcessNameTypes(), typeRegistry); + } + + @Override + public TopicPartitionOffsetResult handleMessage(AtlasKafkaMessage msg) { + return this.manager.submit(new Ticket(msg)); + } + + @Override + public TopicPartitionOffsetResult collectResults() { + return this.manager.getResult(); + } + + @Override + public void shutdown() { + this.manager.shutdown(); + } + + static { + try { + Configuration config = ApplicationProperties.get(); + consumerNumWorkers = config.getInt(PROPERTY_CONSUMER_NUM_WORKERS, CONSUMER_NUM_WORKERS_DEFAULT); + if (consumerNumWorkers <= 0) { + LOG.warn("{} set to invalid value: {}. Using default.", PROPERTY_CONSUMER_NUM_WORKERS, consumerNumWorkers); + consumerNumWorkers = CONSUMER_NUM_WORKERS_DEFAULT; + } + + LOG.info("{} = {}", PROPERTY_CONSUMER_NUM_WORKERS, consumerNumWorkers); + consumerBatchSize = config.getInt(PROPERTY_CONSUMER_BATCH_SIZE, CONSUMER_BATCH_SIZE_DEFAULT); + if (consumerBatchSize <= 0) { + LOG.warn("{} set to invalid value: {}. Using default.", PROPERTY_CONSUMER_BATCH_SIZE, consumerBatchSize); + consumerBatchSize = CONSUMER_BATCH_SIZE_DEFAULT; + } + + LOG.info("{} = {}", PROPERTY_CONSUMER_BATCH_SIZE, consumerBatchSize); + } catch (Exception exception) { + LOG.error("Error fetching configuration. Will use default!"); + } + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/NotificationEntityProcessor.java b/webapp/src/main/java/org/apache/atlas/notification/NotificationEntityProcessor.java new file mode 100644 index 00000000000..739392f9c97 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/NotificationEntityProcessor.java @@ -0,0 +1,41 @@ +/** + * 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.kafka.AtlasKafkaMessage; +import org.apache.atlas.model.notification.HookNotification; + +public interface NotificationEntityProcessor { + /*** + * Handle a given message. + * @param msg + * @return topic-partition-offset that got committed + */ + TopicPartitionOffsetResult handleMessage(AtlasKafkaMessage msg); + + /*** + * Collect results which were fetched after the last message was processed. + * @return + */ + TopicPartitionOffsetResult collectResults(); + + /*** + * Shutdown the processor. + */ + void shutdown(); +} 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 bc9b97bf8f9..ec3f69bbc4c 100644 --- a/webapp/src/main/java/org/apache/atlas/notification/NotificationHookConsumer.java +++ b/webapp/src/main/java/org/apache/atlas/notification/NotificationHookConsumer.java @@ -20,63 +20,25 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.atlas.ApplicationProperties; -import org.apache.atlas.AtlasClient; -import org.apache.atlas.AtlasClientV2; import org.apache.atlas.AtlasConfiguration; -import org.apache.atlas.AtlasErrorCode; import org.apache.atlas.AtlasException; -import org.apache.atlas.RequestContext; import org.apache.atlas.exception.AtlasBaseException; import org.apache.atlas.ha.HAConfiguration; import org.apache.atlas.hook.AtlasHook; import org.apache.atlas.kafka.AtlasKafkaMessage; import org.apache.atlas.kafka.KafkaNotification; import org.apache.atlas.listener.ActiveStateChangeHandler; -import org.apache.atlas.model.instance.AtlasEntity; -import org.apache.atlas.model.instance.AtlasEntity.AtlasEntitiesWithExtInfo; -import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo; -import org.apache.atlas.model.instance.AtlasEntityHeader; -import org.apache.atlas.model.instance.AtlasObjectId; -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.HookNotification.EntityDeleteRequestV2; -import org.apache.atlas.model.notification.HookNotification.EntityPartialUpdateRequestV2; -import org.apache.atlas.model.notification.HookNotification.EntityUpdateRequestV2; -import org.apache.atlas.model.notification.ImportNotification.AtlasEntityImportNotification; -import org.apache.atlas.model.notification.ImportNotification.AtlasTypesDefImportNotification; -import org.apache.atlas.model.typedef.AtlasTypesDef; import org.apache.atlas.notification.NotificationInterface.NotificationType; -import org.apache.atlas.notification.preprocessor.EntityPreprocessor; -import org.apache.atlas.notification.preprocessor.GenericEntityPreprocessor; -import org.apache.atlas.notification.preprocessor.PreprocessorContext; -import org.apache.atlas.notification.preprocessor.PreprocessorContext.PreprocessAction; 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.EntityCorrelationStore; -import org.apache.atlas.repository.store.graph.v2.AtlasEntityStream; -import org.apache.atlas.repository.store.graph.v2.AtlasGraphUtilsV2; import org.apache.atlas.service.Service; -import org.apache.atlas.type.AtlasEntityType; -import org.apache.atlas.type.AtlasStructType.AtlasAttribute; import org.apache.atlas.type.AtlasTypeRegistry; -import org.apache.atlas.util.AtlasMetricsCounter; +import org.apache.atlas.util.AdaptiveWaiter; import org.apache.atlas.util.AtlasMetricsUtil; -import org.apache.atlas.util.AtlasMetricsUtil.NotificationStat; -import org.apache.atlas.utils.AtlasJson; -import org.apache.atlas.utils.AtlasPerfTracer; -import org.apache.atlas.utils.LruCache; -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.atlas.web.filters.AuditFilter; -import org.apache.atlas.web.filters.AuditFilter.AuditLog; import org.apache.atlas.web.service.ServiceState; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.collections.MapUtils; import org.apache.commons.collections4.map.PassiveExpiringMap; import org.apache.commons.configuration2.Configuration; import org.apache.commons.lang3.StringUtils; @@ -86,41 +48,26 @@ import org.springframework.context.annotation.DependsOn; import org.springframework.context.annotation.Lazy; import org.springframework.core.annotation.Order; -import org.springframework.security.authentication.UsernamePasswordAuthenticationToken; import org.springframework.security.core.Authentication; -import org.springframework.security.core.GrantedAuthority; -import org.springframework.security.core.context.SecurityContextHolder; -import org.springframework.security.core.userdetails.User; -import org.springframework.security.core.userdetails.UserDetails; import org.springframework.stereotype.Component; import javax.inject.Inject; -import javax.ws.rs.core.Response; -import java.time.Instant; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.ListIterator; import java.util.Map; -import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.regex.Pattern; -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; -import static org.apache.atlas.notification.preprocessor.EntityPreprocessor.TYPE_HIVE_PROCESS; -import static org.apache.atlas.web.security.AtlasAbstractAuthenticationProvider.getAuthoritiesFromUGI; +import static org.apache.atlas.notification.NotificationInterface.NotificationType.ASYNC_IMPORT; /** * Consumer of notifications from hooks e.g., hive hook etc. @@ -130,7 +77,6 @@ @DependsOn(value = {"atlasTypeDefStoreInitializer", "atlasTypeDefGraphStoreV2"}) public class NotificationHookConsumer implements Service, ActiveStateChangeHandler { private static final Logger LOG = LoggerFactory.getLogger(NotificationHookConsumer.class); - private static final Logger PERF_LOG = AtlasPerfTracer.getPerfLogger(NotificationHookConsumer.class); private static final Logger FAILED_LOG = LoggerFactory.getLogger("FAILED"); private static final Logger LARGE_MESSAGES_LOG = LoggerFactory.getLogger("LARGE_MESSAGES"); @@ -168,49 +114,21 @@ 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 SC_OK = 200; - private static final int SC_BAD_REQUEST = 400; - private static final String TYPE_HIVE_COLUMN_LINEAGE = "hive_column_lineage"; - private static final String ATTRIBUTE_INPUTS = "inputs"; - private static final String ATTRIBUTE_QUALIFIED_NAME = "qualifiedName"; - private static final String EXCEPTION_CLASS_NAME_JANUSGRAPH_EXCEPTION = "JanusGraphException"; - private static final String EXCEPTION_CLASS_NAME_PERMANENTLOCKING_EXCEPTION = "PermanentLockingException"; - 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 THREADNAME_PREFIX = NotificationHookConsumer.class.getSimpleName(); private final AtlasEntityStore atlasEntityStore; private final ServiceState serviceState; private final AtlasInstanceConverter instanceConverter; private final AtlasTypeRegistry typeRegistry; private final AtlasMetricsUtil metricsUtil; - private final int maxRetries; - private final int failedMsgCacheSize; private final int minWaitDuration; private final int maxWaitDuration; - private final int commitBatchSize; - private final boolean skipHiveColumnLineageHive20633; - private final int skipHiveColumnLineageHive20633InputsThreshold; - private final boolean updateHiveProcessNameWithQualifiedName; - private final int largeMessageProcessingTimeThresholdMs; private final boolean consumerDisabled; - private final List entityTypesToIgnore = new ArrayList<>(); - private final List entitiesToIgnore = new ArrayList<>(); - private final List hiveTablesToIgnore = new ArrayList<>(); - private final List hiveTablesToPrune = new ArrayList<>(); - private final List hiveDummyDatabasesToIgnore; - private final List hiveDummyTablesToIgnore; - private final List hiveTablePrefixesToIgnore; - private final Map hiveTablesCache; - private final boolean hiveTypesRemoveOwnedRefAttrs; - private final boolean rdbmsTypesRemoveOwnedRefAttrs; - private final boolean s3V2DirectoryPruneObjectPrefix; - private final boolean preprocessEnabled; - private final boolean createShellEntityForNonExistingReference; private final boolean authorizeUsingMessageUser; - private final boolean sparkProcessAttributes; private final Map authnCache; private final NotificationInterface notificationInterface; @@ -222,7 +140,6 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl private final AsyncImporter asyncImporter; private ExecutorService executors; - private Instant nextStatsLogTime = AtlasMetricsCounter.getNextHourStartTime(Instant.now()); @VisibleForTesting final int consumerRetryInterval; @@ -242,144 +159,20 @@ public NotificationHookConsumer(NotificationInterface notificationInterface, Atl this.lastCommittedPartitionOffset = new HashMap<>(); this.asyncImporter = asyncImporter; - maxRetries = applicationProperties.getInt(CONSUMER_RETRIES_PROPERTY, 3); - failedMsgCacheSize = applicationProperties.getInt(CONSUMER_FAILEDCACHESIZE_PROPERTY, 1); consumerRetryInterval = applicationProperties.getInt(CONSUMER_RETRY_INTERVAL, 500); minWaitDuration = applicationProperties.getInt(CONSUMER_MIN_RETRY_INTERVAL, consumerRetryInterval); // 500 ms by default maxWaitDuration = applicationProperties.getInt(CONSUMER_MAX_RETRY_INTERVAL, minWaitDuration * 60); // 30 sec by default - commitBatchSize = applicationProperties.getInt(CONSUMER_COMMIT_BATCH_SIZE, 50); - - skipHiveColumnLineageHive20633 = applicationProperties.getBoolean(CONSUMER_SKIP_HIVE_COLUMN_LINEAGE_HIVE_20633, false); - skipHiveColumnLineageHive20633InputsThreshold = applicationProperties.getInt(CONSUMER_SKIP_HIVE_COLUMN_LINEAGE_HIVE_20633_INPUTS_THRESHOLD, 15); // skip if avg # of inputs is > 15 - updateHiveProcessNameWithQualifiedName = applicationProperties.getBoolean(CONSUMER_PREPROCESS_HIVE_PROCESS_UPD_NAME_WITH_QUALIFIED_NAME, true); - consumerDisabled = applicationProperties.getBoolean(CONSUMER_DISABLED, false); - largeMessageProcessingTimeThresholdMs = applicationProperties.getInt("atlas.notification.consumer.large.message.processing.time.threshold.ms", 60 * 1000); // 60 sec by default - createShellEntityForNonExistingReference = AtlasConfiguration.NOTIFICATION_CREATE_SHELL_ENTITY_FOR_NON_EXISTING_REF.getBoolean(); - 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(); + + consumerDisabled = applicationProperties.getBoolean(CONSUMER_DISABLED, false); + 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(); int authnCacheTtlSeconds = applicationProperties.getInt(CONSUMER_AUTHORIZE_AUTHN_CACHE_TTL_SECONDS, 300); authnCache = (authorizeUsingMessageUser && authnCacheTtlSeconds > 0) ? new PassiveExpiringMap<>(authnCacheTtlSeconds * 1000L) : null; - String[] patternEntityTypesToIgnore = applicationProperties.getStringArray(CONSUMER_PREPROCESS_ENTITY_TYPE_IGNORE_PATTERN); - String[] patternEntitiesToIgnore = applicationProperties.getStringArray(CONSUMER_PREPROCESS_ENTITY_IGNORE_PATTERN); - - String[] patternHiveTablesToIgnore = applicationProperties.getStringArray(CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_PATTERN); - String[] patternHiveTablesToPrune = applicationProperties.getStringArray(CONSUMER_PREPROCESS_HIVE_TABLE_PRUNE_PATTERN); - - if (patternEntityTypesToIgnore != null) { - for (String pattern : patternEntityTypesToIgnore) { - try { - this.entityTypesToIgnore.add(Pattern.compile(pattern)); - } catch (Throwable t) { - LOG.warn("failed to compile pattern {}", pattern, t); - LOG.warn("Ignoring invalid pattern in configuration {}: {}", CONSUMER_PREPROCESS_ENTITY_TYPE_IGNORE_PATTERN, pattern); - } - } - - LOG.info("{}={}", CONSUMER_PREPROCESS_ENTITY_TYPE_IGNORE_PATTERN, entityTypesToIgnore); - } - - if (patternEntitiesToIgnore != null) { - for (String pattern : patternEntitiesToIgnore) { - try { - this.entitiesToIgnore.add(Pattern.compile(pattern)); - } catch (Throwable t) { - LOG.warn("failed to compile pattern {}", pattern, t); - LOG.warn("Ignoring invalid pattern in configuration {}: {}", CONSUMER_PREPROCESS_ENTITY_IGNORE_PATTERN, pattern); - } - } - - LOG.info("{}={}", CONSUMER_PREPROCESS_ENTITY_IGNORE_PATTERN, entitiesToIgnore); - } - - if (patternHiveTablesToIgnore != null) { - for (String pattern : patternHiveTablesToIgnore) { - try { - hiveTablesToIgnore.add(Pattern.compile(pattern)); - - LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_PATTERN, pattern); - } catch (Throwable t) { - LOG.warn("failed to compile pattern {}", pattern, t); - LOG.warn("Ignoring invalid pattern in configuration {}: {}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_PATTERN, pattern); - } - } - } - - if (patternHiveTablesToPrune != null) { - for (String pattern : patternHiveTablesToPrune) { - try { - hiveTablesToPrune.add(Pattern.compile(pattern)); - - LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_PRUNE_PATTERN, pattern); - } catch (Throwable t) { - LOG.warn("failed to compile pattern {}", pattern, t); - LOG.warn("Ignoring invalid pattern in configuration {}: {}", CONSUMER_PREPROCESS_HIVE_TABLE_PRUNE_PATTERN, pattern); - } - } - } - - if (!hiveTablesToIgnore.isEmpty() || !hiveTablesToPrune.isEmpty()) { - hiveTablesCache = new LruCache<>(applicationProperties.getInt(CONSUMER_PREPROCESS_HIVE_TABLE_CACHE_SIZE, 10000), 0); - } else { - hiveTablesCache = Collections.emptyMap(); - } - - boolean hiveDbIgnoreDummyEnabled = applicationProperties.getBoolean(CONSUMER_PREPROCESS_HIVE_DB_IGNORE_DUMMY_ENABLED, true); - boolean hiveTableIgnoreDummyEnabled = applicationProperties.getBoolean(CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_DUMMY_ENABLED, true); - boolean hiveTableIgnoreNamePrefixEnabled = applicationProperties.getBoolean(CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_NAME_PREFIXES_ENABLED, true); - - LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_DB_IGNORE_DUMMY_ENABLED, hiveDbIgnoreDummyEnabled); - LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_DUMMY_ENABLED, hiveTableIgnoreDummyEnabled); - LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_NAME_PREFIXES_ENABLED, hiveTableIgnoreNamePrefixEnabled); - - if (hiveDbIgnoreDummyEnabled) { - String[] dummyDatabaseNames = applicationProperties.getStringArray(CONSUMER_PREPROCESS_HIVE_DB_IGNORE_DUMMY_NAMES); - - hiveDummyDatabasesToIgnore = trimAndPurge(dummyDatabaseNames, DUMMY_DATABASE); - - LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_DB_IGNORE_DUMMY_NAMES, StringUtils.join(hiveDummyDatabasesToIgnore, ',')); - } else { - hiveDummyDatabasesToIgnore = Collections.emptyList(); - } - - if (hiveTableIgnoreDummyEnabled) { - String[] dummyTableNames = applicationProperties.getStringArray(CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_DUMMY_NAMES); - - hiveDummyTablesToIgnore = trimAndPurge(dummyTableNames, DUMMY_TABLE); - - LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_DUMMY_NAMES, StringUtils.join(hiveDummyTablesToIgnore, ',')); - } else { - hiveDummyTablesToIgnore = Collections.emptyList(); - } - - if (hiveTableIgnoreNamePrefixEnabled) { - String[] ignoreNamePrefixes = applicationProperties.getStringArray(CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_NAME_PREFIXES); - - hiveTablePrefixesToIgnore = trimAndPurge(ignoreNamePrefixes, VALUES_TMP_TABLE_NAME_PREFIX); - - LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_NAME_PREFIXES, StringUtils.join(hiveTablePrefixesToIgnore, ',')); - } else { - hiveTablePrefixesToIgnore = Collections.emptyList(); - } - - LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_PROCESS_UPD_NAME_WITH_QUALIFIED_NAME, updateHiveProcessNameWithQualifiedName); - - hiveTypesRemoveOwnedRefAttrs = applicationProperties.getBoolean(CONSUMER_PREPROCESS_HIVE_TYPES_REMOVE_OWNEDREF_ATTRS, true); - rdbmsTypesRemoveOwnedRefAttrs = applicationProperties.getBoolean(CONSUMER_PREPROCESS_RDBMS_TYPES_REMOVE_OWNEDREF_ATTRS, true); - s3V2DirectoryPruneObjectPrefix = applicationProperties.getBoolean(CONSUMER_PREPROCESS_S3_V2_DIRECTORY_PRUNE_OBJECT_PREFIX, true); - sparkProcessAttributes = this.applicationProperties.getBoolean(CONSUMER_PREPROCESS_SPARK_PROCESS_ATTRIBUTES, false); - preprocessEnabled = skipHiveColumnLineageHive20633 || updateHiveProcessNameWithQualifiedName || hiveTypesRemoveOwnedRefAttrs || rdbmsTypesRemoveOwnedRefAttrs || s3V2DirectoryPruneObjectPrefix || !hiveTablesToIgnore.isEmpty() || !hiveTablesToPrune.isEmpty() || !hiveDummyDatabasesToIgnore.isEmpty() || !hiveDummyTablesToIgnore.isEmpty() || !hiveTablePrefixesToIgnore.isEmpty() || sparkProcessAttributes; entityCorrelationManager = new EntityCorrelationManager(entityCorrelationStore); - - LOG.info("{}={}", CONSUMER_SKIP_HIVE_COLUMN_LINEAGE_HIVE_20633, skipHiveColumnLineageHive20633); - LOG.info("{}={}", CONSUMER_SKIP_HIVE_COLUMN_LINEAGE_HIVE_20633_INPUTS_THRESHOLD, skipHiveColumnLineageHive20633InputsThreshold); - LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TYPES_REMOVE_OWNEDREF_ATTRS, hiveTypesRemoveOwnedRefAttrs); - LOG.info("{}={}", CONSUMER_PREPROCESS_RDBMS_TYPES_REMOVE_OWNEDREF_ATTRS, rdbmsTypesRemoveOwnedRefAttrs); - LOG.info("{}={}", CONSUMER_PREPROCESS_S3_V2_DIRECTORY_PRUNE_OBJECT_PREFIX, s3V2DirectoryPruneObjectPrefix); - LOG.info("{}={}", CONSUMER_COMMIT_BATCH_SIZE, commitBatchSize); LOG.info("{}={}", CONSUMER_DISABLED, consumerDisabled); } @@ -460,7 +253,7 @@ public int getHandlerOrder() { public void closeImportConsumer(String importId, String topic) { try { LOG.info("==> closeImportConsumer(importId={}, topic={})", importId, topic); -//ATLAS_IMPORT_e22a73f9f6a16620a8655b36d71fb5be + String consumerName = ATLAS_IMPORT_CONSUMER_THREAD_PREFIX + importId; ListIterator consumersIterator = consumers.listIterator(); @@ -472,9 +265,8 @@ public void closeImportConsumer(String importId, String topic) { consumersIterator.remove(); } } - - notificationInterface.closeConsumer(NotificationInterface.NotificationType.ASYNC_IMPORT, topic); - notificationInterface.deleteTopic(NotificationInterface.NotificationType.ASYNC_IMPORT, topic); + notificationInterface.closeConsumer(ASYNC_IMPORT, topic); + notificationInterface.deleteTopic(ASYNC_IMPORT, topic); lastCommittedPartitionOffset.entrySet().removeIf(entry -> topic.equals(entry.getKey().topic())); } catch (Exception e) { @@ -512,9 +304,9 @@ void startInternal(Configuration configuration, ExecutorService executorService) @VisibleForTesting void startHookConsumers() { - int numThreads = applicationProperties.getInt(CONSUMER_THREADS_PROPERTY, 1); + int numThreads = applicationProperties.getInt(CONSUMER_THREADS_PROPERTY, 1); Map, NotificationType> notificationConsumersByType = new HashMap<>(); - List> notificationConsumers = notificationInterface.createConsumers(NotificationType.HOOK, numThreads); + List> notificationConsumers = notificationInterface.createConsumers(NotificationType.HOOK, numThreads); for (NotificationConsumer notificationConsumer : notificationConsumers) { notificationConsumersByType.put(notificationConsumer, NotificationType.HOOK); @@ -604,537 +396,34 @@ private void stopConsumerThreads() { LOG.info("<== stopConsumerThreads()"); } - private List trimAndPurge(String[] values, String defaultValue) { - final List ret; - - if (values != null && values.length > 0) { - ret = new ArrayList<>(values.length); - - for (String val : values) { - if (StringUtils.isNotBlank(val)) { - ret.add(val.trim()); - } - } - } else if (StringUtils.isNotBlank(defaultValue)) { - ret = Collections.singletonList(defaultValue.trim()); - } else { - ret = Collections.emptyList(); - } - - return ret; - } - - private void preprocessEntities(PreprocessorContext context) { - GenericEntityPreprocessor genericEntityPreprocessor = new GenericEntityPreprocessor(this.entityTypesToIgnore, this.entitiesToIgnore); - List entities = context.getEntities(); - - if (entities != null) { - for (int i = 0; i < entities.size(); i++) { - AtlasEntity entity = entities.get(i); - - genericEntityPreprocessor.preprocess(entity, context); - - if (context.isIgnoredEntity(entity.getGuid())) { - entities.remove(i--); - } - } - } - - Map referredEntities = context.getReferredEntities(); - - if (referredEntities != null) { - for (Iterator> iterator = referredEntities.entrySet().iterator(); iterator.hasNext(); ) { - AtlasEntity entity = iterator.next().getValue(); - - genericEntityPreprocessor.preprocess(entity, context); - - if (context.isIgnoredEntity(entity.getGuid())) { - iterator.remove(); - } - } - } - } - - private PreprocessorContext preProcessNotificationMessage(AtlasKafkaMessage kafkaMsg) { - PreprocessorContext context = null; - - if (preprocessEnabled) { - context = new PreprocessorContext(kafkaMsg, typeRegistry, hiveTablesToIgnore, hiveTablesToPrune, hiveTablesCache, - hiveDummyDatabasesToIgnore, hiveDummyTablesToIgnore, hiveTablePrefixesToIgnore, hiveTypesRemoveOwnedRefAttrs, - rdbmsTypesRemoveOwnedRefAttrs, s3V2DirectoryPruneObjectPrefix, updateHiveProcessNameWithQualifiedName, entityCorrelationManager); - - if (CollectionUtils.isNotEmpty(this.entityTypesToIgnore) || CollectionUtils.isNotEmpty(this.entitiesToIgnore)) { - preprocessEntities(context); - } - - if (context.isHivePreprocessEnabled()) { - preprocessHiveTypes(context); - } - - if (skipHiveColumnLineageHive20633) { - skipHiveColumnLineage(context); - } - - if (rdbmsTypesRemoveOwnedRefAttrs) { - rdbmsTypeRemoveOwnedRefAttrs(context); - } - - if (s3V2DirectoryPruneObjectPrefix) { - pruneObjectPrefixForS3V2Directory(context); - } - - if (sparkProcessAttributes) { - preprocessSparkProcessAttributes(context); - } - - context.moveRegisteredReferredEntities(); - - if (context.isHivePreprocessEnabled() && CollectionUtils.isNotEmpty(context.getEntities()) && context.getEntities().size() > 1) { - // move hive_process and hive_column_lineage entities to end of the list - List entities = context.getEntities(); - int count = entities.size(); - - for (int i = 0; i < count; i++) { - AtlasEntity entity = entities.get(i); - - switch (entity.getTypeName()) { - case TYPE_HIVE_PROCESS: - case TYPE_HIVE_COLUMN_LINEAGE: - entities.remove(i--); - entities.add(entity); - count--; - break; - } - } - - if (entities.size() - count > 0) { - LOG.info("preprocess: moved {} hive_process/hive_column_lineage entities to end of list (listSize={}). topic={}, partition={}, offset={}", entities.size() - count, entities.size(), kafkaMsg.getTopic(), kafkaMsg.getPartition(), kafkaMsg.getOffset()); - } - } - } - - return context; - } - - private void rdbmsTypeRemoveOwnedRefAttrs(PreprocessorContext context) { - List entities = context.getEntities(); - - if (entities != null) { - for (int i = 0; i < entities.size(); i++) { - AtlasEntity entity = entities.get(i); - EntityPreprocessor preprocessor = EntityPreprocessor.getRdbmsPreprocessor(entity.getTypeName()); - - if (preprocessor != null) { - preprocessor.preprocess(entity, context); - } - } - } - } - - private void pruneObjectPrefixForS3V2Directory(PreprocessorContext context) { - List entities = new ArrayList<>(); - - if (CollectionUtils.isNotEmpty(context.getEntities())) { - entities.addAll(context.getEntities()); - } - - if (MapUtils.isNotEmpty(context.getReferredEntities())) { - entities.addAll(context.getReferredEntities().values()); - } - - if (CollectionUtils.isNotEmpty(entities)) { - for (AtlasEntity entity : entities) { - EntityPreprocessor preprocessor = EntityPreprocessor.getS3V2Preprocessor(entity.getTypeName()); - - if (preprocessor != null) { - preprocessor.preprocess(entity, context); - } - } - } - } - - private void preprocessHiveTypes(PreprocessorContext context) { - List entities = context.getEntities(); - - if (entities != null) { - for (int i = 0; i < entities.size(); i++) { - AtlasEntity entity = entities.get(i); - EntityPreprocessor preprocessor = EntityPreprocessor.getHivePreprocessor(entity.getTypeName()); - - if (preprocessor != null) { - preprocessor.preprocess(entity, context); - - if (context.isIgnoredEntity(entity.getGuid())) { - entities.remove(i--); - } - } - } - - Map referredEntities = context.getReferredEntities(); - - if (referredEntities != null) { - for (Iterator> iter = referredEntities.entrySet().iterator(); iter.hasNext(); ) { - AtlasEntity entity = iter.next().getValue(); - EntityPreprocessor preprocessor = EntityPreprocessor.getHivePreprocessor(entity.getTypeName()); - - if (preprocessor != null) { - preprocessor.preprocess(entity, context); - - if (context.isIgnoredEntity(entity.getGuid())) { - iter.remove(); - } - } - } - } - - int ignoredEntities = context.getIgnoredEntities().size(); - int prunedEntities = context.getPrunedEntities().size(); - - if (ignoredEntities > 0 || prunedEntities > 0) { - LOG.info("preprocess: ignored entities={}; pruned entities={}. topic-offset={}, partition={}", ignoredEntities, prunedEntities, context.getKafkaMessageOffset(), context.getKafkaPartition()); - } - } - } - - private void preprocessSparkProcessAttributes(PreprocessorContext context) { - List entities = context.getEntities(); - - if (entities != null) { - for (int i = 0; i < entities.size(); i++) { - AtlasEntity entity = entities.get(i); - EntityPreprocessor preprocessor = EntityPreprocessor.getSparkPreprocessor(entity.getTypeName()); - - if (preprocessor != null) { - preprocessor.preprocess(entity, context); - } - } - } - } - - private void skipHiveColumnLineage(PreprocessorContext context) { - List entities = context.getEntities(); - - if (entities != null) { - int lineageCount = 0; - int lineageInputsCount = 0; - int numRemovedEntities = 0; - Set lineageQNames = new HashSet<>(); - - // find if all hive_column_lineage entities have same number of inputs, which is likely to be caused by HIVE-20633 that results in incorrect lineage in some cases - for (int i = 0; i < entities.size(); i++) { - AtlasEntity entity = entities.get(i); - - if (StringUtils.equals(entity.getTypeName(), TYPE_HIVE_COLUMN_LINEAGE)) { - final Object qName = entity.getAttribute(ATTRIBUTE_QUALIFIED_NAME); - - if (qName != null) { - final String qualifiedName = qName.toString(); - - if (lineageQNames.contains(qualifiedName)) { - entities.remove(i--); - - LOG.warn("removed duplicate hive_column_lineage entity: qualifiedName={}. topic-offset={}, partition={}", qualifiedName, context.getKafkaMessageOffset(), context.getKafkaPartition()); - - numRemovedEntities++; - - continue; - } else { - lineageQNames.add(qualifiedName); - } - } - - lineageCount++; - - Object objInputs = entity.getAttribute(ATTRIBUTE_INPUTS); - - if (objInputs instanceof Collection) { - Collection inputs = (Collection) objInputs; - - lineageInputsCount += inputs.size(); - } - } - } - - float avgInputsCount = lineageCount > 0 ? (((float) lineageInputsCount) / lineageCount) : 0; - - if (avgInputsCount > skipHiveColumnLineageHive20633InputsThreshold) { - for (int i = 0; i < entities.size(); i++) { - AtlasEntity entity = entities.get(i); - - if (StringUtils.equals(entity.getTypeName(), TYPE_HIVE_COLUMN_LINEAGE)) { - entities.remove(i--); - - numRemovedEntities++; - } - } - } - - if (numRemovedEntities > 0) { - LOG.warn("removed {} hive_column_lineage entities. Average # of inputs={}, threshold={}, total # of inputs={}. topic-offset={}, partition={}", numRemovedEntities, avgInputsCount, skipHiveColumnLineageHive20633InputsThreshold, lineageInputsCount, context.getKafkaMessageOffset(), context.getKafkaPartition()); - } - } - } - - private boolean isEmptyMessage(AtlasKafkaMessage kafkaMsg) { - final boolean ret; - final HookNotification message = kafkaMsg.getMessage(); - - switch (message.getType()) { - case ENTITY_CREATE_V2: { - AtlasEntitiesWithExtInfo entities = ((EntityCreateRequestV2) message).getEntities(); - - ret = entities == null || CollectionUtils.isEmpty(entities.getEntities()); - } - break; - - case ENTITY_FULL_UPDATE_V2: { - AtlasEntitiesWithExtInfo entities = ((EntityUpdateRequestV2) message).getEntities(); - - ret = entities == null || CollectionUtils.isEmpty(entities.getEntities()); - } - break; - - default: - ret = false; - break; - } - - return ret; - } - - private void recordProcessedEntities(EntityMutationResponse mutationResponse, NotificationStat stats, PreprocessorContext context) { - if (mutationResponse != null) { - if (stats != null) { - stats.updateStats(mutationResponse); - } - - if (context != null) { - if (MapUtils.isNotEmpty(mutationResponse.getGuidAssignments())) { - context.getGuidAssignments().putAll(mutationResponse.getGuidAssignments()); - } - - if (CollectionUtils.isNotEmpty(mutationResponse.getCreatedEntities())) { - for (AtlasEntityHeader entity : mutationResponse.getCreatedEntities()) { - if (entity != null && entity.getGuid() != null) { - context.getCreatedEntities().add(entity.getGuid()); - } - } - } - - if (CollectionUtils.isNotEmpty(mutationResponse.getDeletedEntities())) { - for (AtlasEntityHeader entity : mutationResponse.getDeletedEntities()) { - if (entity != null && entity.getGuid() != null) { - context.getDeletedEntities().add(entity.getGuid()); - } - } - } - } - } - } - - private void updateProcessedEntityReferences(List entities, Map guidAssignments) { - if (CollectionUtils.isNotEmpty(entities) && MapUtils.isNotEmpty(guidAssignments)) { - for (AtlasEntity entity : entities) { - AtlasEntityType entityType = typeRegistry.getEntityTypeByName(entity.getTypeName()); - - if (entityType == null) { - continue; - } - - if (MapUtils.isNotEmpty(entity.getAttributes())) { - for (Map.Entry entry : entity.getAttributes().entrySet()) { - String attrName = entry.getKey(); - Object attrValue = entry.getValue(); - - if (attrValue == null) { - continue; - } - - AtlasAttribute attribute = entityType.getAttribute(attrName); - - if (attribute == null) { // look for a relationship attribute with the same name - attribute = entityType.getRelationshipAttribute(attrName, null); - } - - if (attribute != null && attribute.isObjectRef()) { - updateProcessedEntityReferences(attrValue, guidAssignments); - } - } - } - - if (MapUtils.isNotEmpty(entity.getRelationshipAttributes())) { - for (Map.Entry entry : entity.getRelationshipAttributes().entrySet()) { - Object attrValue = entry.getValue(); - - if (attrValue != null) { - updateProcessedEntityReferences(attrValue, guidAssignments); - } - } - } - } - } - } - - private void updateProcessedEntityReferences(Object objVal, Map guidAssignments) { - if (objVal instanceof AtlasObjectId) { - updateProcessedEntityReferences((AtlasObjectId) objVal, guidAssignments); - } else if (objVal instanceof Collection) { - updateProcessedEntityReferences((Collection) objVal, guidAssignments); - } else if (objVal instanceof Map) { - updateProcessedEntityReferences((Map) objVal, guidAssignments); - } - } - - private void updateProcessedEntityReferences(AtlasObjectId objId, Map guidAssignments) { - String guid = objId.getGuid(); - - if (guid != null && guidAssignments.containsKey(guid)) { - String assignedGuid = guidAssignments.get(guid); - - if (LOG.isDebugEnabled()) { - LOG.debug("{}(guid={}) is already processed; updating its reference to use assigned-guid={}", objId.getTypeName(), guid, assignedGuid); - } - - objId.setGuid(assignedGuid); - objId.setTypeName(null); - objId.setUniqueAttributes(null); - } - } - - private void updateProcessedEntityReferences(Map objId, Map guidAssignments) { - Object guid = objId.get(KEY_GUID); - - if (guid != null && guidAssignments.containsKey(guid)) { - String assignedGuid = guidAssignments.get(guid); - - if (LOG.isDebugEnabled()) { - LOG.debug("{}(guid={}) is already processed; updating its reference to use assigned-guid={}", objId.get(KEY_TYPENAME), guid, assignedGuid); - } - - objId.put(KEY_GUID, assignedGuid); - objId.remove(KEY_TYPENAME); - objId.remove(KEY_UNIQUE_ATTRIBUTES); - } - } - - private void updateProcessedEntityReferences(Collection objIds, Map guidAssignments) { - for (Object objId : objIds) { - updateProcessedEntityReferences(objId, guidAssignments); - } - } - - private void setCurrentUser(String userName) { - Authentication authentication = getAuthenticationForUser(userName); - - if (LOG.isDebugEnabled()) { - if (authentication != null) { - LOG.debug("setCurrentUser(): notification processing will be authorized as user '{}'", userName); - } else { - LOG.debug("setCurrentUser(): Failed to get authentication for user '{}'.", userName); - } - } - - SecurityContextHolder.getContext().setAuthentication(authentication); - } - - private Authentication getAuthenticationForUser(String userName) { - Authentication ret = null; - - if (StringUtils.isNotBlank(userName)) { - ret = authnCache != null ? authnCache.get(userName) : null; - - if (ret == null) { - List grantedAuths = getAuthoritiesFromUGI(userName); - UserDetails principal = new User(userName, "", grantedAuths); - - ret = new UsernamePasswordAuthenticationToken(principal, ""); - - if (authnCache != null) { - authnCache.put(userName, ret); - } - } - } - - return ret; - } - static class Timer { public void sleep(int interval) throws InterruptedException { Thread.sleep(interval); } } - static class AdaptiveWaiter { - private final long increment; - private final long maxDuration; - private final long minDuration; - private final long resetInterval; - - @VisibleForTesting - long waitDuration; - - private long lastWaitAt; - - public AdaptiveWaiter(long minDuration, long maxDuration, long increment) { - this.minDuration = minDuration; - this.maxDuration = maxDuration; - this.increment = increment; - this.waitDuration = minDuration; - this.lastWaitAt = 0; - this.resetInterval = maxDuration * 2; - } - - public void pause(Throwable ex) { - setWaitDurations(); - - try { - if (LOG.isDebugEnabled()) { - LOG.debug("{} in NotificationHookConsumer. Waiting for {} ms for recovery.", ex.getClass().getName(), waitDuration, ex); - } - - Thread.sleep(waitDuration); - } catch (InterruptedException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("{} in NotificationHookConsumer. Waiting for recovery interrupted.", ex.getClass().getName(), e); - } - } - } - - private void setWaitDurations() { - long timeSinceLastWait = (lastWaitAt == 0) ? 0 : System.currentTimeMillis() - lastWaitAt; - - lastWaitAt = System.currentTimeMillis(); - - if (timeSinceLastWait > resetInterval) { - waitDuration = minDuration; - } else { - waitDuration += increment; - if (waitDuration > maxDuration) { - waitDuration = maxDuration; - } - } - } - } - @VisibleForTesting class HookConsumer extends Thread { private final NotificationConsumer consumer; private final AtomicBoolean shouldRun = new AtomicBoolean(false); - private final List failedMessages = new ArrayList<>(); + private final NotificationEntityProcessor entityProcessor; private final AdaptiveWaiter adaptiveWaiter = new AdaptiveWaiter(minWaitDuration, maxWaitDuration, minWaitDuration); private int duplicateKeyCounter = 1; public HookConsumer(NotificationConsumer consumer) { - super(ATLAS_HOOK_CONSUMER_THREAD_NAME); - - this.consumer = consumer; + this(ATLAS_HOOK_CONSUMER_THREAD_NAME, consumer); } public HookConsumer(String consumerThreadName, NotificationConsumer consumer) { super(consumerThreadName); - this.consumer = consumer; + this.consumer = consumer; + this.entityProcessor = AtlasConfiguration.NOTIFICATION_CONCURRENT_PROCESSING.getBoolean() + ? new ConcurrentEntityProcessor(applicationProperties, metricsUtil, authnCache, atlasEntityStore, instanceConverter, entityCorrelationManager, typeRegistry, FAILED_LOG, LARGE_MESSAGES_LOG, asyncImporter) + : new SerialEntityProcessor(applicationProperties, metricsUtil, authnCache, atlasEntityStore, instanceConverter, entityCorrelationManager, typeRegistry, FAILED_LOG, LARGE_MESSAGES_LOG, asyncImporter); + + LOG.info("entityProcessor: {}", entityProcessor.getClass().getSimpleName()); } @Override @@ -1156,7 +445,16 @@ public void run() { sortAndPublishMsgsToAtlasHook(msgBufferingStartTime, msgBuffer); } else { - List> messages = consumer.receiveWithCheckedCommit(lastCommittedPartitionOffset); + List> messages; + TopicPartitionOffsetResult result = entityProcessor.collectResults(); + + commit(result); + + if (StringUtils.contains(this.getName(), ATLAS_IMPORT_CONSUMER_THREAD_PREFIX)) { + messages = consumer.receive(); + } else { + messages = consumer.receiveWithCheckedCommit(lastCommittedPartitionOffset); + } for (AtlasKafkaMessage msg : messages) { handleMessage(msg); @@ -1187,6 +485,7 @@ public void run() { public void shutdown() { LOG.info("==> HookConsumer shutdown()"); + this.entityProcessor.shutdown(); // handle the case where thread was not started at all // and shutdown called @@ -1241,351 +540,11 @@ void sortAndPublishMsgsToAtlasHook(long msgBufferingStartTime, Map kafkaMsg) { - AtlasPerfTracer perf = null; - HookNotification message = kafkaMsg.getMessage(); - String messageUser = message.getUser(); - long startTime = System.currentTimeMillis(); - NotificationStat stats = new NotificationStat(); - AuditLog auditLog = null; - boolean importRequestComplete = false; - - if (authorizeUsingMessageUser) { - setCurrentUser(messageUser); - } - - if (AtlasPerfTracer.isPerfTraceEnabled(PERF_LOG)) { - perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, message.getType().name()); - } - - try { - // covert V1 messages to V2 to enable preProcess - try { - switch (message.getType()) { - case ENTITY_CREATE: { - final EntityCreateRequest createRequest = (EntityCreateRequest) message; - final AtlasEntitiesWithExtInfo entities = instanceConverter.toAtlasEntities(createRequest.getEntities()); - final EntityCreateRequestV2 v2Request = new EntityCreateRequestV2(message.getUser(), entities); - - kafkaMsg = new AtlasKafkaMessage<>(v2Request, kafkaMsg.getOffset(), kafkaMsg.getTopic(), kafkaMsg.getPartition()); - message = kafkaMsg.getMessage(); - } - break; - - case ENTITY_FULL_UPDATE: { - final EntityUpdateRequest updateRequest = (EntityUpdateRequest) message; - final AtlasEntitiesWithExtInfo entities = instanceConverter.toAtlasEntities(updateRequest.getEntities()); - final EntityUpdateRequestV2 v2Request = new EntityUpdateRequestV2(messageUser, entities); - - kafkaMsg = new AtlasKafkaMessage<>(v2Request, kafkaMsg.getOffset(), kafkaMsg.getTopic(), kafkaMsg.getPartition()); - message = kafkaMsg.getMessage(); - } - break; - } - } catch (AtlasBaseException excp) { - LOG.error("handleMessage({}): failed to convert V1 message to V2", message.getType().name()); - } - - PreprocessorContext context = preProcessNotificationMessage(kafkaMsg); - - if (isEmptyMessage(kafkaMsg)) { - commit(kafkaMsg); - - return; - } - - // Used for intermediate conversions during create and update - String exceptionClassName = StringUtils.EMPTY; - for (int numRetries = 0; numRetries < maxRetries; numRetries++) { - LOG.debug("handleMessage({}): attempt {}", message.getType().name(), numRetries); - - try { - RequestContext requestContext = RequestContext.get(); - - requestContext.setAttemptCount(numRetries + 1); - requestContext.setMaxAttempts(maxRetries); - - requestContext.setUser(messageUser, null); - requestContext.setInNotificationProcessing(true); - requestContext.setCreateShellEntityForNonExistingReference(createShellEntityForNonExistingReference); - - switch (message.getType()) { - case ENTITY_CREATE: { - final EntityCreateRequest createRequest = (EntityCreateRequest) message; - final AtlasEntitiesWithExtInfo entities = instanceConverter.toAtlasEntities(createRequest.getEntities()); - - if (auditLog == null) { - auditLog = new AuditLog(messageUser, THREADNAME_PREFIX, AtlasClient.API_V1.CREATE_ENTITY.getMethod(), AtlasClient.API_V1.CREATE_ENTITY.getNormalizedPath()); - } - - createOrUpdate(entities, false, stats, context); - } - break; - - case ENTITY_PARTIAL_UPDATE: { - final EntityPartialUpdateRequest partialUpdateRequest = (EntityPartialUpdateRequest) message; - final Referenceable referenceable = partialUpdateRequest.getEntity(); - final AtlasEntitiesWithExtInfo entities = instanceConverter.toAtlasEntity(referenceable); - - if (auditLog == null) { - auditLog = new AuditLog(messageUser, THREADNAME_PREFIX, - AtlasClientV2.API_V2.UPDATE_ENTITY_BY_ATTRIBUTE.getMethod(), - String.format(AtlasClientV2.API_V2.UPDATE_ENTITY_BY_ATTRIBUTE.getNormalizedPath(), partialUpdateRequest.getTypeName())); - } - - AtlasEntityType entityType = typeRegistry.getEntityTypeByName(partialUpdateRequest.getTypeName()); - String guid = AtlasGraphUtilsV2.getGuidByUniqueAttributes(entityType, Collections.singletonMap(partialUpdateRequest.getAttribute(), partialUpdateRequest.getAttributeValue())); - - // There should only be one root entity - entities.getEntities().get(0).setGuid(guid); - - createOrUpdate(entities, true, stats, context); - } - break; - - case ENTITY_DELETE: { - final EntityDeleteRequest deleteRequest = (EntityDeleteRequest) message; + void handleMessage(AtlasKafkaMessage msg) { + LOG.info("Message type: {}", msg.getMessage().getType().name()); - if (auditLog == null) { - auditLog = new AuditLog(messageUser, THREADNAME_PREFIX, - AtlasClientV2.API_V2.DELETE_ENTITY_BY_ATTRIBUTE.getMethod(), - String.format(AtlasClientV2.API_V2.DELETE_ENTITY_BY_ATTRIBUTE.getNormalizedPath(), deleteRequest.getTypeName())); - } - - try { - AtlasEntityType type = (AtlasEntityType) typeRegistry.getType(deleteRequest.getTypeName()); - - EntityMutationResponse response = atlasEntityStore.deleteByUniqueAttributes(type, Collections.singletonMap(deleteRequest.getAttribute(), deleteRequest.getAttributeValue())); - - stats.updateStats(response); - - entityCorrelationManager.add(kafkaMsg.getSpooled(), kafkaMsg.getMsgCreated(), response.getDeletedEntities()); - } catch (ClassCastException cle) { - LOG.error("Failed to delete entity {}", deleteRequest); - } - } - break; - - case ENTITY_FULL_UPDATE: { - final EntityUpdateRequest updateRequest = (EntityUpdateRequest) message; - final AtlasEntitiesWithExtInfo entities = instanceConverter.toAtlasEntities(updateRequest.getEntities()); - - if (auditLog == null) { - auditLog = new AuditLog(messageUser, THREADNAME_PREFIX, AtlasClientV2.API_V2.UPDATE_ENTITY.getMethod(), AtlasClientV2.API_V2.UPDATE_ENTITY.getNormalizedPath()); - } - - createOrUpdate(entities, false, stats, context); - } - break; - - case ENTITY_CREATE_V2: { - final EntityCreateRequestV2 createRequestV2 = (EntityCreateRequestV2) message; - final AtlasEntitiesWithExtInfo entities = createRequestV2.getEntities(); - - if (auditLog == null) { - auditLog = new AuditLog(messageUser, THREADNAME_PREFIX, AtlasClientV2.API_V2.CREATE_ENTITY.getMethod(), AtlasClientV2.API_V2.CREATE_ENTITY.getNormalizedPath()); - } - - createOrUpdate(entities, false, stats, context); - } - break; - - case ENTITY_PARTIAL_UPDATE_V2: { - final EntityPartialUpdateRequestV2 partialUpdateRequest = (EntityPartialUpdateRequestV2) message; - final AtlasObjectId entityId = partialUpdateRequest.getEntityId(); - final AtlasEntityWithExtInfo entity = partialUpdateRequest.getEntity(); - - if (auditLog == null) { - auditLog = new AuditLog(messageUser, THREADNAME_PREFIX, AtlasClientV2.API_V2.UPDATE_ENTITY.getMethod(), AtlasClientV2.API_V2.UPDATE_ENTITY.getNormalizedPath()); - } - - EntityMutationResponse response = atlasEntityStore.updateEntity(entityId, entity, true); - - stats.updateStats(response); - } - break; - - case ENTITY_FULL_UPDATE_V2: { - final EntityUpdateRequestV2 updateRequest = (EntityUpdateRequestV2) message; - final AtlasEntitiesWithExtInfo entities = updateRequest.getEntities(); - - if (auditLog == null) { - auditLog = new AuditLog(messageUser, THREADNAME_PREFIX, AtlasClientV2.API_V2.UPDATE_ENTITY.getMethod(), AtlasClientV2.API_V2.UPDATE_ENTITY.getNormalizedPath()); - } - - createOrUpdate(entities, false, stats, context); - } - break; - - case ENTITY_DELETE_V2: { - final EntityDeleteRequestV2 deleteRequest = (EntityDeleteRequestV2) message; - final List entities = deleteRequest.getEntities(); - - try { - for (AtlasObjectId entity : entities) { - if (auditLog == null) { - auditLog = new AuditLog(messageUser, THREADNAME_PREFIX, - AtlasClientV2.API_V2.DELETE_ENTITY_BY_ATTRIBUTE.getMethod(), - String.format(AtlasClientV2.API_V2.DELETE_ENTITY_BY_ATTRIBUTE.getNormalizedPath(), entity.getTypeName())); - } - - AtlasEntityType type = (AtlasEntityType) typeRegistry.getType(entity.getTypeName()); - - EntityMutationResponse response = atlasEntityStore.deleteByUniqueAttributes(type, entity.getUniqueAttributes()); - - stats.updateStats(response); - - entityCorrelationManager.add(kafkaMsg.getSpooled(), kafkaMsg.getMsgCreated(), response.getDeletedEntities()); - } - } catch (ClassCastException cle) { - LOG.error("Failed to do delete entities {}", entities); - } - } - break; - - case IMPORT_TYPES_DEF: { - final AtlasTypesDefImportNotification typesDefImportNotification = (AtlasTypesDefImportNotification) message; - final String importId = typesDefImportNotification.getImportId(); - final AtlasTypesDef typesDef = typesDefImportNotification.getTypesDef(); - - try { - asyncImporter.onImportTypeDef(typesDef, importId); - } catch (AtlasBaseException abe) { - LOG.error("IMPORT_TYPE_DEF: {} failed to import type definition: {}", importId, typesDef); - asyncImporter.onImportComplete(importId); - importRequestComplete = true; - } - } - break; - - case IMPORT_ENTITY: { - final AtlasEntityImportNotification entityImportNotification = (AtlasEntityImportNotification) message; - final String importId = entityImportNotification.getImportId(); - final AtlasEntityWithExtInfo entityWithExtInfo = entityImportNotification.getEntity(); - final int position = entityImportNotification.getPosition(); - - LOG.info("==> IMPORT_ENTITY:processing entity: {} at position: {}", importId, position); - - try { - importRequestComplete = asyncImporter.onImportEntity(entityWithExtInfo, importId, position); - } catch (AtlasBaseException abe) { - importRequestComplete = true; - - LOG.error("IMPORT_ENTITY: {} failed to import entity: {}", importId, entityImportNotification); - } finally { - if (importRequestComplete) { - asyncImporter.onImportComplete(importId); - } - LOG.info("<== IMPORT_ENTITY:processing entity: {} at position: {}", importId, position); - } - } - break; - - default: - throw new IllegalStateException("Unknown notification type: " + message.getType().name()); - } - - if (StringUtils.isNotEmpty(exceptionClassName)) { - LOG.warn("{}: Pausing & retry: Try: {}: Pause: {} ms. Handled!", exceptionClassName, numRetries, adaptiveWaiter.waitDuration); - - exceptionClassName = StringUtils.EMPTY; - } - break; - } catch (Throwable e) { - 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())); - - if (maxRetriesReached || unrecoverableFailure) { - try { - 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); - } - - stats.isFailedMsg = true; - - failedMessages.add(strMessage); - - if (failedMessages.size() >= failedMsgCacheSize) { - recordFailedMessages(); - } - } catch (Throwable t) { - LOG.warn("error while recording failed message: type={}, topic={}, partition={}, offset={}", message.getType(), kafkaMsg.getTopic(), kafkaMsg.getPartition(), kafkaMsg.getOffset(), t); - } - - return; - } else if (e instanceof org.apache.atlas.repository.graphdb.AtlasSchemaViolationException) { - LOG.warn("{}: Continuing: {}", exceptionClassName, e.getMessage()); - } else if (exceptionClassName.equals(EXCEPTION_CLASS_NAME_JANUSGRAPH_EXCEPTION) || exceptionClassName.equals(EXCEPTION_CLASS_NAME_PERMANENTLOCKING_EXCEPTION)) { - LOG.warn("{}: Pausing & retry: Try: {}: Pause: {} ms. {}", exceptionClassName, numRetries, adaptiveWaiter.waitDuration, e.getMessage()); - - adaptiveWaiter.pause(e); - } else { - LOG.warn("Error handling message", e); - - try { - LOG.info("Sleeping for {} ms before retry", consumerRetryInterval); - - Thread.sleep(consumerRetryInterval); - } catch (InterruptedException ie) { - LOG.error("Notification consumer thread sleep interrupted"); - } - } - } finally { - RequestContext.clear(); - } - } - - commit(kafkaMsg); - } finally { - AtlasPerfTracer.log(perf); - - stats.timeTakenMs = System.currentTimeMillis() - startTime; - - metricsUtil.onNotificationProcessingComplete(kafkaMsg.getTopic(), kafkaMsg.getPartition(), kafkaMsg.getOffset(), stats); - - if (stats.timeTakenMs > largeMessageProcessingTimeThresholdMs) { - try { - String strMessage = AbstractNotification.getMessageJson(message); - - LOG.warn("msgProcessingTime={}, msgSize={}, topic={}, partition={}, offset={}}", stats.timeTakenMs, strMessage.length(), kafkaMsg.getTopic(), kafkaMsg.getPartition(), kafkaMsg.getOffset()); - - LARGE_MESSAGES_LOG.warn("{\"msgProcessingTime\":{},\"msgSize\":{},\"topic\":{},\"partition\":{},\"topicOffset\":{},\"data\":{}}", stats.timeTakenMs, strMessage.length(), kafkaMsg.getTopic(), kafkaMsg.getPartition(), kafkaMsg.getOffset(), strMessage); - } catch (Throwable t) { - LOG.warn("error while recording large message: msgProcessingTime={}, type={}, topic={}, partition={}, offset={}", stats.timeTakenMs, message.getType(), kafkaMsg.getTopic(), kafkaMsg.getPartition(), kafkaMsg.getOffset(), t); - } - } - - if (auditLog != null) { - auditLog.setHttpStatus(stats.isFailedMsg ? SC_BAD_REQUEST : SC_OK); - auditLog.setTimeTaken(stats.timeTakenMs); - - AuditFilter.audit(auditLog); - } - - Instant now = Instant.now(); - - if (now.isAfter(nextStatsLogTime)) { - LOG.info("STATS: {}", AtlasJson.toJson(metricsUtil.getStats())); - - nextStatsLogTime = AtlasMetricsCounter.getNextHourStartTime(now); - } - - if (importRequestComplete) { - asyncImporter.onCompleteImportRequest(((AtlasEntityImportNotification) message).getImportId()); - } - } + TopicPartitionOffsetResult result = entityProcessor.handleMessage(msg); + commit(result); } boolean serverAvailable(Timer timer) { @@ -1632,68 +591,33 @@ private void sortMessages(AtlasKafkaMessage msg, Map entitiesList = entities.getEntities(); - AtlasEntityStream entityStream = new AtlasEntityStream(entities); - - if (commitBatchSize <= 0 || entitiesList.size() <= commitBatchSize) { - EntityMutationResponse response = atlasEntityStore.createOrUpdate(entityStream, isPartialUpdate); - - recordProcessedEntities(response, stats, context); - } else { - for (int fromIdx = 0; fromIdx < entitiesList.size(); fromIdx += commitBatchSize) { - int toIndex = fromIdx + commitBatchSize; - - if (toIndex > entitiesList.size()) { - toIndex = entitiesList.size(); - } - - List entitiesBatch = new ArrayList<>(entitiesList.subList(fromIdx, toIndex)); - - updateProcessedEntityReferences(entitiesBatch, context.getGuidAssignments()); - - AtlasEntitiesWithExtInfo batch = new AtlasEntitiesWithExtInfo(entitiesBatch); - AtlasEntityStream batchStream = new AtlasEntityStream(batch, entityStream); - EntityMutationResponse response = atlasEntityStore.createOrUpdate(batchStream, isPartialUpdate); - - recordProcessedEntities(response, stats, context); - - RequestContext.get().resetEntityGuidUpdates(); - - entityCorrelationManager.add(context.isSpooledMessage(), context.getMsgCreated(), response.getDeletedEntities()); - - RequestContext.get().clearCache(); - } - } - - if (context != null) { - context.prepareForPostUpdate(); - - List postUpdateEntities = context.getPostUpdateEntities(); - - if (CollectionUtils.isNotEmpty(postUpdateEntities)) { - atlasEntityStore.createOrUpdate(new AtlasEntityStream(postUpdateEntities), true); + private void commit(TopicPartitionOffsetResult result) { + if (result != null) { + if (!lastCommittedPartitionOffset.containsKey(result.getTopicPartition()) + || (lastCommittedPartitionOffset.containsKey(result.getTopicPartition()) + && lastCommittedPartitionOffset.get(result.getTopicPartition()) != null + && lastCommittedPartitionOffset.get(result.getTopicPartition()) != result.getOffset())) { + LOG.info("Committing the result: {}", result.getKey()); + commit(result.getTopicPartition(), result.getOffset()); + } else { + consumer.poll(); } } } - private void recordFailedMessages() { - //logging failed messages - for (String message : failedMessages) { - FAILED_LOG.error("[DROPPED_NOTIFICATION] {}", message); - } - - failedMessages.clear(); - } - private void commit(AtlasKafkaMessage kafkaMessage) { - recordFailedMessages(); - long commitOffset = kafkaMessage.getOffset() + 1; - lastCommittedPartitionOffset.put(kafkaMessage.getTopicPartition(), commitOffset); - consumer.commit(kafkaMessage.getTopicPartition(), commitOffset); } + + private void commit(TopicPartition topicPartition, long commitOffset) { + long offsetToCommit = commitOffset + 1; + consumer.commit(topicPartition, offsetToCommit); + lastCommittedPartitionOffset.put(topicPartition, commitOffset); + if (LOG.isDebugEnabled()) { + LOG.debug("Committing: topicPartition: {} with offset: {}", topicPartition, offsetToCommit); + } + } } } diff --git a/webapp/src/main/java/org/apache/atlas/notification/SerialEntityProcessor.java b/webapp/src/main/java/org/apache/atlas/notification/SerialEntityProcessor.java new file mode 100644 index 00000000000..fd39249d5cf --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/SerialEntityProcessor.java @@ -0,0 +1,1206 @@ +/** + * 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.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; +import org.apache.atlas.model.instance.AtlasEntity; +import org.apache.atlas.model.instance.AtlasEntity.AtlasEntitiesWithExtInfo; +import org.apache.atlas.model.instance.AtlasEntity.AtlasEntityWithExtInfo; +import org.apache.atlas.model.instance.AtlasEntityHeader; +import org.apache.atlas.model.instance.AtlasObjectId; +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.HookNotification.EntityDeleteRequestV2; +import org.apache.atlas.model.notification.HookNotification.EntityPartialUpdateRequestV2; +import org.apache.atlas.model.notification.HookNotification.EntityUpdateRequestV2; +import org.apache.atlas.model.notification.ImportNotification.AtlasEntityImportNotification; +import org.apache.atlas.model.notification.ImportNotification.AtlasTypesDefImportNotification; +import org.apache.atlas.model.typedef.AtlasTypesDef; +import org.apache.atlas.notification.pc.Ticket; +import org.apache.atlas.notification.preprocessor.EntityPreprocessor; +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.impexp.AsyncImporter; +import org.apache.atlas.repository.store.graph.AtlasEntityStore; +import org.apache.atlas.repository.store.graph.v2.AtlasEntityStream; +import org.apache.atlas.repository.store.graph.v2.AtlasGraphUtilsV2; +import org.apache.atlas.type.AtlasEntityType; +import org.apache.atlas.type.AtlasStructType.AtlasAttribute; +import org.apache.atlas.type.AtlasTypeRegistry; +import org.apache.atlas.util.AdaptiveWaiter; +import org.apache.atlas.util.AtlasMetricsCounter; +import org.apache.atlas.util.AtlasMetricsUtil; +import org.apache.atlas.util.AtlasMetricsUtil.NotificationStat; +import org.apache.atlas.utils.AtlasJson; +import org.apache.atlas.utils.AtlasPerfTracer; +import org.apache.atlas.utils.LruCache; +import org.apache.atlas.v1.model.instance.Referenceable; +import org.apache.atlas.v1.model.notification.HookNotificationV1; +import org.apache.atlas.web.filters.AuditFilter; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.collections.MapUtils; +import org.apache.commons.configuration2.Configuration; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.authentication.UsernamePasswordAuthenticationToken; +import org.springframework.security.core.Authentication; +import org.springframework.security.core.GrantedAuthority; +import org.springframework.security.core.context.SecurityContextHolder; +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; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +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; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_AUTHORIZE_USING_MESSAGE_USER; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_COMMIT_BATCH_SIZE; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_FAILEDCACHESIZE_PROPERTY; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_MAX_RETRY_INTERVAL; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_MIN_RETRY_INTERVAL; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_ENTITY_IGNORE_PATTERN; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_ENTITY_TYPE_IGNORE_PATTERN; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_HIVE_DB_IGNORE_DUMMY_ENABLED; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_HIVE_DB_IGNORE_DUMMY_NAMES; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_HIVE_PROCESS_UPD_NAME_WITH_QUALIFIED_NAME; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_HIVE_TABLE_CACHE_SIZE; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_DUMMY_ENABLED; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_DUMMY_NAMES; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_NAME_PREFIXES; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_NAME_PREFIXES_ENABLED; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_PATTERN; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_HIVE_TABLE_PRUNE_PATTERN; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_HIVE_TYPES_REMOVE_OWNEDREF_ATTRS; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_RDBMS_TYPES_REMOVE_OWNEDREF_ATTRS; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_S3_V2_DIRECTORY_PRUNE_OBJECT_PREFIX; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_PREPROCESS_SPARK_PROCESS_ATTRIBUTES; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_RETRIES_PROPERTY; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_RETRY_INTERVAL; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_SKIP_HIVE_COLUMN_LINEAGE_HIVE_20633; +import static org.apache.atlas.notification.NotificationHookConsumer.CONSUMER_SKIP_HIVE_COLUMN_LINEAGE_HIVE_20633_INPUTS_THRESHOLD; +import static org.apache.atlas.notification.NotificationHookConsumer.DUMMY_DATABASE; +import static org.apache.atlas.notification.NotificationHookConsumer.DUMMY_TABLE; +import static org.apache.atlas.notification.NotificationHookConsumer.VALUES_TMP_TABLE_NAME_PREFIX; +import static org.apache.atlas.notification.preprocessor.EntityPreprocessor.TYPE_HIVE_PROCESS; +import static org.apache.atlas.web.security.AtlasAbstractAuthenticationProvider.getAuthoritiesFromUGI; + +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 int SC_OK = 200; + private static final int SC_BAD_REQUEST = 400; + private static final String TYPE_HIVE_COLUMN_LINEAGE = "hive_column_lineage"; + private static final String ATTRIBUTE_INPUTS = "inputs"; + private static final String ATTRIBUTE_QUALIFIED_NAME = "qualifiedName"; + + private final int commitBatchSize; + private final AtlasEntityStore atlasEntityStore; + private final EntityCorrelationManager entityCorrelationManager; + private final AtlasTypeRegistry typeRegistry; + private final Configuration applicationProperties; + private final int minWaitDuration; + private final int maxWaitDuration; + private final int consumerRetryInterval; + private final AdaptiveWaiter adaptiveWaiter; + private final boolean createShellEntityForNonExistingReference; + private final boolean authorizeUsingMessageUser; + private final int largeMessageProcessingTimeThresholdMs; + private final List failedMessages; + private final int failedMsgCacheSize; + private final boolean s3V2DirectoryPruneObjectPrefix; + private final int skipHiveColumnLineageHive20633InputsThreshold; + private final boolean skipHiveColumnLineageHive20633; + private final boolean updateHiveProcessNameWithQualifiedName; + private final boolean hiveTypesRemoveOwnedRefAttrs; + private final boolean rdbmsTypesRemoveOwnedRefAttrs; + private final boolean sparkProcessAttributes; + private final List entityTypesToIgnore = new ArrayList<>(); + private final List entitiesToIgnore = new ArrayList<>(); + private final List hiveTablesToIgnore = new ArrayList<>(); + private final List hiveTablesToPrune = new ArrayList<>(); + private final List hiveDummyDatabasesToIgnore; + private final List hiveDummyTablesToIgnore; + private final List hiveTablePrefixesToIgnore; + private final Map hiveTablesCache; + private final AsyncImporter asyncImporter; + private int maxRetries = 3; + private Map authnCache; + private AtlasInstanceConverter instanceConverter; + private AtlasMetricsUtil metricsUtil; + private Logger failedMessageLog; + private Logger largeMessagesLog; + private Instant nextStatsLogTime = AtlasMetricsCounter.getNextHourStartTime(Instant.now()); + private boolean preprocessEnabled; + + public SerialEntityProcessor(Configuration applicationProperties, AtlasMetricsUtil metricsUtil, Map authnCache, + AtlasEntityStore atlasEntityStore, AtlasInstanceConverter instanceConverter, EntityCorrelationManager entityCorrelationManager, + AtlasTypeRegistry typeRegistry, Logger failedMessageLogger, Logger largeMessagesLogger, AsyncImporter asyncImporter) { + this.failedMessageLog = failedMessageLogger; + this.largeMessagesLog = largeMessagesLogger; + this.metricsUtil = metricsUtil; + this.failedMessages = new ArrayList<>(); + this.authnCache = authnCache; + this.instanceConverter = instanceConverter; + this.applicationProperties = applicationProperties; + + this.atlasEntityStore = atlasEntityStore; + this.entityCorrelationManager = entityCorrelationManager; + this.typeRegistry = typeRegistry; + this.failedMsgCacheSize = this.applicationProperties.getInt(CONSUMER_FAILEDCACHESIZE_PROPERTY, 1); + + this.maxRetries = applicationProperties.getInt(CONSUMER_RETRIES_PROPERTY, 3); + this.consumerRetryInterval = applicationProperties.getInt(CONSUMER_RETRY_INTERVAL, 500); + this.largeMessageProcessingTimeThresholdMs = this.applicationProperties.getInt("atlas.notification.consumer.large.message.processing.time.threshold.ms", 60 * 1000); // 60 sec by default + this.minWaitDuration = this.applicationProperties.getInt(CONSUMER_MIN_RETRY_INTERVAL, consumerRetryInterval); // 500 ms by default + this.maxWaitDuration = this.applicationProperties.getInt(CONSUMER_MAX_RETRY_INTERVAL, minWaitDuration * 60); // 30 sec by default + this.commitBatchSize = this.applicationProperties.getInt(CONSUMER_COMMIT_BATCH_SIZE, 50); + this.authorizeUsingMessageUser = this.applicationProperties.getBoolean(CONSUMER_AUTHORIZE_USING_MESSAGE_USER, false); + this.skipHiveColumnLineageHive20633InputsThreshold = this.applicationProperties.getInt(CONSUMER_SKIP_HIVE_COLUMN_LINEAGE_HIVE_20633_INPUTS_THRESHOLD, 15); // skip if avg # of inputs is > 15 + this.skipHiveColumnLineageHive20633 = this.applicationProperties.getBoolean(CONSUMER_SKIP_HIVE_COLUMN_LINEAGE_HIVE_20633, false); + this.updateHiveProcessNameWithQualifiedName = this.applicationProperties.getBoolean(CONSUMER_PREPROCESS_HIVE_PROCESS_UPD_NAME_WITH_QUALIFIED_NAME, true); + this.hiveTypesRemoveOwnedRefAttrs = this.applicationProperties.getBoolean(CONSUMER_PREPROCESS_HIVE_TYPES_REMOVE_OWNEDREF_ATTRS, true); + this.rdbmsTypesRemoveOwnedRefAttrs = this.applicationProperties.getBoolean(CONSUMER_PREPROCESS_RDBMS_TYPES_REMOVE_OWNEDREF_ATTRS, true); + this.s3V2DirectoryPruneObjectPrefix = applicationProperties.getBoolean(CONSUMER_PREPROCESS_S3_V2_DIRECTORY_PRUNE_OBJECT_PREFIX, true); + this.sparkProcessAttributes = this.applicationProperties.getBoolean(CONSUMER_PREPROCESS_SPARK_PROCESS_ATTRIBUTES, false); + this.adaptiveWaiter = new AdaptiveWaiter(minWaitDuration, maxWaitDuration, minWaitDuration); + this.createShellEntityForNonExistingReference = AtlasConfiguration.NOTIFICATION_CREATE_SHELL_ENTITY_FOR_NON_EXISTING_REF.getBoolean(); + this.asyncImporter = asyncImporter; + + String[] patternEntityTypesToIgnore = applicationProperties.getStringArray(CONSUMER_PREPROCESS_ENTITY_TYPE_IGNORE_PATTERN); + String[] patternEntitiesToIgnore = applicationProperties.getStringArray(CONSUMER_PREPROCESS_ENTITY_IGNORE_PATTERN); + + String[] patternHiveTablesToIgnore = applicationProperties.getStringArray(CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_PATTERN); + String[] patternHiveTablesToPrune = applicationProperties.getStringArray(CONSUMER_PREPROCESS_HIVE_TABLE_PRUNE_PATTERN); + + if (patternEntityTypesToIgnore != null) { + for (String pattern : patternEntityTypesToIgnore) { + try { + this.entityTypesToIgnore.add(Pattern.compile(pattern)); + } catch (Throwable t) { + LOG.warn("failed to compile pattern {}", pattern, t); + LOG.warn("Ignoring invalid pattern in configuration {}: {}", CONSUMER_PREPROCESS_ENTITY_TYPE_IGNORE_PATTERN, pattern); + } + } + LOG.info("{}={}", CONSUMER_PREPROCESS_ENTITY_TYPE_IGNORE_PATTERN, entityTypesToIgnore); + } + + if (patternEntitiesToIgnore != null) { + for (String pattern : patternEntitiesToIgnore) { + try { + this.entitiesToIgnore.add(Pattern.compile(pattern)); + } catch (Throwable t) { + LOG.warn("failed to compile pattern {}", pattern, t); + LOG.warn("Ignoring invalid pattern in configuration {}: {}", CONSUMER_PREPROCESS_ENTITY_IGNORE_PATTERN, pattern); + } + } + LOG.info("{}={}", CONSUMER_PREPROCESS_ENTITY_IGNORE_PATTERN, entitiesToIgnore); + } + + if (patternHiveTablesToIgnore != null) { + for (String pattern : patternHiveTablesToIgnore) { + try { + hiveTablesToIgnore.add(Pattern.compile(pattern)); + + LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_PATTERN, pattern); + } catch (Throwable t) { + LOG.warn("failed to compile pattern {}", pattern, t); + LOG.warn("Ignoring invalid pattern in configuration {}: {}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_PATTERN, pattern); + } + } + } + + if (patternHiveTablesToPrune != null) { + for (String pattern : patternHiveTablesToPrune) { + try { + hiveTablesToPrune.add(Pattern.compile(pattern)); + + LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_PRUNE_PATTERN, pattern); + } catch (Throwable t) { + LOG.warn("failed to compile pattern {}", pattern, t); + LOG.warn("Ignoring invalid pattern in configuration {}: {}", CONSUMER_PREPROCESS_HIVE_TABLE_PRUNE_PATTERN, pattern); + } + } + } + + if (!hiveTablesToIgnore.isEmpty() || !hiveTablesToPrune.isEmpty()) { + hiveTablesCache = new LruCache<>(this.applicationProperties.getInt(CONSUMER_PREPROCESS_HIVE_TABLE_CACHE_SIZE, 10000), 0); + } else { + hiveTablesCache = Collections.emptyMap(); + } + + boolean hiveDbIgnoreDummyEnabled = applicationProperties.getBoolean(CONSUMER_PREPROCESS_HIVE_DB_IGNORE_DUMMY_ENABLED, true); + boolean hiveTableIgnoreDummyEnabled = applicationProperties.getBoolean(CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_DUMMY_ENABLED, true); + boolean hiveTableIgnoreNamePrefixEnabled = applicationProperties.getBoolean(CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_NAME_PREFIXES_ENABLED, true); + + LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_DB_IGNORE_DUMMY_ENABLED, hiveDbIgnoreDummyEnabled); + LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_DUMMY_ENABLED, hiveTableIgnoreDummyEnabled); + LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_NAME_PREFIXES_ENABLED, hiveTableIgnoreNamePrefixEnabled); + + if (hiveDbIgnoreDummyEnabled) { + String[] dummyDatabaseNames = applicationProperties.getStringArray(CONSUMER_PREPROCESS_HIVE_DB_IGNORE_DUMMY_NAMES); + + hiveDummyDatabasesToIgnore = trimAndPurge(dummyDatabaseNames, DUMMY_DATABASE); + + LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_DB_IGNORE_DUMMY_NAMES, StringUtils.join(hiveDummyDatabasesToIgnore, ',')); + } else { + hiveDummyDatabasesToIgnore = Collections.emptyList(); + } + + if (hiveTableIgnoreDummyEnabled) { + String[] dummyTableNames = applicationProperties.getStringArray(CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_DUMMY_NAMES); + + hiveDummyTablesToIgnore = trimAndPurge(dummyTableNames, DUMMY_TABLE); + + LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_DUMMY_NAMES, StringUtils.join(hiveDummyTablesToIgnore, ',')); + } else { + hiveDummyTablesToIgnore = Collections.emptyList(); + } + + if (hiveTableIgnoreNamePrefixEnabled) { + String[] ignoreNamePrefixes = applicationProperties.getStringArray(CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_NAME_PREFIXES); + + hiveTablePrefixesToIgnore = trimAndPurge(ignoreNamePrefixes, VALUES_TMP_TABLE_NAME_PREFIX); + + LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TABLE_IGNORE_NAME_PREFIXES, StringUtils.join(hiveTablePrefixesToIgnore, ',')); + } else { + hiveTablePrefixesToIgnore = Collections.emptyList(); + } + + LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_PROCESS_UPD_NAME_WITH_QUALIFIED_NAME, updateHiveProcessNameWithQualifiedName); + + preprocessEnabled = skipHiveColumnLineageHive20633 || updateHiveProcessNameWithQualifiedName || hiveTypesRemoveOwnedRefAttrs || rdbmsTypesRemoveOwnedRefAttrs || s3V2DirectoryPruneObjectPrefix || !hiveTablesToIgnore.isEmpty() || !hiveTablesToPrune.isEmpty() || !hiveDummyDatabasesToIgnore.isEmpty() || !hiveDummyTablesToIgnore.isEmpty() || !hiveTablePrefixesToIgnore.isEmpty() || sparkProcessAttributes; + + LOG.info("{}={}", CONSUMER_SKIP_HIVE_COLUMN_LINEAGE_HIVE_20633, skipHiveColumnLineageHive20633); + LOG.info("{}={}", CONSUMER_SKIP_HIVE_COLUMN_LINEAGE_HIVE_20633_INPUTS_THRESHOLD, skipHiveColumnLineageHive20633InputsThreshold); + LOG.info("{}={}", CONSUMER_PREPROCESS_HIVE_TYPES_REMOVE_OWNEDREF_ATTRS, hiveTypesRemoveOwnedRefAttrs); + LOG.info("{}={}", CONSUMER_PREPROCESS_RDBMS_TYPES_REMOVE_OWNEDREF_ATTRS, rdbmsTypesRemoveOwnedRefAttrs); + LOG.info("{}={}", CONSUMER_PREPROCESS_S3_V2_DIRECTORY_PRUNE_OBJECT_PREFIX, s3V2DirectoryPruneObjectPrefix); + LOG.info("{}={}", CONSUMER_COMMIT_BATCH_SIZE, commitBatchSize); + } + + public TopicPartitionOffsetResult handleMessage(AtlasKafkaMessage kafkaMsg) { + return handleMessage(new Ticket(kafkaMsg)); + } + + @Override + public TopicPartitionOffsetResult collectResults() { + return null; + } + + @Override + public void shutdown() { + recordFailedMessages(failedMessages); + } + + public TopicPartitionOffsetResult handleMessage(Ticket ticket) { + AtlasPerfTracer perf = null; + AtlasKafkaMessage kafkaMsg = ticket.getMessage(); + HookNotification message = kafkaMsg.getMessage(); + String messageUser = message.getUser(); + long startTime = System.currentTimeMillis(); + AtlasMetricsUtil.NotificationStat stats = new AtlasMetricsUtil.NotificationStat(); + AuditFilter.AuditLog auditLog = null; + boolean importRequestComplete = false; + + if (authorizeUsingMessageUser) { + setCurrentUser(messageUser); + } + + if (AtlasPerfTracer.isPerfTraceEnabled(PERF_LOG)) { + perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, message.getType().name()); + } + + try { + // covert V1 messages to V2 to enable preProcess + try { + switch (message.getType()) { + case ENTITY_CREATE: { + final HookNotificationV1.EntityCreateRequest createRequest = (HookNotificationV1.EntityCreateRequest) message; + final AtlasEntity.AtlasEntitiesWithExtInfo entities = instanceConverter.toAtlasEntities(createRequest.getEntities()); + final EntityCreateRequestV2 v2Request = new EntityCreateRequestV2(message.getUser(), entities); + + kafkaMsg = new AtlasKafkaMessage<>(v2Request, kafkaMsg.getOffset(), kafkaMsg.getTopic(), kafkaMsg.getPartition()); + message = kafkaMsg.getMessage(); + } + break; + + case ENTITY_FULL_UPDATE: { + final HookNotificationV1.EntityUpdateRequest updateRequest = (HookNotificationV1.EntityUpdateRequest) message; + final AtlasEntity.AtlasEntitiesWithExtInfo entities = instanceConverter.toAtlasEntities(updateRequest.getEntities()); + final EntityUpdateRequestV2 v2Request = new EntityUpdateRequestV2(messageUser, entities); + + kafkaMsg = new AtlasKafkaMessage<>(v2Request, kafkaMsg.getOffset(), kafkaMsg.getTopic(), kafkaMsg.getPartition()); + message = kafkaMsg.getMessage(); + } + break; + } + } catch (AtlasBaseException excp) { + LOG.error("handleMessage(): failed to convert V1 message to V2", message.getType().name()); + } + + PreprocessorContext context = preProcessNotificationMessage(kafkaMsg); + + if (isEmptyMessage(kafkaMsg)) { + return new TopicPartitionOffsetResult(kafkaMsg.getTopicPartition(), kafkaMsg.getOffset()); + } + + // Used for intermediate conversions during create and update + String exceptionClassName = StringUtils.EMPTY; + for (int numRetries = 0; numRetries < maxRetries; numRetries++) { + if (LOG.isDebugEnabled()) { + LOG.debug("handleMessage({}): attempt {}", message.getType().name(), numRetries); + } + + try { + RequestContext requestContext = RequestContext.get(); + + requestContext.setAttemptCount(numRetries + 1); + requestContext.setMaxAttempts(maxRetries); + + requestContext.setUser(messageUser, null); + requestContext.setInNotificationProcessing(true); + requestContext.setCreateShellEntityForNonExistingReference(createShellEntityForNonExistingReference); + + switch (message.getType()) { + case ENTITY_CREATE: { + final HookNotificationV1.EntityCreateRequest createRequest = (HookNotificationV1.EntityCreateRequest) message; + final AtlasEntity.AtlasEntitiesWithExtInfo entities = instanceConverter.toAtlasEntities(createRequest.getEntities()); + + if (auditLog == null) { + auditLog = new AuditFilter.AuditLog(messageUser, THREADNAME_PREFIX, + AtlasClient.API_V1.CREATE_ENTITY.getMethod(), + AtlasClient.API_V1.CREATE_ENTITY.getNormalizedPath()); + } + + createOrUpdate(entities, false, stats, context); + } + break; + + case ENTITY_PARTIAL_UPDATE: { + final HookNotificationV1.EntityPartialUpdateRequest partialUpdateRequest = (HookNotificationV1.EntityPartialUpdateRequest) message; + final Referenceable referenceable = partialUpdateRequest.getEntity(); + final AtlasEntity.AtlasEntitiesWithExtInfo entities = instanceConverter.toAtlasEntity(referenceable); + + if (auditLog == null) { + auditLog = new AuditFilter.AuditLog(messageUser, THREADNAME_PREFIX, + AtlasClientV2.API_V2.UPDATE_ENTITY_BY_ATTRIBUTE.getMethod(), + String.format(AtlasClientV2.API_V2.UPDATE_ENTITY_BY_ATTRIBUTE.getNormalizedPath(), partialUpdateRequest.getTypeName())); + } + + AtlasEntityType entityType = typeRegistry.getEntityTypeByName(partialUpdateRequest.getTypeName()); + String guid = AtlasGraphUtilsV2.getGuidByUniqueAttributes(entityType, Collections.singletonMap(partialUpdateRequest.getAttribute(), (Object) partialUpdateRequest.getAttributeValue())); + + // There should only be one root entity + entities.getEntities().get(0).setGuid(guid); + + createOrUpdate(entities, true, stats, context); + } + break; + + case ENTITY_DELETE: { + final HookNotificationV1.EntityDeleteRequest deleteRequest = (HookNotificationV1.EntityDeleteRequest) message; + + if (auditLog == null) { + auditLog = new AuditFilter.AuditLog(messageUser, THREADNAME_PREFIX, + AtlasClientV2.API_V2.DELETE_ENTITY_BY_ATTRIBUTE.getMethod(), + String.format(AtlasClientV2.API_V2.DELETE_ENTITY_BY_ATTRIBUTE.getNormalizedPath(), deleteRequest.getTypeName())); + } + + try { + AtlasEntityType type = (AtlasEntityType) typeRegistry.getType(deleteRequest.getTypeName()); + + EntityMutationResponse response = atlasEntityStore.deleteByUniqueAttributes(type, Collections.singletonMap(deleteRequest.getAttribute(), (Object) deleteRequest.getAttributeValue())); + + stats.updateStats(response); + entityCorrelationManager.add(kafkaMsg.getSpooled(), kafkaMsg.getMsgCreated(), response.getDeletedEntities()); + } catch (ClassCastException cle) { + LOG.error("Failed to delete entity {}", deleteRequest); + } + } + break; + + case ENTITY_FULL_UPDATE: { + final HookNotificationV1.EntityUpdateRequest updateRequest = (HookNotificationV1.EntityUpdateRequest) message; + final AtlasEntity.AtlasEntitiesWithExtInfo entities = instanceConverter.toAtlasEntities(updateRequest.getEntities()); + + if (auditLog == null) { + auditLog = new AuditFilter.AuditLog(messageUser, THREADNAME_PREFIX, + AtlasClientV2.API_V2.UPDATE_ENTITY.getMethod(), + AtlasClientV2.API_V2.UPDATE_ENTITY.getNormalizedPath()); + } + + createOrUpdate(entities, false, stats, context); + } + break; + + case ENTITY_CREATE_V2: { + final EntityCreateRequestV2 createRequestV2 = (EntityCreateRequestV2) message; + final AtlasEntitiesWithExtInfo entities = createRequestV2.getEntities(); + + if (auditLog == null) { + auditLog = new AuditFilter.AuditLog(messageUser, THREADNAME_PREFIX, + AtlasClientV2.API_V2.CREATE_ENTITY.getMethod(), + AtlasClientV2.API_V2.CREATE_ENTITY.getNormalizedPath()); + } + + createOrUpdate(entities, false, stats, context); + } + break; + + case ENTITY_PARTIAL_UPDATE_V2: { + final EntityPartialUpdateRequestV2 partialUpdateRequest = (EntityPartialUpdateRequestV2) message; + final AtlasObjectId entityId = partialUpdateRequest.getEntityId(); + final AtlasEntityWithExtInfo entity = partialUpdateRequest.getEntity(); + + if (auditLog == null) { + auditLog = new AuditFilter.AuditLog(messageUser, THREADNAME_PREFIX, + AtlasClientV2.API_V2.UPDATE_ENTITY.getMethod(), + AtlasClientV2.API_V2.UPDATE_ENTITY.getNormalizedPath()); + } + + EntityMutationResponse response = atlasEntityStore.updateEntity(entityId, entity, true); + + stats.updateStats(response); + } + break; + + case ENTITY_FULL_UPDATE_V2: { + final EntityUpdateRequestV2 updateRequest = (EntityUpdateRequestV2) message; + final AtlasEntitiesWithExtInfo entities = updateRequest.getEntities(); + + if (auditLog == null) { + auditLog = new AuditFilter.AuditLog(messageUser, THREADNAME_PREFIX, + AtlasClientV2.API_V2.UPDATE_ENTITY.getMethod(), + AtlasClientV2.API_V2.UPDATE_ENTITY.getNormalizedPath()); + } + + createOrUpdate(entities, false, stats, context); + } + break; + + case ENTITY_DELETE_V2: { + final EntityDeleteRequestV2 deleteRequest = (EntityDeleteRequestV2) message; + final List entities = deleteRequest.getEntities(); + + try { + for (AtlasObjectId entity : entities) { + if (auditLog == null) { + auditLog = new AuditFilter.AuditLog(messageUser, THREADNAME_PREFIX, + AtlasClientV2.API_V2.DELETE_ENTITY_BY_ATTRIBUTE.getMethod(), + String.format(AtlasClientV2.API_V2.DELETE_ENTITY_BY_ATTRIBUTE.getNormalizedPath(), entity.getTypeName())); + } + + AtlasEntityType type = (AtlasEntityType) typeRegistry.getType(entity.getTypeName()); + + EntityMutationResponse response = atlasEntityStore.deleteByUniqueAttributes(type, entity.getUniqueAttributes()); + + stats.updateStats(response); + entityCorrelationManager.add(kafkaMsg.getSpooled(), kafkaMsg.getMsgCreated(), response.getDeletedEntities()); + } + } catch (ClassCastException cle) { + LOG.error("Failed to do delete entities {}", entities); + } + } + break; + + case IMPORT_TYPES_DEF: { + final AtlasTypesDefImportNotification typeDefImportNotification = (AtlasTypesDefImportNotification) message; + final String importId = typeDefImportNotification.getImportId(); + final AtlasTypesDef typesDef = typeDefImportNotification.getTypesDef(); + try { + asyncImporter.onImportTypeDef(typesDef, importId); + } catch (AtlasBaseException abe) { + LOG.error("IMPORT_TYPE_DEF: {} failed to import type definition: {}", importId, typesDef.toString()); + + asyncImporter.onImportComplete(importId); + importRequestComplete = true; + } + } + break; + + case IMPORT_ENTITY: { + final AtlasEntityImportNotification entityImportNotification = (AtlasEntityImportNotification) message; + final String importId = entityImportNotification.getImportId(); + final AtlasEntity.AtlasEntityWithExtInfo entityWithExtInfo = entityImportNotification.getEntity(); + final int position = entityImportNotification.getPosition(); + + LOG.info("==> IMPORT_ENTITY:processing entity: {} at position: {}", importId, position); + + try { + importRequestComplete = asyncImporter.onImportEntity(entityWithExtInfo, importId, position); + } catch (AtlasBaseException abe) { + importRequestComplete = true; + + LOG.error("IMPORT_ENTITY: {} failed to import entity: {}", importId, entityImportNotification); + } finally { + if (importRequestComplete) { + asyncImporter.onImportComplete(importId); + } + LOG.info("<== IMPORT_ENTITY:processing entity: {} at position: {}", importId, position); + } + } + break; + + default: + throw new IllegalStateException("Unknown notification type: " + message.getType().name()); + } + + if (StringUtils.isNotEmpty(exceptionClassName)) { + LOG.warn("{}: Offset: {}: Pausing & retry: Try: {}: Pause: {} ms. Handled!", + exceptionClassName, kafkaMsg.getOffset(), numRetries, adaptiveWaiter.getWaitDuration()); + exceptionClassName = StringUtils.EMPTY; + } + break; + } catch (Throwable e) { + 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())); + + if (maxRetriesReached || unrecoverableFailure) { + try { + 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); + } + + stats.isFailedMsg = true; + + 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); + } + + 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); + + Thread.sleep(consumerRetryInterval); + } catch (InterruptedException ie) { + LOG.error("Notification consumer thread sleep interrupted"); + } + } + } finally { + RequestContext.clear(); + } + } + + return new TopicPartitionOffsetResult(kafkaMsg.getTopicPartition(), kafkaMsg.getOffset()); + } finally { + AtlasPerfTracer.log(perf); + + stats.timeTakenMs = System.currentTimeMillis() - startTime; + + metricsUtil.onNotificationProcessingComplete(kafkaMsg.getTopic(), kafkaMsg.getPartition(), kafkaMsg.getOffset(), stats); + + if (stats.timeTakenMs > largeMessageProcessingTimeThresholdMs) { + String strMessage = AbstractNotification.getMessageJson(message); + + LOG.warn("msgProcessingTime={}, msgSize={}, topicOffset={}}", stats.timeTakenMs, strMessage.length(), kafkaMsg.getOffset()); + largeMessagesLog.warn("{\"msgProcessingTime\":{},\"msgSize\":{},\"topicOffset\":{},\"data\":{}}", stats.timeTakenMs, strMessage.length(), kafkaMsg.getOffset(), strMessage); + } + + if (auditLog != null) { + auditLog.setHttpStatus(stats.isFailedMsg ? SC_BAD_REQUEST : SC_OK); + auditLog.setTimeTaken(stats.timeTakenMs); + + AuditFilter.audit(auditLog); + } + + Instant now = Instant.now(); + + if (now.isAfter(nextStatsLogTime)) { + LOG.info("STATS: {}", AtlasJson.toJson(metricsUtil.getStats())); + + nextStatsLogTime = AtlasMetricsCounter.getNextHourStartTime(now); + } + + if (importRequestComplete) { + asyncImporter.onCompleteImportRequest(((AtlasEntityImportNotification) message).getImportId()); + } + } + } + + public List getFailedMessages() { + return this.failedMessages; + } + + private void createOrUpdate(AtlasEntitiesWithExtInfo entities, boolean isPartialUpdate, NotificationStat stats, PreprocessorContext context) throws AtlasBaseException { + List entitiesList = entities.getEntities(); + AtlasEntityStream entityStream = new AtlasEntityStream(entities); + + if (commitBatchSize <= 0 || entitiesList.size() <= commitBatchSize) { + EntityMutationResponse response = atlasEntityStore.createOrUpdate(entityStream, isPartialUpdate); + + recordProcessedEntities(response, stats, context); + } else { + for (int fromIdx = 0; fromIdx < entitiesList.size(); fromIdx += commitBatchSize) { + int toIndex = fromIdx + commitBatchSize; + + if (toIndex > entitiesList.size()) { + toIndex = entitiesList.size(); + } + + List entitiesBatch = new ArrayList<>(entitiesList.subList(fromIdx, toIndex)); + + updateProcessedEntityReferences(entitiesBatch, context.getGuidAssignments()); + + AtlasEntitiesWithExtInfo batch = new AtlasEntitiesWithExtInfo(entitiesBatch); + AtlasEntityStream batchStream = new AtlasEntityStream(batch, entityStream); + + EntityMutationResponse response = atlasEntityStore.createOrUpdate(batchStream, isPartialUpdate); + + recordProcessedEntities(response, stats, context); + + RequestContext.get().resetEntityGuidUpdates(); + + entityCorrelationManager.add(context.isSpooledMessage(), context.getMsgCreated(), response.getDeletedEntities()); + + RequestContext.get().clearCache(); + } + } + + if (context != null) { + context.prepareForPostUpdate(); + + List postUpdateEntities = context.getPostUpdateEntities(); + + if (CollectionUtils.isNotEmpty(postUpdateEntities)) { + atlasEntityStore.createOrUpdate(new AtlasEntityStream(postUpdateEntities), true); + } + } + } + + private void preprocessEntities(PreprocessorContext context) { + GenericEntityPreprocessor genericEntityPreprocessor = new GenericEntityPreprocessor(this.entityTypesToIgnore, this.entitiesToIgnore); + + List entities = context.getEntities(); + + if (entities != null) { + for (int i = 0; i < entities.size(); i++) { + AtlasEntity entity = entities.get(i); + genericEntityPreprocessor.preprocess(entity, context); + + if (context.isIgnoredEntity(entity.getGuid())) { + entities.remove(i--); + } + } + } + + Map referredEntities = context.getReferredEntities(); + + if (referredEntities != null) { + for (Iterator> iterator = referredEntities.entrySet().iterator(); iterator.hasNext(); ) { + AtlasEntity entity = iterator.next().getValue(); + genericEntityPreprocessor.preprocess(entity, context); + + if (context.isIgnoredEntity(entity.getGuid())) { + iterator.remove(); + } + } + } + } + + private PreprocessorContext preProcessNotificationMessage(AtlasKafkaMessage kafkaMsg) { + PreprocessorContext context = null; + + if (preprocessEnabled) { + context = new PreprocessorContext(kafkaMsg, typeRegistry, hiveTablesToIgnore, hiveTablesToPrune, hiveTablesCache, + hiveDummyDatabasesToIgnore, hiveDummyTablesToIgnore, hiveTablePrefixesToIgnore, hiveTypesRemoveOwnedRefAttrs, + rdbmsTypesRemoveOwnedRefAttrs, s3V2DirectoryPruneObjectPrefix, updateHiveProcessNameWithQualifiedName, entityCorrelationManager); + + if (CollectionUtils.isNotEmpty(this.entityTypesToIgnore) || CollectionUtils.isNotEmpty(this.entitiesToIgnore)) { + preprocessEntities(context); + } + + if (context.isHivePreprocessEnabled()) { + preprocessHiveTypes(context); + } + + if (skipHiveColumnLineageHive20633) { + skipHiveColumnLineage(context); + } + + if (rdbmsTypesRemoveOwnedRefAttrs) { + rdbmsTypeRemoveOwnedRefAttrs(context); + } + + if (s3V2DirectoryPruneObjectPrefix) { + pruneObjectPrefixForS3V2Directory(context); + } + + if (sparkProcessAttributes) { + preprocessSparkProcessAttributes(context); + } + + context.moveRegisteredReferredEntities(); + + if (context.isHivePreprocessEnabled() && CollectionUtils.isNotEmpty(context.getEntities()) && context.getEntities().size() > 1) { + // move hive_process and hive_column_lineage entities to end of the list + List entities = context.getEntities(); + int count = entities.size(); + + for (int i = 0; i < count; i++) { + AtlasEntity entity = entities.get(i); + + switch (entity.getTypeName()) { + case TYPE_HIVE_PROCESS: + case TYPE_HIVE_COLUMN_LINEAGE: + entities.remove(i--); + entities.add(entity); + count--; + break; + } + } + + if (entities.size() - count > 0) { + LOG.debug("preprocess: moved {} hive_process/hive_column_lineage entities to end of list (listSize={}). topic-offset={}, partition={}", entities.size() - count, entities.size(), kafkaMsg.getOffset(), kafkaMsg.getPartition()); + } + } + } + + return context; + } + + private void rdbmsTypeRemoveOwnedRefAttrs(PreprocessorContext context) { + List entities = context.getEntities(); + + if (entities != null) { + for (int i = 0; i < entities.size(); i++) { + AtlasEntity entity = entities.get(i); + EntityPreprocessor preprocessor = EntityPreprocessor.getRdbmsPreprocessor(entity.getTypeName()); + + if (preprocessor != null) { + preprocessor.preprocess(entity, context); + } + } + } + } + + private void pruneObjectPrefixForS3V2Directory(PreprocessorContext context) { + List entities = new ArrayList<>(); + + if (CollectionUtils.isNotEmpty(context.getEntities())) { + entities.addAll(context.getEntities()); + } + + if (MapUtils.isNotEmpty(context.getReferredEntities())) { + entities.addAll(context.getReferredEntities().values()); + } + + if (CollectionUtils.isNotEmpty(entities)) { + for (AtlasEntity entity : entities) { + EntityPreprocessor preprocessor = EntityPreprocessor.getS3V2Preprocessor(entity.getTypeName()); + + if (preprocessor != null) { + preprocessor.preprocess(entity, context); + } + } + } + } + + private void preprocessHiveTypes(PreprocessorContext context) { + List entities = context.getEntities(); + + if (entities != null) { + for (int i = 0; i < entities.size(); i++) { + AtlasEntity entity = entities.get(i); + EntityPreprocessor preprocessor = EntityPreprocessor.getHivePreprocessor(entity.getTypeName()); + + if (preprocessor != null) { + preprocessor.preprocess(entity, context); + + if (context.isIgnoredEntity(entity.getGuid())) { + entities.remove(i--); + } + } + } + + Map referredEntities = context.getReferredEntities(); + + if (referredEntities != null) { + for (Iterator> iter = referredEntities.entrySet().iterator(); iter.hasNext(); ) { + AtlasEntity entity = iter.next().getValue(); + EntityPreprocessor preprocessor = EntityPreprocessor.getHivePreprocessor(entity.getTypeName()); + + if (preprocessor != null) { + preprocessor.preprocess(entity, context); + + if (context.isIgnoredEntity(entity.getGuid())) { + iter.remove(); + } + } + } + } + + int ignoredEntities = context.getIgnoredEntities().size(); + int prunedEntities = context.getPrunedEntities().size(); + + if (ignoredEntities > 0 || prunedEntities > 0) { + LOG.info("preprocess: ignored entities={}; pruned entities={}. topic-offset={}, partition={}", ignoredEntities, prunedEntities, context.getKafkaMessageOffset(), context.getKafkaPartition()); + } + } + } + + private void skipHiveColumnLineage(PreprocessorContext context) { + List entities = context.getEntities(); + + if (entities != null) { + int lineageCount = 0; + int lineageInputsCount = 0; + int numRemovedEntities = 0; + Set lineageQNames = new HashSet<>(); + + // find if all hive_column_lineage entities have same number of inputs, which is likely to be caused by HIVE-20633 that results in incorrect lineage in some cases + for (int i = 0; i < entities.size(); i++) { + AtlasEntity entity = entities.get(i); + + if (StringUtils.equals(entity.getTypeName(), TYPE_HIVE_COLUMN_LINEAGE)) { + final Object qName = entity.getAttribute(ATTRIBUTE_QUALIFIED_NAME); + + if (qName != null) { + final String qualifiedName = qName.toString(); + + if (lineageQNames.contains(qualifiedName)) { + entities.remove(i--); + + LOG.warn("removed duplicate hive_column_lineage entity: qualifiedName={}. topic-offset={}, partition={}", qualifiedName, context.getKafkaMessageOffset(), context.getKafkaPartition()); + + numRemovedEntities++; + + continue; + } else { + lineageQNames.add(qualifiedName); + } + } + + lineageCount++; + + Object objInputs = entity.getAttribute(ATTRIBUTE_INPUTS); + + if (objInputs instanceof Collection) { + Collection inputs = (Collection) objInputs; + + lineageInputsCount += inputs.size(); + } + } + } + + float avgInputsCount = lineageCount > 0 ? (((float) lineageInputsCount) / lineageCount) : 0; + + if (avgInputsCount > skipHiveColumnLineageHive20633InputsThreshold) { + for (int i = 0; i < entities.size(); i++) { + AtlasEntity entity = entities.get(i); + + if (StringUtils.equals(entity.getTypeName(), TYPE_HIVE_COLUMN_LINEAGE)) { + entities.remove(i--); + + numRemovedEntities++; + } + } + } + + if (numRemovedEntities > 0) { + LOG.warn("removed {} hive_column_lineage entities. Average # of inputs={}, threshold={}, total # of inputs={}. topic-offset={}, partition={}", numRemovedEntities, avgInputsCount, skipHiveColumnLineageHive20633InputsThreshold, lineageInputsCount, context.getKafkaMessageOffset(), context.getKafkaPartition()); + } + } + } + + private boolean isEmptyMessage(AtlasKafkaMessage kafkaMsg) { + final boolean ret; + final HookNotification message = kafkaMsg.getMessage(); + + switch (message.getType()) { + case ENTITY_CREATE_V2: { + AtlasEntitiesWithExtInfo entities = ((EntityCreateRequestV2) message).getEntities(); + + ret = entities == null || CollectionUtils.isEmpty(entities.getEntities()); + } + break; + + case ENTITY_FULL_UPDATE_V2: { + AtlasEntitiesWithExtInfo entities = ((EntityUpdateRequestV2) message).getEntities(); + + ret = entities == null || CollectionUtils.isEmpty(entities.getEntities()); + } + break; + + default: + ret = false; + break; + } + + return ret; + } + + private void recordProcessedEntities(EntityMutationResponse mutationResponse, NotificationStat stats, PreprocessorContext context) { + if (mutationResponse != null) { + if (stats != null) { + stats.updateStats(mutationResponse); + } + + if (context != null) { + if (MapUtils.isNotEmpty(mutationResponse.getGuidAssignments())) { + context.getGuidAssignments().putAll(mutationResponse.getGuidAssignments()); + } + + if (CollectionUtils.isNotEmpty(mutationResponse.getCreatedEntities())) { + for (AtlasEntityHeader entity : mutationResponse.getCreatedEntities()) { + if (entity != null && entity.getGuid() != null) { + context.getCreatedEntities().add(entity.getGuid()); + } + } + } + + if (CollectionUtils.isNotEmpty(mutationResponse.getDeletedEntities())) { + for (AtlasEntityHeader entity : mutationResponse.getDeletedEntities()) { + if (entity != null && entity.getGuid() != null) { + context.getDeletedEntities().add(entity.getGuid()); + } + } + } + } + } + } + + private void updateProcessedEntityReferences(List entities, Map guidAssignments) { + if (CollectionUtils.isNotEmpty(entities) && MapUtils.isNotEmpty(guidAssignments)) { + for (AtlasEntity entity : entities) { + AtlasEntityType entityType = typeRegistry.getEntityTypeByName(entity.getTypeName()); + + if (entityType == null) { + continue; + } + + if (MapUtils.isNotEmpty(entity.getAttributes())) { + for (Map.Entry entry : entity.getAttributes().entrySet()) { + String attrName = entry.getKey(); + Object attrValue = entry.getValue(); + + if (attrValue == null) { + continue; + } + + AtlasAttribute attribute = entityType.getAttribute(attrName); + + if (attribute == null) { // look for a relationship attribute with the same name + attribute = entityType.getRelationshipAttribute(attrName, null); + } + + if (attribute != null && attribute.isObjectRef()) { + updateProcessedEntityReferences(attrValue, guidAssignments); + } + } + } + + if (MapUtils.isNotEmpty(entity.getRelationshipAttributes())) { + for (Map.Entry entry : entity.getRelationshipAttributes().entrySet()) { + Object attrValue = entry.getValue(); + + if (attrValue != null) { + updateProcessedEntityReferences(attrValue, guidAssignments); + } + } + } + } + } + } + + private void updateProcessedEntityReferences(Object objVal, Map guidAssignments) { + if (objVal instanceof AtlasObjectId) { + updateProcessedEntityReferences((AtlasObjectId) objVal, guidAssignments); + } else if (objVal instanceof Collection) { + updateProcessedEntityReferences((Collection) objVal, guidAssignments); + } else if (objVal instanceof Map) { + updateProcessedEntityReferences((Map) objVal, guidAssignments); + } + } + + private void updateProcessedEntityReferences(AtlasObjectId objId, Map guidAssignments) { + String guid = objId.getGuid(); + + if (guid != null && guidAssignments.containsKey(guid)) { + String assignedGuid = guidAssignments.get(guid); + + if (LOG.isDebugEnabled()) { + LOG.debug("{}(guid={}) is already processed; updating its reference to use assigned-guid={}", objId.getTypeName(), guid, assignedGuid); + } + + objId.setGuid(assignedGuid); + objId.setTypeName(null); + objId.setUniqueAttributes(null); + } + } + + private void updateProcessedEntityReferences(Map objId, Map guidAssignments) { + Object guid = objId.get(KEY_GUID); + + if (guid != null && guidAssignments.containsKey(guid)) { + String assignedGuid = guidAssignments.get(guid); + + if (LOG.isDebugEnabled()) { + LOG.debug("{}(guid={}) is already processed; updating its reference to use assigned-guid={}", objId.get(KEY_TYPENAME), guid, assignedGuid); + } + + objId.put(KEY_GUID, assignedGuid); + objId.remove(KEY_TYPENAME); + objId.remove(KEY_UNIQUE_ATTRIBUTES); + } + } + + private void updateProcessedEntityReferences(Collection objIds, Map guidAssignments) { + for (Object objId : objIds) { + updateProcessedEntityReferences(objId, guidAssignments); + } + } + + private void setCurrentUser(String userName) { + Authentication authentication = getAuthenticationForUser(userName); + + if (LOG.isDebugEnabled()) { + if (authentication != null) { + LOG.debug("setCurrentUser(): notification processing will be authorized as user '{}'", userName); + } else { + LOG.debug("setCurrentUser(): Failed to get authentication for user '{}'.", userName); + } + } + + SecurityContextHolder.getContext().setAuthentication(authentication); + } + + private Authentication getAuthenticationForUser(String userName) { + Authentication ret = null; + + if (StringUtils.isNotBlank(userName)) { + ret = authnCache != null ? authnCache.get(userName) : null; + + if (ret == null) { + List grantedAuths = getAuthoritiesFromUGI(userName); + UserDetails principal = new User(userName, "", grantedAuths); + + ret = new UsernamePasswordAuthenticationToken(principal, ""); + + if (authnCache != null) { + authnCache.put(userName, ret); + } + } + } + + return ret; + } + + private void recordFailedMessages(List failedMessages) { + //logging failed messages + for (String message : failedMessages) { + failedMessageLog.error("[DROPPED_NOTIFICATION] {}", message); + } + + failedMessages.clear(); + } + + private List trimAndPurge(String[] values, String defaultValue) { + final List ret; + + if (values != null && values.length > 0) { + ret = new ArrayList<>(values.length); + + for (String val : values) { + if (StringUtils.isNotBlank(val)) { + ret.add(val.trim()); + } + } + } else if (StringUtils.isNotBlank(defaultValue)) { + ret = Collections.singletonList(defaultValue.trim()); + } else { + ret = Collections.emptyList(); + } + + return ret; + } + + private void preprocessSparkProcessAttributes(PreprocessorContext context) { + List entities = context.getEntities(); + + if (entities != null) { + for (int i = 0; i < entities.size(); i++) { + AtlasEntity entity = entities.get(i); + EntityPreprocessor preprocessor = EntityPreprocessor.getSparkPreprocessor(entity.getTypeName()); + + if (preprocessor != null) { + preprocessor.preprocess(entity, context); + } + } + } + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/TopicPartitionOffsetResult.java b/webapp/src/main/java/org/apache/atlas/notification/TopicPartitionOffsetResult.java new file mode 100644 index 00000000000..aebf1b744d9 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/TopicPartitionOffsetResult.java @@ -0,0 +1,61 @@ +/** + * 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.notification.pc.Ticket; +import org.apache.kafka.common.TopicPartition; + +import java.util.Set; + +public class TopicPartitionOffsetResult { + private final TopicPartition topicPartition; + private final long offset; + private final String key; + + private Set additionalInfo; + + public TopicPartitionOffsetResult(TopicPartition topicPartition, long offset) { + this.topicPartition = topicPartition; + this.offset = offset; + this.key = toKey(); + } + + public TopicPartition getTopicPartition() { + return this.topicPartition; + } + + public long getOffset() { + return this.offset; + } + + public Set getAdditionalInfo() { + return additionalInfo; + } + + public void setAdditionalInfo(Set additionalInfo) { + this.additionalInfo = additionalInfo; + } + + public String getKey() { + return this.key; + } + + private String toKey() { + return Ticket.getKey(getTopicPartition().topic(), getTopicPartition().partition(), getOffset()); + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/pc/Consumer.java b/webapp/src/main/java/org/apache/atlas/notification/pc/Consumer.java new file mode 100644 index 00000000000..6da0a6aae20 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/pc/Consumer.java @@ -0,0 +1,86 @@ +/** + * 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.pc; + +import org.apache.atlas.notification.SerialEntityProcessor; +import org.apache.atlas.notification.TopicPartitionOffsetResult; +import org.apache.atlas.pc.WorkItemConsumer; +import org.apache.atlas.type.AtlasType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.atomic.AtomicLong; + +public class Consumer extends WorkItemConsumer { + private static final Logger LOG = LoggerFactory.getLogger(Consumer.class); + + private ReferenceKeeper referenceKeeper; + private final SerialEntityProcessor serialEntityCreator; + + private final AtomicLong counter; + private String lastKey; + + public Consumer(ReferenceKeeper referenceKeeper, + SerialEntityProcessor serialEntityCreator, + BlockingQueue queue) { + super(queue); + this.referenceKeeper = referenceKeeper; + this.serialEntityCreator = serialEntityCreator; + this.counter = new AtomicLong(0); + } + + @Override + protected void processItem(Ticket ticket) { + TopicPartitionOffsetResult result = null; + long count = counter.incrementAndGet(); + + List dependents = ticket.getDependents(); + try { + lastKey = ticket.getKey(); + + referenceKeeper.awaitDependents(dependents); + + result = serialEntityCreator.handleMessage(ticket); + } catch (Exception e) { + LOG.error("Error handling message: {}", AtlasType.toJson(ticket.msg), e); + } finally { + this.referenceKeeper.countDown(ticket.getKey()); + if (result == null) { + result = ticket.createTopicPartitionOffsetResult(); + } + + result.setAdditionalInfo(ticket.getQualifiedNamesSet()); + addResult(result); + LOG.debug("Total: {}: Ticket: {}: Unlocking: {}: QNames: {}: Refs: {}", + count, ticket.getKey(), dependents, + ticket.getQualifiedNamesSet(), ticket.getReferencedSet()); + } + } + + @Override + protected void doCommit() { + } + + @Override + protected void commitDirty() { + super.commitDirty(); + LOG.debug("{}: Total: {}", lastKey, counter.get()); + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/pc/ConsumerBuilder.java b/webapp/src/main/java/org/apache/atlas/notification/pc/ConsumerBuilder.java new file mode 100644 index 00000000000..f9439e63c5f --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/pc/ConsumerBuilder.java @@ -0,0 +1,73 @@ +/** + * 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.pc; + +import org.apache.atlas.notification.EntityCorrelationManager; +import org.apache.atlas.notification.SerialEntityProcessor; +import org.apache.atlas.pc.WorkItemBuilder; +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.type.AtlasTypeRegistry; +import org.apache.atlas.util.AtlasMetricsUtil; +import org.apache.commons.configuration2.Configuration; +import org.slf4j.Logger; +import org.springframework.security.core.Authentication; + +import java.util.Map; +import java.util.concurrent.BlockingQueue; + +public class ConsumerBuilder implements WorkItemBuilder { + private final AtlasEntityStore entityStore; + private final AtlasInstanceConverter instanceConverter; + private final EntityCorrelationManager entityCorrelationManager; + private final AtlasTypeRegistry typeRegistry; + private final Logger failedLog; + private final Logger largeMessagesLog; + private final Configuration applicationProperties; + private final AtlasMetricsUtil metricsUtil; + private final Map authnCache; + private final AsyncImporter asyncImporter; + private ReferenceKeeper referenceKeeper; + + public ConsumerBuilder(ReferenceKeeper referenceKeeper, Configuration applicationProperties, AtlasMetricsUtil metricsUtil, Map authnCache, + AtlasEntityStore entityStore, AtlasInstanceConverter instanceConverter, EntityCorrelationManager entityCorrelationManager, + AtlasTypeRegistry typeRegistry, Logger failedLog, Logger largeMessagesLog, AsyncImporter asyncImporter) { + this.referenceKeeper = referenceKeeper; + this.applicationProperties = applicationProperties; + this.metricsUtil = metricsUtil; + this.authnCache = authnCache; + this.entityStore = entityStore; + this.instanceConverter = instanceConverter; + this.entityCorrelationManager = entityCorrelationManager; + this.typeRegistry = typeRegistry; + this.failedLog = failedLog; + this.largeMessagesLog = largeMessagesLog; + this.asyncImporter = asyncImporter; + } + + @Override + public Consumer build(BlockingQueue queue) { + return new Consumer( + this.referenceKeeper, + new SerialEntityProcessor(applicationProperties, metricsUtil, authnCache, + entityStore, instanceConverter, entityCorrelationManager, typeRegistry, + failedLog, largeMessagesLog, asyncImporter), + queue); + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/pc/DependentsTracking.java b/webapp/src/main/java/org/apache/atlas/notification/pc/DependentsTracking.java new file mode 100644 index 00000000000..05ddf47bc47 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/pc/DependentsTracking.java @@ -0,0 +1,73 @@ +/** + * 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.pc; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; + +public class DependentsTracking { + private static final Logger LOG = LoggerFactory.getLogger(DependentsTracking.class); + + private final Map objectSync; + + public DependentsTracking() { + this.objectSync = new ConcurrentHashMap<>(); + } + + public CountDownLatch get(String key) { + return objectSync.get(key); + } + + public void addReference(String key) { + this.objectSync.put(key, new CountDownLatch(1)); + } + + public void decrementReference(String key) { + if (!objectSync.containsKey(key)) { + LOG.warn("Key: {}: Not found!", key); + return; + } + + CountDownLatch countDownLatch = objectSync.get(key); + countDownLatch.countDown(); + } + + public void remove(String key) { + this.objectSync.remove(key); + } + + public void awaitDependents(Collection keysToLock) throws InterruptedException { + for (String key : keysToLock) { + CountDownLatch latch = objectSync.get(key); + if (latch == null) { + continue; + } + + latch.await(); + } + } + + public int size() { + return this.objectSync.size(); + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/pc/Manager.java b/webapp/src/main/java/org/apache/atlas/notification/pc/Manager.java new file mode 100644 index 00000000000..b5a66bae619 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/pc/Manager.java @@ -0,0 +1,83 @@ +/** + * 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.pc; + +import org.apache.atlas.notification.TopicPartitionOffsetResult; +import org.apache.atlas.pc.WorkItemManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.util.CollectionUtils; + +import java.util.List; + +public class Manager extends WorkItemManager { + private static final Logger LOG = LoggerFactory.getLogger(Manager.class); + private static final String MANAGER_NAME_FORMAT = "ingest"; + + private final ReferenceKeeper referenceKeeper; + + public Manager(ReferenceKeeper referenceKeeper, int batchSize, int numWorkers, ConsumerBuilder consumerBuilder) { + super(consumerBuilder, String.format(MANAGER_NAME_FORMAT), batchSize, numWorkers, true); + + this.referenceKeeper = referenceKeeper; + } + + public TopicPartitionOffsetResult submit(Ticket ticket) { + try { + if (!ticket.isMessageHandled()) { + this.drain(); + } + + this.referenceKeeper.register(ticket); + super.checkProduce(ticket); + + if (LOG.isDebugEnabled()) { + LOG.debug("{}: {}: Types: {}, QNames: {}, Refs: {}, Dependents: {}, Types: {}", ticket.getKey(), ticket.getMessage().getMessage().getType().name(), + ticket.getTypes(), ticket.getQualifiedNamesSet(), ticket.getReferencedSet(), ticket.getDependents(), ticket.getTypes()); + } + } + catch (Exception e) { + LOG.error("{}: Error: Could not submit!", ticket.getKey(), e); + } + + return this.referenceKeeper.getCachedResult(); + } + + public void shutdown() { + try { + super.shutdown(); + } catch (InterruptedException e) { + LOG.warn("Interrupted!"); + } + } + + public TopicPartitionOffsetResult getResult() { + extractAccumulatedResults(); + return this.referenceKeeper.getResult(); + } + + private void extractAccumulatedResults() { + List res = super.getTypedResults(); + if (CollectionUtils.isEmpty(res)) { + return; + } + + res.parallelStream().forEach(x -> this.referenceKeeper.deregister(x)); + res.clear(); + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/pc/MiscUtils.java b/webapp/src/main/java/org/apache/atlas/notification/pc/MiscUtils.java new file mode 100644 index 00000000000..b58c7169b38 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/pc/MiscUtils.java @@ -0,0 +1,43 @@ +/** + * 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.pc; + +import org.apache.atlas.type.AtlasEntityType; +import org.apache.atlas.type.AtlasTypeRegistry; + +import java.util.Collection; +import java.util.Set; + +public class MiscUtils { + private static final String[] ENTITY_TYPE_NAME_PROCESS = new String[] {"Process", "ProcessExecution", "ddl"}; + + private MiscUtils() { + // to block instantiation + } + + public static void extractAllProcessTypeNames(Set processTypeNames, AtlasTypeRegistry typeRegistry) { + Collection typeCollection = typeRegistry.getAllEntityTypes(); + for (String processTypes : ENTITY_TYPE_NAME_PROCESS) { + for (AtlasEntityType entityType : typeCollection) { + if (entityType.isSubTypeOf(processTypes)) { + processTypeNames.add(entityType.getTypeName()); + } + } + } + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/pc/ReferenceKeeper.java b/webapp/src/main/java/org/apache/atlas/notification/pc/ReferenceKeeper.java new file mode 100644 index 00000000000..5aaee7e26c9 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/pc/ReferenceKeeper.java @@ -0,0 +1,118 @@ +/** + * 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.pc; + +import org.apache.atlas.notification.TopicPartitionOffsetResult; +import org.apache.commons.collections.CollectionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +public class ReferenceKeeper { + private static final Logger LOG = LoggerFactory.getLogger(ReferenceKeeper.class); + + private final ReferenceFinder referenceFinder; + private final Map> currentlyProcessedEntities; + private final ResultsCollector resultsCollector; + private final DependentsTracking dependentsTracking; + + public ReferenceKeeper() { + this.currentlyProcessedEntities = new ConcurrentHashMap<>(); + this.referenceFinder = new ReferenceFinder(); + this.resultsCollector = new ResultsCollector(); + this.dependentsTracking = new DependentsTracking(); + } + + public void register(Ticket ticket) { + this.dependentsTracking.addReference(ticket.getKey()); + this.resultsCollector.markProduced(ticket.getKey(), null); + + Set refs = referenceFinder.find(this.currentlyProcessedEntities, ticket.getQualifiedNamesSet(), ticket.getReferencedSet()); + ticket.addDependents(refs); + LOG.debug("Ticket: {}: References: {}: Total: {}", ticket.getKey(), refs.size(), this.currentlyProcessedEntities.size()); + addAll(ticket.getQualifiedNamesSet(), ticket.getKey()); + } + + private void addAll(Collection entityKeys, String ticketKey) { + entityKeys.parallelStream().forEach(x -> { + if (!currentlyProcessedEntities.containsKey(x)) { + currentlyProcessedEntities.put(x, new ConcurrentHashMap<>()); + } + currentlyProcessedEntities.get(x).put(ticketKey, 0); + }); + } + + public void removeAll(Collection entityKeys, String ticketKey) { + if (CollectionUtils.isEmpty(entityKeys)) { + return; + } + + entityKeys.parallelStream().forEach(key -> { + if (currentlyProcessedEntities.containsKey(key)) { + currentlyProcessedEntities.get(key).remove(ticketKey); + if (currentlyProcessedEntities.get(key).size() == 0) { + currentlyProcessedEntities.remove(key); + } + } + }); + } + + public TopicPartitionOffsetResult getCachedResult() { + return this.resultsCollector.getCached(); + } + + public TopicPartitionOffsetResult getResult() { + return this.resultsCollector.get(); + } + + public void deregister(TopicPartitionOffsetResult result) { + removeAll(result.getAdditionalInfo(), result.getKey()); + + this.resultsCollector.markProcessed(result); + this.dependentsTracking.remove(result.getKey()); + } + + public void awaitDependents(List dependents) throws InterruptedException { + this.dependentsTracking.awaitDependents(dependents); + } + + public void countDown(String key) { + this.dependentsTracking.decrementReference(key); + } + + public static class ReferenceFinder { + public Set find(Map> currentlyProcessedEntities, Set qualifiedNamesSet, Set referencedSet) { + Set ret = ConcurrentHashMap.newKeySet(); + + isReferenced(currentlyProcessedEntities, ret, referencedSet); + isReferenced(currentlyProcessedEntities, ret, qualifiedNamesSet); + + return ret; + } + + private void isReferenced(Map> currentlyProcessedEntities, Set ret, Set incoming) { + incoming.parallelStream().map(currentlyProcessedEntities::get).filter(Objects::nonNull).forEach(x -> ret.addAll(x.keySet())); + } + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/pc/ResultsCollector.java b/webapp/src/main/java/org/apache/atlas/notification/pc/ResultsCollector.java new file mode 100644 index 00000000000..363c198204c --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/pc/ResultsCollector.java @@ -0,0 +1,50 @@ +/** + * 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.pc; + +import org.apache.atlas.notification.TopicPartitionOffsetResult; +import org.apache.atlas.pc.StatusReporter; + +public class ResultsCollector { + private static final long STATUS_REPORT_TIMEOUT_DURATION = 20 * 1000; // 20 secs + + private final StatusReporter statusReporter; + + private TopicPartitionOffsetResult storedResult; + + public ResultsCollector() { + this.statusReporter = new StatusReporter<>(STATUS_REPORT_TIMEOUT_DURATION); + } + + public TopicPartitionOffsetResult getCached() { + return this.storedResult; + } + + public TopicPartitionOffsetResult get() { + this.storedResult = statusReporter.ack(); + return this.storedResult; + } + + public void markProduced(String key, TopicPartitionOffsetResult value) { + this.statusReporter.produced(key, value); + } + + public void markProcessed(TopicPartitionOffsetResult result) { + this.statusReporter.processed(result.getKey(), result); + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/pc/Ticket.java b/webapp/src/main/java/org/apache/atlas/notification/pc/Ticket.java new file mode 100644 index 00000000000..ddafbd2aa83 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/notification/pc/Ticket.java @@ -0,0 +1,329 @@ +/** + * 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.pc; + +import org.apache.atlas.kafka.AtlasKafkaMessage; +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.notification.TopicPartitionOffsetResult; +import org.apache.atlas.notification.preprocessor.EntityPreprocessor; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class Ticket { + private static final Logger LOG = LoggerFactory.getLogger(Ticket.class); + private static final String QUALIFIED_NAME_NAMESPACE_SEPARATOR = "@"; + private static final String QUALIFIED_NAME_LHS_SEPARATOR = "."; + private static final String QUALIFIED_NAME_FORMAT = "%s" + QUALIFIED_NAME_NAMESPACE_SEPARATOR + "%s"; + private static final Set processTypes = new HashSet<>(); + + final AtlasKafkaMessage msg; + + private final Set qualifiedNamesSet; + private final Set referencedSet; + private final Set allTypes; + private final List dependents; + + public static Set getProcessNameTypes() { + return processTypes; + } + + public Ticket(AtlasKafkaMessage msg) { + this.msg = msg; + this.qualifiedNamesSet = new HashSet<>(); + this.referencedSet = new HashSet<>(); + this.allTypes = new HashSet<>(); + + try { + updateMetaInfo(msg); + } catch (Exception e) { + LOG.error("Error fetching metadata.", e); + } + + dependents = new ArrayList<>(); + } + + private void updateMetaInfo(AtlasKafkaMessage msg) { + HookNotification message = msg.getMessage(); + switch (message.getType()) { + case ENTITY_CREATE_V2: { + final HookNotification.EntityCreateRequestV2 createRequestV2 = (HookNotification.EntityCreateRequestV2) message; + updateMetadata(createRequestV2.getEntities()); + break; + } + case ENTITY_FULL_UPDATE_V2: { + final HookNotification.EntityUpdateRequestV2 updateRequest = (HookNotification.EntityUpdateRequestV2) message; + updateMetadata(updateRequest.getEntities()); + break; + } + + case ENTITY_PARTIAL_UPDATE_V2: { + final HookNotification.EntityPartialUpdateRequestV2 partialUpdateRequest = (HookNotification.EntityPartialUpdateRequestV2) message; + updateMetadata(partialUpdateRequest.getEntityId()); + + fillQualifiedNames(qualifiedNamesSet, partialUpdateRequest.getEntity()); + fillRefs(referencedSet, qualifiedNamesSet, partialUpdateRequest.getEntity()); + break; + } + + case ENTITY_DELETE_V2: { + final HookNotification.EntityDeleteRequestV2 deleteRequest = (HookNotification.EntityDeleteRequestV2) message; + for (AtlasObjectId objectId : deleteRequest.getEntities()) { + updateMetadata(objectId); + } + break; + } + } + } + + private void updateMetadata(AtlasObjectId objectId) { + if (MapUtils.isNotEmpty(objectId.getUniqueAttributes()) && objectId.getUniqueAttributes().values() instanceof Collection) { + this.allTypes.add(objectId.getTypeName()); + + for (Object o : objectId.getUniqueAttributes().values()) { + if (o instanceof String) { + String qualifiedName = (String) o; + this.qualifiedNamesSet.add(qualifiedName); + addInferredQNames(this.referencedSet, qualifiedName, true); + } + } + } + } + + private void updateMetadata(AtlasEntity.AtlasEntitiesWithExtInfo entities) { + addEntityQualifiedNames(this.qualifiedNamesSet, entities); + getReferencedSet(this.referencedSet, this.qualifiedNamesSet, entities); + addTypes(this.allTypes, entities); + } + + public void addTypes(Set ret, AtlasEntity.AtlasEntitiesWithExtInfo entities) { + if (entities != null && CollectionUtils.isNotEmpty(entities.getEntities())) { + for (AtlasEntity entity : entities.getEntities()) { + ret.add(entity.getTypeName()); + } + } + + if (entities != null && MapUtils.isNotEmpty(entities.getReferredEntities())) { + for (AtlasEntity entity : entities.getReferredEntities().values()) { + ret.add(entity.getTypeName()); + } + } + } + + public void getReferencedSet(Set ret, Set qualifiedNamesSet, AtlasEntity.AtlasEntitiesWithExtInfo entities) { + if (entities != null && CollectionUtils.isNotEmpty(entities.getEntities())) { + for (AtlasEntity entity : entities.getEntities()) { + fetchReferences(ret, qualifiedNamesSet, entity); + } + } + + if (entities != null && MapUtils.isNotEmpty(entities.getReferredEntities())) { + for (AtlasEntity entity : entities.getReferredEntities().values()) { + fetchReferences(ret, qualifiedNamesSet, entity); + } + } + } + + public static void fillQualifiedNames(Set ret, AtlasEntity.AtlasEntityWithExtInfo entityWithExtInfo) { + if (entityWithExtInfo == null) { + return; + } + + fillQualifiedNames(ret, Collections.singleton(entityWithExtInfo.getEntity())); + if (MapUtils.isNotEmpty(entityWithExtInfo.getReferredEntities())) { + fillQualifiedNames(ret, entityWithExtInfo.getReferredEntities().values()); + } + } + + public void fillRefs(Set ret, Set qualifiedNamesSet, AtlasEntity.AtlasEntityWithExtInfo entityWithExtInfo) { + if (entityWithExtInfo == null) { + return; + } + + fetchReferences(ret, qualifiedNamesSet, entityWithExtInfo.getEntity()); + if (MapUtils.isNotEmpty(entityWithExtInfo.getReferredEntities())) { + for (AtlasEntity entity : entityWithExtInfo.getReferredEntities().values()) { + fetchReferences(ret, qualifiedNamesSet, entity); + } + } + } + + public static void addEntityQualifiedNames(Set ret, AtlasEntity.AtlasEntitiesWithExtInfo entities) { + if (entities == null) { + return; + } + + fillQualifiedNames(ret, entities.getEntities()); + if (MapUtils.isNotEmpty(entities.getReferredEntities())) { + fillQualifiedNames(ret, entities.getReferredEntities().values()); + } + } + + private void fetchReferences(Set ret, Set qualifiedNamesSet, AtlasEntity entity) { + if (entity == null) { + return; + } + + boolean processReferenceFound = processTypes.contains(entity.getTypeName()); + if (MapUtils.isNotEmpty(entity.getRelationshipAttributes())) { + for (Object o : entity.getRelationshipAttributes().values()) { + if (o instanceof List) { + List list = (List) o; + for (Object ox : list) { + String qualifiedName = EntityPreprocessor.getQualifiedName(ox); + + if (StringUtils.isNotEmpty(qualifiedName) && !qualifiedNamesSet.contains(qualifiedName)) { + ret.add(qualifiedName); + + if (processReferenceFound) { + addInferredQNames(ret, qualifiedName); + } + } + } + } else { + String qualifiedName = EntityPreprocessor.getQualifiedName(o); + if (StringUtils.isNotEmpty(qualifiedName) && !qualifiedNamesSet.contains(qualifiedName)) { + ret.add(qualifiedName); + + if (processReferenceFound) { + addInferredQNames(ret, qualifiedName); + } + } + } + } + } + + if (MapUtils.isNotEmpty(entity.getAttributes())) { + for (Object o : entity.getAttributes().values()) { + if (o instanceof List) { + List list = (List) o; + for (Object ox : list) { + String qualifiedName = EntityPreprocessor.getQualifiedName(ox); + + if (StringUtils.isNotEmpty(qualifiedName) && !qualifiedNamesSet.contains(qualifiedName)) { + ret.add(qualifiedName); + + if (processReferenceFound) { + addInferredQNames(ret, qualifiedName); + } + } + } + } else { + String qualifiedName = EntityPreprocessor.getQualifiedName(o); + + if (StringUtils.isNotEmpty(qualifiedName) && !qualifiedNamesSet.contains(qualifiedName)) { + ret.add(qualifiedName); + + if (processReferenceFound) { + addInferredQNames(ret, qualifiedName); + } + } + } + } + } + + if (processReferenceFound) { + this.qualifiedNamesSet.addAll(this.referencedSet); + } + } + + private static void fillQualifiedNames(Set ret, Collection entityCollection) { + if (CollectionUtils.isNotEmpty(entityCollection)) { + for (AtlasEntity entity : entityCollection) { + String qualifiedName = EntityPreprocessor.getQualifiedName(entity); + if (StringUtils.isEmpty(qualifiedName)) { + continue; + } + + ret.add(qualifiedName); + } + } + } + + private static void addInferredQNames(Set ret, String qualifiedName) { + addInferredQNames(ret, qualifiedName, false); + } + + private static void addInferredQNames(Set ret, String qualifiedName, boolean inferAll) { + if (StringUtils.isEmpty(qualifiedName) || !StringUtils.contains(qualifiedName, QUALIFIED_NAME_LHS_SEPARATOR)) { + return; + } + + String namespace = StringUtils.substringAfter(qualifiedName, QUALIFIED_NAME_NAMESPACE_SEPARATOR); + String shortened = qualifiedName; + while (StringUtils.isNotEmpty(shortened) && StringUtils.contains(shortened, QUALIFIED_NAME_LHS_SEPARATOR)) { + shortened = StringUtils.substringBeforeLast(shortened, QUALIFIED_NAME_LHS_SEPARATOR); + if (inferAll || shortened.contains(".")) { + ret.add(String.format(QUALIFIED_NAME_FORMAT, shortened, namespace)); + } + } + } + + public boolean isMessageHandled() { + return CollectionUtils.isNotEmpty(this.allTypes) + || CollectionUtils.isNotEmpty(this.qualifiedNamesSet) + || CollectionUtils.isNotEmpty(this.referencedSet); + } + + public String getKey() { + return getKey(msg.getTopicPartition().topic(), msg.getTopicPartition().partition(), msg.getOffset()); + } + + public static String getKey(String topic, long partition, long offset) { + return String.format("%s-%s-%s", topic, partition, offset); + } + + public AtlasKafkaMessage getMessage() { + return msg; + } + + public Set getQualifiedNamesSet() { + return this.qualifiedNamesSet; + } + + public Set getReferencedSet() { + return this.referencedSet; + } + + public Set getTypes() { + return this.allTypes; + } + + public void addDependents(Collection dependents) { + this.dependents.addAll(dependents); + } + + public List getDependents() { + return this.dependents; + } + + public TopicPartitionOffsetResult createTopicPartitionOffsetResult() { + return new TopicPartitionOffsetResult(this.msg.getTopicPartition(), this.msg.getOffset()); + } +} diff --git a/webapp/src/main/java/org/apache/atlas/notification/preprocessor/EntityPreprocessor.java b/webapp/src/main/java/org/apache/atlas/notification/preprocessor/EntityPreprocessor.java index a950a43a6f7..197cf9dbb14 100644 --- a/webapp/src/main/java/org/apache/atlas/notification/preprocessor/EntityPreprocessor.java +++ b/webapp/src/main/java/org/apache/atlas/notification/preprocessor/EntityPreprocessor.java @@ -111,7 +111,7 @@ public String getTypeName() { public abstract void preprocess(AtlasEntity entity, PreprocessorContext context); - public String getTypeName(Object obj) { + public static String getTypeName(Object obj) { Object ret = null; if (obj instanceof AtlasObjectId) { @@ -127,7 +127,7 @@ public String getTypeName(Object obj) { return ret != null ? ret.toString() : null; } - public String getQualifiedName(Object obj) { + public static String getQualifiedName(Object obj) { Map attributes = null; if (obj instanceof AtlasObjectId) { diff --git a/webapp/src/main/java/org/apache/atlas/util/AdaptiveWaiter.java b/webapp/src/main/java/org/apache/atlas/util/AdaptiveWaiter.java new file mode 100644 index 00000000000..f831ff45315 --- /dev/null +++ b/webapp/src/main/java/org/apache/atlas/util/AdaptiveWaiter.java @@ -0,0 +1,79 @@ +/** + * 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.util; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class AdaptiveWaiter { + private static final Logger LOG = LoggerFactory.getLogger(AdaptiveWaiter.class); + + private final long increment; + private final long maxDuration; + private final long minDuration; + private final long resetInterval; + private long lastWaitAt; + + @VisibleForTesting + public long waitDuration; + + public AdaptiveWaiter(long minDuration, long maxDuration, long increment) { + this.minDuration = minDuration; + this.maxDuration = maxDuration; + this.increment = increment; + this.waitDuration = minDuration; + this.lastWaitAt = 0; + this.resetInterval = maxDuration * 2; + } + + public void pause(Exception ex) { + setWaitDurations(); + + try { + if (LOG.isDebugEnabled()) { + LOG.debug("{} in NotificationHookConsumer. Waiting for {} ms for recovery.", ex.getClass().getName(), waitDuration, ex); + } + + Thread.sleep(waitDuration); + } catch (InterruptedException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("{} in NotificationHookConsumer. Waiting for recovery interrupted.", ex.getClass().getName(), e); + } + } + } + + private void setWaitDurations() { + long timeSinceLastWait = (lastWaitAt == 0) ? 0 : System.currentTimeMillis() - lastWaitAt; + + lastWaitAt = System.currentTimeMillis(); + + if (timeSinceLastWait > resetInterval) { + waitDuration = minDuration; + } else { + waitDuration += increment; + if (waitDuration > maxDuration) { + waitDuration = maxDuration; + } + } + } + + public long getWaitDuration() { + return this.waitDuration; + } +} diff --git a/webapp/src/test/java/org/apache/atlas/notification/AdaptiveWaiterTest.java b/webapp/src/test/java/org/apache/atlas/notification/AdaptiveWaiterTest.java index 9b4a3a8b17b..5be4b55d41f 100644 --- a/webapp/src/test/java/org/apache/atlas/notification/AdaptiveWaiterTest.java +++ b/webapp/src/test/java/org/apache/atlas/notification/AdaptiveWaiterTest.java @@ -17,6 +17,7 @@ */ package org.apache.atlas.notification; +import org.apache.atlas.util.AdaptiveWaiter; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -26,11 +27,11 @@ public class AdaptiveWaiterTest { private final int maxDuration = 100; private final int minDuration = 5; private final int increment = 5; - private NotificationHookConsumer.AdaptiveWaiter waiter; + private AdaptiveWaiter waiter; @BeforeClass public void setup() { - waiter = new NotificationHookConsumer.AdaptiveWaiter(minDuration, maxDuration, increment); + waiter = new AdaptiveWaiter(minDuration, maxDuration, increment); } @Test diff --git a/webapp/src/test/java/org/apache/atlas/notification/ReferenceFinderTest.java b/webapp/src/test/java/org/apache/atlas/notification/ReferenceFinderTest.java new file mode 100644 index 00000000000..c6ec1255b73 --- /dev/null +++ b/webapp/src/test/java/org/apache/atlas/notification/ReferenceFinderTest.java @@ -0,0 +1,174 @@ +/** + * 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.notification.pc.ReferenceKeeper; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +public class ReferenceFinderTest { + private ReferenceKeeper.ReferenceFinder referenceFinder; + + // Helpers to build currentlyProcessedEntities: qualifiedName -> {ticketKey -> 0} + private Map> buildProcessedEntities(String qualifiedName, String... ticketKeys) { + Map> map = new ConcurrentHashMap<>(); + addToProcessedEntities(map, qualifiedName, ticketKeys); + return map; + } + + private void addToProcessedEntities(Map> map, String qualifiedName, String... ticketKeys) { + map.computeIfAbsent(qualifiedName, k -> new ConcurrentHashMap<>()); + for (String key : ticketKeys) { + map.get(qualifiedName).put(key, 0); + } + } + + @BeforeMethod + public void setup() { + referenceFinder = new ReferenceKeeper.ReferenceFinder(); + } + + @Test + public void testFind_withEmptyCurrentlyProcessedEntities_returnsEmpty() { + Set result = referenceFinder.find( + Collections.emptyMap(), + Collections.singleton("tableA@cluster"), + Collections.singleton("dbA@cluster")); + + assertTrue(result.isEmpty()); + } + + @Test + public void testFind_withQualifiedNameMatch_returnsDependentTicketKey() { + // ticket-1 is currently processing tableA@cluster + Map> inFlight = buildProcessedEntities("tableA@cluster", "ticket-1"); + + // current ticket also owns tableA@cluster + Set result = referenceFinder.find( + inFlight, + Collections.singleton("tableA@cluster"), + Collections.emptySet()); + + assertEquals(result.size(), 1); + assertTrue(result.contains("ticket-1")); + } + + @Test + public void testFind_withReferencedSetMatch_returnsDependentTicketKey() { + // ticket-2 is currently processing dbA@cluster + Map> inFlight = buildProcessedEntities("dbA@cluster", "ticket-2"); + + // current ticket references dbA@cluster (not as an owner, but as a reference) + Set result = referenceFinder.find( + inFlight, + Collections.emptySet(), + Collections.singleton("dbA@cluster")); + + assertEquals(result.size(), 1); + assertTrue(result.contains("ticket-2")); + } + + @Test + public void testFind_withNoOverlap_returnsEmpty() { + // ticket-3 is processing an unrelated entity + Map> inFlight = buildProcessedEntities("tableZ@cluster", "ticket-3"); + + Set result = referenceFinder.find( + inFlight, + Collections.singleton("tableA@cluster"), + Collections.singleton("dbA@cluster")); + + assertTrue(result.isEmpty()); + } + + @Test + public void testFind_withMultipleTicketsOnSameEntity_returnsAllKeys() { + // two tickets both processing tableA@cluster — current ticket must wait for both + Map> inFlight = buildProcessedEntities("tableA@cluster", "ticket-1", "ticket-2"); + + Set result = referenceFinder.find( + inFlight, + Collections.singleton("tableA@cluster"), + Collections.emptySet()); + + assertEquals(result.size(), 2); + assertTrue(result.contains("ticket-1")); + assertTrue(result.contains("ticket-2")); + } + + @Test + public void testFind_withOverlapOnBothQNamesAndRefs_returnsUnion() { + // ticket-1 owns tableA, ticket-2 owns dbA — current ticket overlaps with both + Map> inFlight = new ConcurrentHashMap<>(); + addToProcessedEntities(inFlight, "tableA@cluster", "ticket-1"); + addToProcessedEntities(inFlight, "dbA@cluster", "ticket-2"); + + Set qualifiedNames = Collections.singleton("tableA@cluster"); + Set referencedSet = Collections.singleton("dbA@cluster"); + + Set result = referenceFinder.find(inFlight, qualifiedNames, referencedSet); + + assertEquals(result.size(), 2); + assertTrue(result.contains("ticket-1")); + assertTrue(result.contains("ticket-2")); + } + + @Test + public void testFind_withPartialOverlap_returnsOnlyMatchingKeys() { + // three entities in flight, current ticket overlaps with only one + Map> inFlight = new ConcurrentHashMap<>(); + addToProcessedEntities(inFlight, "tableA@cluster", "ticket-1"); + addToProcessedEntities(inFlight, "tableB@cluster", "ticket-2"); + addToProcessedEntities(inFlight, "tableC@cluster", "ticket-3"); + + Set result = referenceFinder.find( + inFlight, + Collections.singleton("tableB@cluster"), + Collections.emptySet()); + + assertEquals(result.size(), 1); + assertTrue(result.contains("ticket-2")); + } + + @Test + public void testFind_sameTicketKeyUnderMultipleEntities_returnedOnlyOnce() { + // ticket-1 is processing both tableA and tableB; current ticket overlaps with both + Map> inFlight = new ConcurrentHashMap<>(); + addToProcessedEntities(inFlight, "tableA@cluster", "ticket-1"); + addToProcessedEntities(inFlight, "tableB@cluster", "ticket-1"); + + Set qualifiedNames = new HashSet<>(); + qualifiedNames.add("tableA@cluster"); + qualifiedNames.add("tableB@cluster"); + + Set result = referenceFinder.find(inFlight, qualifiedNames, Collections.emptySet()); + + // ticket-1 should appear only once even though matched via two different entities + assertEquals(result.size(), 1); + assertTrue(result.contains("ticket-1")); + } +}