Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 9 additions & 3 deletions common/src/main/java/org/apache/atlas/pc/StatusReporter.java
Original file line number Diff line number Diff line change
Expand Up @@ -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<T, U> {
private static final Logger LOG = LoggerFactory.getLogger(StatusReporter.class);

private final Map<T, U> producedItems = new LinkedHashMap<>();
private final Set<T> processedSet = new HashSet<>();
private final Map<T, U> producedItems = Collections.synchronizedMap(new LinkedHashMap<>());
private final Set<T> processedSet = new ConcurrentSkipListSet<>();
private final long timeoutDuration;
private long lastAck;

Expand All @@ -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));
}
Expand Down
20 changes: 18 additions & 2 deletions common/src/main/java/org/apache/atlas/pc/WorkItemManager.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down Expand Up @@ -46,7 +46,7 @@ public class WorkItemManager<T, U extends WorkItemConsumer<T>> {
public WorkItemManager(WorkItemBuilder<U, T> 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);

Expand Down Expand Up @@ -147,4 +147,20 @@ private int getAvgCommitTimeSeconds() {

return (int) ((commitTimeSeconds / consumers.size()) / 1000);
}

protected <T> List<T> getTypedResults() {
if (getResults().isEmpty()) {
return null;
}

List<T> ret = new ArrayList<>();
Object result;
while (((result = getResults().poll())) != null) {
T res = (T) result;

ret.add(res);
}

return ret;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
}

Expand Down
11 changes: 6 additions & 5 deletions intg/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -109,11 +109,6 @@
</exclusions>
</dependency>

<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-all</artifactId>
</dependency>

<dependency>
<groupId>org.springframework</groupId>
<artifactId>spring-context</artifactId>
Expand All @@ -127,6 +122,12 @@
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-all</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.testng</groupId>
<artifactId>testng</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -46,6 +47,7 @@ public class AtlasKafkaConsumer<T> extends AbstractNotificationConsumer<T> {
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);
Expand All @@ -57,6 +59,7 @@ public AtlasKafkaConsumer(AtlasNotificationMessageDeserializer<T> deserializer,
this.autoCommitEnabled = autoCommitEnabled;
this.kafkaConsumer = kafkaConsumer;
this.pollTimeoutMilliSeconds = pollTimeoutMilliSeconds;
this.duration = Duration.ofMillis(pollTimeoutMilliSeconds);
}

@Override
Expand All @@ -72,7 +75,7 @@ public Set<String> 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)));
}
Expand All @@ -92,37 +95,42 @@ public void wakeup() {
}
}

@Override
public void poll() {
this.kafkaConsumer.poll(this.duration);
}

public List<AtlasKafkaMessage<T>> receive() {
return this.receive(this.pollTimeoutMilliSeconds);
}

@Override
public List<AtlasKafkaMessage<T>> receive(long timeoutMilliSeconds) {
return receive(this.pollTimeoutMilliSeconds, null);
return receive(this.duration, null);
}

@Override
public List<AtlasKafkaMessage<T>> receiveWithCheckedCommit(Map<TopicPartition, Long> lastCommittedPartitionOffset) {
return receive(this.pollTimeoutMilliSeconds, lastCommittedPartitionOffset);
return receive(this.duration, lastCommittedPartitionOffset);
}

@Override
public List<AtlasKafkaMessage<T>> receiveRawRecordsWithCheckedCommit(Map<TopicPartition, Long> lastCommittedPartitionOffset) {
return receiveRawRecords(this.pollTimeoutMilliSeconds, lastCommittedPartitionOffset);
return receiveRawRecords(this.duration, lastCommittedPartitionOffset);
}

private List<AtlasKafkaMessage<T>> receiveRawRecords(long timeoutMilliSeconds, Map<TopicPartition, Long> lastCommittedPartitionOffset) {
return receive(timeoutMilliSeconds, lastCommittedPartitionOffset, true);
private List<AtlasKafkaMessage<T>> receiveRawRecords(Duration duration, Map<TopicPartition, Long> lastCommittedPartitionOffset) {
return receive(duration, lastCommittedPartitionOffset, true);
}

private List<AtlasKafkaMessage<T>> receive(long timeoutMilliSeconds, Map<TopicPartition, Long> lastCommittedPartitionOffset) {
return receive(timeoutMilliSeconds, lastCommittedPartitionOffset, false);
private List<AtlasKafkaMessage<T>> receive(Duration duration, Map<TopicPartition, Long> lastCommittedPartitionOffset) {
return receive(duration, lastCommittedPartitionOffset, false);
}

private List<AtlasKafkaMessage<T>> receive(long timeoutMilliSeconds, Map<TopicPartition, Long> lastCommittedPartitionOffset, boolean isRawDataRequired) {
private List<AtlasKafkaMessage<T>> receive(Duration duration, Map<TopicPartition, Long> lastCommittedPartitionOffset, boolean isRawDataRequired) {
List<AtlasKafkaMessage<T>> 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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,8 @@ public interface NotificationConsumer<T> {
*/
List<AtlasKafkaMessage<T>> receiveRawRecordsWithCheckedCommit(Map<TopicPartition, Long> lastCommittedPartitionOffset);

void poll();

Set<TopicPartition> getTopicPartition();

Set<String> subscription();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down Expand Up @@ -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<HookNotification> consumer = new AtlasKafkaConsumer<>(NotificationType.HOOK, kafkaConsumer, false, 100L);

Expand Down Expand Up @@ -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<HookNotification> consumer = new AtlasKafkaConsumer<>(NotificationType.HOOK, kafkaConsumer, false, 100L);

Expand All @@ -166,7 +167,7 @@ private List<AtlasKafkaMessage<HookNotification>> testReceiveHelper(EntityUpdate
List<ConsumerRecord<String, String>> klist = Collections.singletonList(new ConsumerRecord<>(topic, 0, 0L, "mykey", json));
ConsumerRecords<String, String> 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));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -221,6 +221,11 @@ public List<AtlasKafkaMessage<TestMessage>> receiveWithCheckedCommit(Map<TopicPa
public List<AtlasKafkaMessage<TestMessage>> receiveRawRecordsWithCheckedCommit(Map<TopicPartition, Long> lastCommittedPartitionOffset) {
return null;
}

@Override
public void poll() {
// Do nothing
}
}

public static class TestMessageDeserializer extends AbstractMessageDeserializer<TestMessage> {
Expand Down
5 changes: 5 additions & 0 deletions repository/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,11 @@
<artifactId>commons-lang3</artifactId>
</dependency>

<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
</dependency>

<dependency>
<groupId>org.apache.poi</groupId>
<artifactId>poi</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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<String, Authentication> 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<HookNotification> 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!");
}
}
}
Loading
Loading