mirror of https://github.com/apache/nifi.git
NIFI-10381 Refactored Azure Event Hubs components with current SDK
- Upgraded Azure Event Hubs components from legacy version 3 to current version 5 - Refactored Azure dependencies using azure-sdk-bom - Merged nifi-azure-record-sink-nar with nifi-azure-nar - Refactored PutAzureCosmosDBRecordTest to use Jackson instead of Gson This closes #6319. Signed-off-by: Peter Turcsanyi <turcsanyi@apache.org>
This commit is contained in:
parent
fdd94009b3
commit
6461061e8f
|
@ -514,12 +514,6 @@ language governing permissions and limitations under the License. -->
|
|||
<version>1.18.0-SNAPSHOT</version>
|
||||
<type>nar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-azure-record-sink-nar</artifactId>
|
||||
<version>1.18.0-SNAPSHOT</version>
|
||||
<type>nar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-azure-services-api-nar</artifactId>
|
||||
|
|
|
@ -20,11 +20,7 @@
|
|||
<artifactId>nifi-azure-processors</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
<properties>
|
||||
<azure-eventhubs.version>3.3.0</azure-eventhubs.version>
|
||||
<azure-eventhubs-eph.version>3.3.0</azure-eventhubs-eph.version>
|
||||
<azure-keyvault.version>1.2.6</azure-keyvault.version>
|
||||
<azure-storage-file-datalake.version>12.7.4</azure-storage-file-datalake.version>
|
||||
<azure-storage-blob.version>12.14.4</azure-storage-blob.version>
|
||||
</properties>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
|
@ -45,6 +41,10 @@
|
|||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-record-serialization-service-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-record-sink-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-record</artifactId>
|
||||
|
@ -62,74 +62,44 @@
|
|||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-core</artifactId>
|
||||
<version>${azure.core.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-identity</artifactId>
|
||||
<version>${azure.identity.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-core</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.microsoft.azure</groupId>
|
||||
<artifactId>azure-eventhubs</artifactId>
|
||||
<version>${azure-eventhubs.version}</version>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-messaging-eventhubs</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-messaging-eventhubs-checkpointstore-blob</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-cosmos</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-storage-file-datalake</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-storage-blob</artifactId>
|
||||
</dependency>
|
||||
<!-- Legacy Microsoft Azure Libraries -->
|
||||
<dependency>
|
||||
<groupId>com.microsoft.azure</groupId>
|
||||
<artifactId>azure-keyvault</artifactId>
|
||||
<version>${azure-keyvault.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.microsoft.azure</groupId>
|
||||
<artifactId>azure-eventhubs-eph</artifactId>
|
||||
<version>${azure-eventhubs-eph.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.microsoft.azure</groupId>
|
||||
<artifactId>azure-storage</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-cosmos</artifactId>
|
||||
<version>${azure-cosmos.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-core</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-storage-file-datalake</artifactId>
|
||||
<version>${azure-storage-file-datalake.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-core</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-storage-blob</artifactId>
|
||||
<version>${azure-storage-blob.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-core</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-io</groupId>
|
||||
|
@ -180,7 +150,7 @@
|
|||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-inline</artifactId>
|
||||
<version>3.3.3</version>
|
||||
<version>${mockito.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
|
|
|
@ -16,14 +16,20 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.azure.eventhub;
|
||||
|
||||
import com.microsoft.azure.eventhubs.EventData;
|
||||
import com.microsoft.azure.eventhubs.ReceiverDisconnectedException;
|
||||
import com.microsoft.azure.eventprocessorhost.CloseReason;
|
||||
import com.microsoft.azure.eventprocessorhost.EventProcessorHost;
|
||||
import com.microsoft.azure.eventprocessorhost.EventProcessorOptions;
|
||||
import com.microsoft.azure.eventprocessorhost.IEventProcessor;
|
||||
import com.microsoft.azure.eventprocessorhost.IEventProcessorFactory;
|
||||
import com.microsoft.azure.eventprocessorhost.PartitionContext;
|
||||
import com.azure.core.credential.AzureNamedKeyCredential;
|
||||
import com.azure.identity.ManagedIdentityCredential;
|
||||
import com.azure.identity.ManagedIdentityCredentialBuilder;
|
||||
import com.azure.messaging.eventhubs.EventData;
|
||||
import com.azure.messaging.eventhubs.EventProcessorClient;
|
||||
import com.azure.messaging.eventhubs.EventProcessorClientBuilder;
|
||||
import com.azure.messaging.eventhubs.checkpointstore.blob.BlobCheckpointStore;
|
||||
import com.azure.messaging.eventhubs.models.ErrorContext;
|
||||
import com.azure.messaging.eventhubs.models.EventBatchContext;
|
||||
import com.azure.messaging.eventhubs.models.EventPosition;
|
||||
import com.azure.messaging.eventhubs.models.LastEnqueuedEventProperties;
|
||||
import com.azure.messaging.eventhubs.models.PartitionContext;
|
||||
import com.azure.storage.blob.BlobContainerAsyncClient;
|
||||
import com.azure.storage.blob.BlobContainerClientBuilder;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.TriggerSerially;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
|
@ -45,10 +51,10 @@ import org.apache.nifi.processor.ProcessContext;
|
|||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.azure.eventhub.position.EarliestEventPositionProvider;
|
||||
import org.apache.nifi.processors.azure.eventhub.position.LegacyBlobStorageEventPositionProvider;
|
||||
import org.apache.nifi.processors.azure.eventhub.utils.AzureEventHubUtils;
|
||||
import org.apache.nifi.schema.access.SchemaNotFoundException;
|
||||
import org.apache.nifi.serialization.RecordReader;
|
||||
import org.apache.nifi.serialization.RecordReaderFactory;
|
||||
import org.apache.nifi.serialization.RecordSetWriter;
|
||||
|
@ -74,9 +80,10 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import static org.apache.nifi.util.StringUtils.isEmpty;
|
||||
import static org.apache.commons.lang3.StringUtils.defaultIfBlank;
|
||||
|
||||
@Tags({"azure", "microsoft", "cloud", "eventhub", "events", "streaming", "streams"})
|
||||
@CapabilityDescription("Receives messages from Azure Event Hubs, writing the contents of the message to the content of the FlowFile.")
|
||||
|
@ -138,8 +145,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
|
|||
static final PropertyDescriptor CONSUMER_HOSTNAME = new PropertyDescriptor.Builder()
|
||||
.name("event-hub-consumer-hostname")
|
||||
.displayName("Consumer Hostname")
|
||||
.description("The hostname of this event hub consumer instance." +
|
||||
" If not specified, an unique identifier is generated in 'nifi-<UUID>' format.")
|
||||
.description("DEPRECATED: This property is no longer used.")
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
|
||||
.required(false)
|
||||
|
@ -271,10 +277,24 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
|
|||
|
||||
static {
|
||||
PROPERTIES = Collections.unmodifiableList(Arrays.asList(
|
||||
NAMESPACE, EVENT_HUB_NAME, SERVICE_BUS_ENDPOINT, ACCESS_POLICY_NAME, POLICY_PRIMARY_KEY, USE_MANAGED_IDENTITY, CONSUMER_GROUP, CONSUMER_HOSTNAME,
|
||||
RECORD_READER, RECORD_WRITER,
|
||||
INITIAL_OFFSET, PREFETCH_COUNT, BATCH_SIZE, RECEIVE_TIMEOUT,
|
||||
STORAGE_ACCOUNT_NAME, STORAGE_ACCOUNT_KEY, STORAGE_SAS_TOKEN, STORAGE_CONTAINER_NAME
|
||||
NAMESPACE,
|
||||
EVENT_HUB_NAME,
|
||||
SERVICE_BUS_ENDPOINT,
|
||||
ACCESS_POLICY_NAME,
|
||||
POLICY_PRIMARY_KEY,
|
||||
USE_MANAGED_IDENTITY,
|
||||
CONSUMER_GROUP,
|
||||
CONSUMER_HOSTNAME,
|
||||
RECORD_READER,
|
||||
RECORD_WRITER,
|
||||
INITIAL_OFFSET,
|
||||
PREFETCH_COUNT,
|
||||
BATCH_SIZE,
|
||||
RECEIVE_TIMEOUT,
|
||||
STORAGE_ACCOUNT_NAME,
|
||||
STORAGE_ACCOUNT_KEY,
|
||||
STORAGE_SAS_TOKEN,
|
||||
STORAGE_CONTAINER_NAME
|
||||
));
|
||||
|
||||
Set<Relationship> relationships = new HashSet<>();
|
||||
|
@ -285,51 +305,16 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
|
|||
RECORD_RELATIONSHIPS = Collections.unmodifiableSet(relationships);
|
||||
}
|
||||
|
||||
private volatile EventProcessorHost eventProcessorHost;
|
||||
private volatile ProcessSessionFactory processSessionFactory;
|
||||
private volatile EventProcessorClient eventProcessorClient;
|
||||
private volatile RecordReaderFactory readerFactory;
|
||||
private volatile RecordSetWriterFactory writerFactory;
|
||||
// The namespace name can not be retrieved from a PartitionContext at EventProcessor.onEvents, so keep it here.
|
||||
|
||||
private volatile String namespaceName;
|
||||
private volatile boolean isRecordReaderSet = false;
|
||||
private volatile boolean isRecordWriterSet = false;
|
||||
private volatile String serviceBusEndpoint;
|
||||
|
||||
/**
|
||||
* For unit test.
|
||||
*/
|
||||
void setProcessSessionFactory(ProcessSessionFactory processSessionFactory) {
|
||||
this.processSessionFactory = processSessionFactory;
|
||||
}
|
||||
|
||||
/**
|
||||
* For unit test.
|
||||
*/
|
||||
void setNamespaceName(String namespaceName) {
|
||||
this.namespaceName = namespaceName;
|
||||
}
|
||||
|
||||
/**
|
||||
* For unit test.
|
||||
*/
|
||||
public void setReaderFactory(RecordReaderFactory readerFactory) {
|
||||
this.readerFactory = readerFactory;
|
||||
}
|
||||
|
||||
/**
|
||||
* For unit test.
|
||||
*/
|
||||
public void setWriterFactory(RecordSetWriterFactory writerFactory) {
|
||||
this.writerFactory = writerFactory;
|
||||
}
|
||||
|
||||
/**
|
||||
* For unit test.
|
||||
*/
|
||||
public void setServiceBusEndpoint(String serviceBusEndpoint) {
|
||||
this.serviceBusEndpoint = serviceBusEndpoint;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return PROPERTIES;
|
||||
|
@ -381,216 +366,23 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
|
||||
public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
|
||||
if (RECORD_READER.equals(descriptor)) {
|
||||
isRecordReaderSet = !StringUtils.isEmpty(newValue);
|
||||
isRecordReaderSet = StringUtils.isNotEmpty(newValue);
|
||||
} else if (RECORD_WRITER.equals(descriptor)) {
|
||||
isRecordWriterSet = !StringUtils.isEmpty(newValue);
|
||||
}
|
||||
}
|
||||
|
||||
public class EventProcessorFactory implements IEventProcessorFactory<EventProcessor> {
|
||||
@Override
|
||||
public EventProcessor createEventProcessor(PartitionContext context) throws Exception {
|
||||
final EventProcessor eventProcessor = new EventProcessor();
|
||||
return eventProcessor;
|
||||
}
|
||||
}
|
||||
|
||||
public class EventProcessor implements IEventProcessor {
|
||||
|
||||
@Override
|
||||
public void onOpen(PartitionContext context) throws Exception {
|
||||
getLogger().info("Consumer group {} opened partition {} of {}",
|
||||
new Object[]{context.getConsumerGroupName(), context.getPartitionId(), context.getEventHubPath()});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onClose(PartitionContext context, CloseReason reason) throws Exception {
|
||||
getLogger().info("Consumer group {} closed partition {} of {}. reason={}",
|
||||
new Object[]{context.getConsumerGroupName(), context.getPartitionId(), context.getEventHubPath(), reason});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onEvents(PartitionContext context, Iterable<EventData> messages) throws Exception {
|
||||
final ProcessSession session = processSessionFactory.createSession();
|
||||
|
||||
try {
|
||||
|
||||
final StopWatch stopWatch = new StopWatch(true);
|
||||
|
||||
if (readerFactory != null && writerFactory != null) {
|
||||
writeRecords(context, messages, session, stopWatch);
|
||||
} else {
|
||||
writeFlowFiles(context, messages, session, stopWatch);
|
||||
}
|
||||
|
||||
// Commit NiFi first.
|
||||
// If creating an Event Hub checkpoint failed, then the same message can be retrieved again.
|
||||
session.commitAsync(context::checkpoint);
|
||||
} catch (Exception e) {
|
||||
getLogger().error("Unable to fully process received message due to " + e, e);
|
||||
// FlowFiles those are already committed will not get rollback.
|
||||
session.rollback();
|
||||
}
|
||||
}
|
||||
|
||||
private void putEventHubAttributes(Map<String, String> attributes, String eventHubName, String partitionId, EventData eventData) {
|
||||
final EventData.SystemProperties systemProperties = eventData.getSystemProperties();
|
||||
if (null != systemProperties) {
|
||||
attributes.put("eventhub.enqueued.timestamp", String.valueOf(systemProperties.getEnqueuedTime()));
|
||||
attributes.put("eventhub.offset", systemProperties.getOffset());
|
||||
attributes.put("eventhub.sequence", String.valueOf(systemProperties.getSequenceNumber()));
|
||||
}
|
||||
|
||||
final Map<String,String> applicationProperties = AzureEventHubUtils.getApplicationProperties(eventData);
|
||||
attributes.putAll(applicationProperties);
|
||||
|
||||
attributes.put("eventhub.name", eventHubName);
|
||||
attributes.put("eventhub.partition", partitionId);
|
||||
}
|
||||
|
||||
private void writeFlowFiles(PartitionContext context, Iterable<EventData> messages, ProcessSession session, StopWatch stopWatch) {
|
||||
final String eventHubName = context.getEventHubPath();
|
||||
final String partitionId = context.getPartitionId();
|
||||
final String consumerGroup = context.getConsumerGroupName();
|
||||
messages.forEach(eventData -> {
|
||||
FlowFile flowFile = session.create();
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
putEventHubAttributes(attributes, eventHubName, partitionId, eventData);
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
|
||||
flowFile = session.write(flowFile, out -> {
|
||||
out.write(eventData.getBytes());
|
||||
});
|
||||
|
||||
transferTo(REL_SUCCESS, session, stopWatch, eventHubName, partitionId, consumerGroup, flowFile);
|
||||
});
|
||||
}
|
||||
|
||||
private void transferTo(Relationship relationship, ProcessSession session, StopWatch stopWatch,
|
||||
String eventHubName, String partitionId, String consumerGroup, FlowFile flowFile) {
|
||||
session.transfer(flowFile, relationship);
|
||||
final String transitUri = String.format("amqps://%s%s/%s/ConsumerGroups/%s/Partitions/%s",
|
||||
namespaceName, serviceBusEndpoint, eventHubName, consumerGroup, partitionId);
|
||||
session.getProvenanceReporter().receive(flowFile, transitUri, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
||||
}
|
||||
|
||||
private void writeRecords(PartitionContext context, Iterable<EventData> messages, ProcessSession session, StopWatch stopWatch)
|
||||
throws SchemaNotFoundException, IOException {
|
||||
|
||||
final String eventHubName = context.getEventHubPath();
|
||||
final String partitionId = context.getPartitionId();
|
||||
final String consumerGroup = context.getConsumerGroupName();
|
||||
final Map<String, String> schemaRetrievalVariables = new HashMap<>();
|
||||
schemaRetrievalVariables.put("eventhub.name", eventHubName);
|
||||
|
||||
final ComponentLog logger = getLogger();
|
||||
FlowFile flowFile = session.create();
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
||||
RecordSetWriter writer = null;
|
||||
EventData lastEventData = null;
|
||||
WriteResult lastWriteResult = null;
|
||||
int recordCount = 0;
|
||||
|
||||
try (final OutputStream out = session.write(flowFile)) {
|
||||
for (final EventData eventData : messages) {
|
||||
final byte[] eventDataBytes = eventData.getBytes();
|
||||
try (final InputStream in = new ByteArrayInputStream(eventDataBytes)) {
|
||||
final RecordReader reader = readerFactory.createRecordReader(schemaRetrievalVariables, in, eventDataBytes.length, logger);
|
||||
|
||||
Record record;
|
||||
while ((record = reader.nextRecord()) != null) {
|
||||
|
||||
if (writer == null) {
|
||||
// Initialize the writer when the first record is read.
|
||||
final RecordSchema readerSchema = record.getSchema();
|
||||
final RecordSchema writeSchema = writerFactory.getSchema(schemaRetrievalVariables, readerSchema);
|
||||
writer = writerFactory.createWriter(logger, writeSchema, out, flowFile);
|
||||
writer.beginRecordSet();
|
||||
}
|
||||
|
||||
lastWriteResult = writer.write(record);
|
||||
recordCount += lastWriteResult.getRecordCount();
|
||||
}
|
||||
|
||||
lastEventData = eventData;
|
||||
|
||||
} catch (Exception e) {
|
||||
// Write it to the parse failure relationship.
|
||||
logger.error("Failed to parse message from Azure Event Hub using configured Record Reader and Writer due to " + e, e);
|
||||
FlowFile failed = session.create();
|
||||
session.write(failed, o -> o.write(eventData.getBytes()));
|
||||
putEventHubAttributes(attributes, eventHubName, partitionId, eventData);
|
||||
failed = session.putAllAttributes(failed, attributes);
|
||||
transferTo(REL_PARSE_FAILURE, session, stopWatch, eventHubName, partitionId, consumerGroup, failed);
|
||||
}
|
||||
}
|
||||
|
||||
if (lastEventData != null) {
|
||||
putEventHubAttributes(attributes, eventHubName, partitionId, lastEventData);
|
||||
|
||||
attributes.put("record.count", String.valueOf(recordCount));
|
||||
if (writer != null) {
|
||||
writer.finishRecordSet();
|
||||
attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
|
||||
if (lastWriteResult != null) {
|
||||
attributes.putAll(lastWriteResult.getAttributes());
|
||||
}
|
||||
|
||||
try {
|
||||
writer.close();
|
||||
} catch (IOException e) {
|
||||
logger.warn("Failed to close Record Writer due to {}" + e, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// This part has to be outside of 'session.write(flowFile)' code block.
|
||||
if (lastEventData != null) {
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
transferTo(REL_SUCCESS, session, stopWatch, eventHubName, partitionId, consumerGroup, flowFile);
|
||||
} else {
|
||||
// If there's no successful event data, then remove the FlowFile.
|
||||
session.remove(flowFile);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onError(PartitionContext context, Throwable e) {
|
||||
if (e instanceof ReceiverDisconnectedException && e.getMessage().startsWith("New receiver with higher epoch of ")) {
|
||||
// This is a known behavior in a NiFi cluster where multiple nodes consumes from the same Event Hub.
|
||||
// Once another node connects, some partitions are given to that node to distribute consumer load.
|
||||
// When that happens, this exception is thrown.
|
||||
getLogger().info("New receiver took over partition {} of Azure Event Hub {}, consumerGroupName={}, message={}",
|
||||
new Object[]{context.getPartitionId(), context.getEventHubPath(), context.getConsumerGroupName(), e.getMessage()});
|
||||
return;
|
||||
}
|
||||
getLogger().error("An error occurred while receiving messages from Azure Event Hub {} at partition {}," +
|
||||
" consumerGroupName={}, exception={}",
|
||||
new Object[]{context.getEventHubPath(), context.getPartitionId(), context.getConsumerGroupName(), e}, e);
|
||||
isRecordWriterSet = StringUtils.isNotEmpty(newValue);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException {
|
||||
|
||||
if (eventProcessorHost == null) {
|
||||
try {
|
||||
registerEventProcessor(context);
|
||||
} catch (IllegalArgumentException e) {
|
||||
// In order to show simple error message without wrapping it by another ProcessException, just throw it as it is.
|
||||
throw e;
|
||||
} catch (Exception e) {
|
||||
throw new ProcessException("Failed to register the event processor due to " + e, e);
|
||||
}
|
||||
public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
|
||||
if (eventProcessorClient == null) {
|
||||
processSessionFactory = sessionFactory;
|
||||
|
||||
readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
|
||||
writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
|
||||
|
||||
eventProcessorClient = createClient(context);
|
||||
eventProcessorClient.start();
|
||||
}
|
||||
|
||||
// After a EventProcessor is registered successfully, nothing has to be done at onTrigger
|
||||
|
@ -599,95 +391,263 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
|
|||
}
|
||||
|
||||
@OnStopped
|
||||
public void unregisterEventProcessor(final ProcessContext context) {
|
||||
if (eventProcessorHost != null) {
|
||||
public void stopClient() {
|
||||
if (eventProcessorClient != null) {
|
||||
try {
|
||||
eventProcessorHost.unregisterEventProcessor();
|
||||
eventProcessorHost = null;
|
||||
processSessionFactory = null;
|
||||
readerFactory = null;
|
||||
writerFactory = null;
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException("Failed to unregister the event processor due to " + e, e);
|
||||
eventProcessorClient.stop();
|
||||
} catch (final Exception e) {
|
||||
getLogger().warn("Event Processor Client stop failed", e);
|
||||
}
|
||||
eventProcessorClient = null;
|
||||
processSessionFactory = null;
|
||||
readerFactory = null;
|
||||
writerFactory = null;
|
||||
}
|
||||
}
|
||||
|
||||
private void registerEventProcessor(final ProcessContext context) throws Exception {
|
||||
// Validate required properties.
|
||||
final String consumerGroupName = context.getProperty(CONSUMER_GROUP).evaluateAttributeExpressions().getValue();
|
||||
validateRequiredProperty(CONSUMER_GROUP, consumerGroupName);
|
||||
|
||||
protected EventProcessorClient createClient(final ProcessContext context) {
|
||||
namespaceName = context.getProperty(NAMESPACE).evaluateAttributeExpressions().getValue();
|
||||
validateRequiredProperty(NAMESPACE, namespaceName);
|
||||
|
||||
final String eventHubName = context.getProperty(EVENT_HUB_NAME).evaluateAttributeExpressions().getValue();
|
||||
validateRequiredProperty(EVENT_HUB_NAME, eventHubName);
|
||||
final String consumerGroup = context.getProperty(CONSUMER_GROUP).evaluateAttributeExpressions().getValue();
|
||||
|
||||
final String containerName = defaultIfBlank(context.getProperty(STORAGE_CONTAINER_NAME).evaluateAttributeExpressions().getValue(), eventHubName);
|
||||
final String storageConnectionString = createStorageConnectionString(context);
|
||||
final BlobContainerAsyncClient blobContainerAsyncClient = new BlobContainerClientBuilder()
|
||||
.connectionString(storageConnectionString)
|
||||
.containerName(containerName)
|
||||
.buildAsyncClient();
|
||||
final BlobCheckpointStore checkpointStore = new BlobCheckpointStore(blobContainerAsyncClient);
|
||||
|
||||
final String consumerHostname = orDefault(context.getProperty(CONSUMER_HOSTNAME).evaluateAttributeExpressions().getValue(),
|
||||
EventProcessorHost.createHostName("nifi"));
|
||||
final Long receiveTimeout = context.getProperty(RECEIVE_TIMEOUT).evaluateAttributeExpressions().asTimePeriod(TimeUnit.MILLISECONDS);
|
||||
final Duration maxWaitTime = Duration.ofMillis(receiveTimeout);
|
||||
final Integer maxBatchSize = context.getProperty(BATCH_SIZE).evaluateAttributeExpressions().asInteger();
|
||||
|
||||
final String containerName = orDefault(context.getProperty(STORAGE_CONTAINER_NAME).evaluateAttributeExpressions().getValue(),
|
||||
eventHubName);
|
||||
final EventProcessorClientBuilder eventProcessorClientBuilder = new EventProcessorClientBuilder()
|
||||
.consumerGroup(consumerGroup)
|
||||
.trackLastEnqueuedEventProperties(true)
|
||||
.checkpointStore(checkpointStore)
|
||||
.processError(errorProcessor)
|
||||
.processEventBatch(eventBatchProcessor, maxBatchSize, maxWaitTime);
|
||||
|
||||
|
||||
final EventProcessorOptions options = new EventProcessorOptions();
|
||||
final String initialOffset = context.getProperty(INITIAL_OFFSET).getValue();
|
||||
if (INITIAL_OFFSET_START_OF_STREAM.getValue().equals(initialOffset)) {
|
||||
options.setInitialPositionProvider(options.new StartOfStreamInitialPositionProvider());
|
||||
} else if (INITIAL_OFFSET_END_OF_STREAM.getValue().equals(initialOffset)){
|
||||
options.setInitialPositionProvider(options.new EndOfStreamInitialPositionProvider());
|
||||
final String fullyQualifiedNamespace = String.format("%s%s", namespaceName, serviceBusEndpoint);
|
||||
final boolean useManagedIdentity = context.getProperty(USE_MANAGED_IDENTITY).asBoolean();
|
||||
if (useManagedIdentity) {
|
||||
final ManagedIdentityCredentialBuilder managedIdentityCredentialBuilder = new ManagedIdentityCredentialBuilder();
|
||||
final ManagedIdentityCredential managedIdentityCredential = managedIdentityCredentialBuilder.build();
|
||||
eventProcessorClientBuilder.credential(fullyQualifiedNamespace, eventHubName, managedIdentityCredential);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Initial offset " + initialOffset + " is not allowed.");
|
||||
final String policyName = context.getProperty(ACCESS_POLICY_NAME).evaluateAttributeExpressions().getValue();
|
||||
final String policyKey = context.getProperty(POLICY_PRIMARY_KEY).evaluateAttributeExpressions().getValue();
|
||||
final AzureNamedKeyCredential azureNamedKeyCredential = new AzureNamedKeyCredential(policyName, policyKey);
|
||||
eventProcessorClientBuilder.credential(fullyQualifiedNamespace, eventHubName, azureNamedKeyCredential);
|
||||
}
|
||||
|
||||
final Integer prefetchCount = context.getProperty(PREFETCH_COUNT).evaluateAttributeExpressions().asInteger();
|
||||
if (prefetchCount != null && prefetchCount > 0) {
|
||||
options.setPrefetchCount(prefetchCount);
|
||||
eventProcessorClientBuilder.prefetchCount(prefetchCount);
|
||||
}
|
||||
|
||||
final Integer batchSize = context.getProperty(BATCH_SIZE).evaluateAttributeExpressions().asInteger();
|
||||
if (batchSize != null && batchSize > 0) {
|
||||
options.setMaxBatchSize(batchSize);
|
||||
}
|
||||
|
||||
final Long receiveTimeoutMillis = context.getProperty(RECEIVE_TIMEOUT)
|
||||
.evaluateAttributeExpressions().asTimePeriod(TimeUnit.MILLISECONDS);
|
||||
options.setReceiveTimeOut(Duration.ofMillis(receiveTimeoutMillis));
|
||||
|
||||
final String storageConnectionString = createStorageConnectionString(context);
|
||||
|
||||
final String connectionString;
|
||||
final boolean useManagedIdentity = context.getProperty(USE_MANAGED_IDENTITY).asBoolean();
|
||||
if(useManagedIdentity) {
|
||||
connectionString = AzureEventHubUtils.getManagedIdentityConnectionString(namespaceName, serviceBusEndpoint, eventHubName);
|
||||
final Map<String, EventPosition> legacyPartitionEventPosition = getLegacyPartitionEventPosition(blobContainerAsyncClient, consumerGroup);
|
||||
if (legacyPartitionEventPosition.isEmpty()) {
|
||||
final String initialOffset = context.getProperty(INITIAL_OFFSET).getValue();
|
||||
// EventPosition.latest() is the default behavior is absence of existing checkpoints
|
||||
if (INITIAL_OFFSET_START_OF_STREAM.getValue().equals(initialOffset)) {
|
||||
final EarliestEventPositionProvider eventPositionProvider = new EarliestEventPositionProvider();
|
||||
final Map<String, EventPosition> partitionEventPosition = eventPositionProvider.getInitialPartitionEventPosition();
|
||||
eventProcessorClientBuilder.initialPartitionEventPosition(partitionEventPosition);
|
||||
}
|
||||
} else {
|
||||
final String sasName = context.getProperty(ACCESS_POLICY_NAME).evaluateAttributeExpressions().getValue();
|
||||
validateRequiredProperty(ACCESS_POLICY_NAME, sasName);
|
||||
final String sasKey = context.getProperty(POLICY_PRIMARY_KEY).evaluateAttributeExpressions().getValue();
|
||||
validateRequiredProperty(POLICY_PRIMARY_KEY, sasKey);
|
||||
connectionString = AzureEventHubUtils.getSharedAccessSignatureConnectionString(namespaceName, serviceBusEndpoint, eventHubName, sasName, sasKey);
|
||||
eventProcessorClientBuilder.initialPartitionEventPosition(legacyPartitionEventPosition);
|
||||
}
|
||||
|
||||
eventProcessorHost = EventProcessorHost.EventProcessorHostBuilder
|
||||
.newBuilder(consumerHostname, consumerGroupName)
|
||||
.useAzureStorageCheckpointLeaseManager(storageConnectionString, containerName, null)
|
||||
.useEventHubConnectionString(connectionString, eventHubName)
|
||||
.build();
|
||||
return eventProcessorClientBuilder.buildEventProcessorClient();
|
||||
}
|
||||
|
||||
options.setExceptionNotification(e -> {
|
||||
getLogger().error("An error occurred while receiving messages from Azure Event Hub {}" +
|
||||
" at consumer group {} and partition {}, action={}, hostname={}, exception={}",
|
||||
new Object[]{eventHubName, consumerGroupName, e.getPartitionId(), e.getAction(), e.getHostname()}, e.getException());
|
||||
protected String getTransitUri(final PartitionContext partitionContext) {
|
||||
return String.format("amqps://%s%s/%s/ConsumerGroups/%s/Partitions/%s",
|
||||
namespaceName,
|
||||
serviceBusEndpoint,
|
||||
partitionContext.getEventHubName(),
|
||||
partitionContext.getConsumerGroup(),
|
||||
partitionContext.getPartitionId()
|
||||
);
|
||||
}
|
||||
|
||||
protected final Consumer<EventBatchContext> eventBatchProcessor = eventBatchContext -> {
|
||||
final ProcessSession session = processSessionFactory.createSession();
|
||||
|
||||
try {
|
||||
final StopWatch stopWatch = new StopWatch(true);
|
||||
|
||||
if (readerFactory == null || writerFactory == null) {
|
||||
writeFlowFiles(eventBatchContext, session, stopWatch);
|
||||
} else {
|
||||
writeRecords(eventBatchContext, session, stopWatch);
|
||||
}
|
||||
|
||||
// Commit ProcessSession and then update Azure Event Hubs checkpoint status
|
||||
session.commitAsync(eventBatchContext::updateCheckpoint);
|
||||
} catch (final Exception e) {
|
||||
final PartitionContext partitionContext = eventBatchContext.getPartitionContext();
|
||||
getLogger().error("Event Batch processing failed Namespace [{}] Event Hub [{}] Consumer Group [{}] Partition [{}]",
|
||||
partitionContext.getFullyQualifiedNamespace(),
|
||||
partitionContext.getEventHubName(),
|
||||
partitionContext.getConsumerGroup(),
|
||||
partitionContext.getPartitionId(),
|
||||
e
|
||||
);
|
||||
session.rollback();
|
||||
}
|
||||
};
|
||||
|
||||
private final Consumer<ErrorContext> errorProcessor = errorContext -> {
|
||||
final PartitionContext partitionContext = errorContext.getPartitionContext();
|
||||
getLogger().error("Receive Events failed Namespace [{}] Event Hub [{}] Consumer Group [{}] Partition [{}]",
|
||||
partitionContext.getFullyQualifiedNamespace(),
|
||||
partitionContext.getEventHubName(),
|
||||
partitionContext.getConsumerGroup(),
|
||||
partitionContext.getPartitionId(),
|
||||
errorContext.getThrowable()
|
||||
);
|
||||
};
|
||||
|
||||
private void putEventHubAttributes(
|
||||
final Map<String, String> attributes,
|
||||
final PartitionContext partitionContext,
|
||||
final EventData eventData,
|
||||
final LastEnqueuedEventProperties lastEnqueuedEventProperties
|
||||
) {
|
||||
if (lastEnqueuedEventProperties != null) {
|
||||
attributes.put("eventhub.enqueued.timestamp", String.valueOf(lastEnqueuedEventProperties.getEnqueuedTime()));
|
||||
attributes.put("eventhub.offset", String.valueOf(lastEnqueuedEventProperties.getOffset()));
|
||||
attributes.put("eventhub.sequence", String.valueOf(lastEnqueuedEventProperties.getSequenceNumber()));
|
||||
}
|
||||
|
||||
final Map<String, String> applicationProperties = AzureEventHubUtils.getApplicationProperties(eventData.getProperties());
|
||||
attributes.putAll(applicationProperties);
|
||||
|
||||
attributes.put("eventhub.name", partitionContext.getEventHubName());
|
||||
attributes.put("eventhub.partition", partitionContext.getPartitionId());
|
||||
}
|
||||
|
||||
private void writeFlowFiles(
|
||||
final EventBatchContext eventBatchContext,
|
||||
final ProcessSession session,
|
||||
final StopWatch stopWatch
|
||||
) {
|
||||
final PartitionContext partitionContext = eventBatchContext.getPartitionContext();
|
||||
final List<EventData> events = eventBatchContext.getEvents();
|
||||
events.forEach(eventData -> {
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
putEventHubAttributes(attributes, partitionContext, eventData, eventBatchContext.getLastEnqueuedEventProperties());
|
||||
|
||||
FlowFile flowFile = session.create();
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
|
||||
final byte[] body = eventData.getBody();
|
||||
flowFile = session.write(flowFile, outputStream -> outputStream.write(body));
|
||||
|
||||
transferTo(REL_SUCCESS, session, stopWatch, partitionContext, flowFile);
|
||||
});
|
||||
}
|
||||
|
||||
eventProcessorHost.registerEventProcessorFactory(new EventProcessorFactory(), options).get();
|
||||
private void writeRecords(
|
||||
final EventBatchContext eventBatchContext,
|
||||
final ProcessSession session,
|
||||
final StopWatch stopWatch
|
||||
) throws IOException {
|
||||
final PartitionContext partitionContext = eventBatchContext.getPartitionContext();
|
||||
final Map<String, String> schemaRetrievalVariables = new HashMap<>();
|
||||
schemaRetrievalVariables.put("eventhub.name", partitionContext.getEventHubName());
|
||||
|
||||
final ComponentLog logger = getLogger();
|
||||
FlowFile flowFile = session.create();
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
||||
RecordSetWriter writer = null;
|
||||
EventData lastEventData = null;
|
||||
WriteResult lastWriteResult = null;
|
||||
int recordCount = 0;
|
||||
|
||||
final LastEnqueuedEventProperties lastEnqueuedEventProperties = eventBatchContext.getLastEnqueuedEventProperties();
|
||||
final List<EventData> events = eventBatchContext.getEvents();
|
||||
|
||||
try (final OutputStream out = session.write(flowFile)) {
|
||||
for (final EventData eventData : events) {
|
||||
final byte[] eventDataBytes = eventData.getBody();
|
||||
try (final InputStream in = new ByteArrayInputStream(eventDataBytes)) {
|
||||
final RecordReader reader = readerFactory.createRecordReader(schemaRetrievalVariables, in, eventDataBytes.length, logger);
|
||||
|
||||
Record record;
|
||||
while ((record = reader.nextRecord()) != null) {
|
||||
|
||||
if (writer == null) {
|
||||
// Initialize the writer when the first record is read.
|
||||
final RecordSchema readerSchema = record.getSchema();
|
||||
final RecordSchema writeSchema = writerFactory.getSchema(schemaRetrievalVariables, readerSchema);
|
||||
writer = writerFactory.createWriter(logger, writeSchema, out, flowFile);
|
||||
writer.beginRecordSet();
|
||||
}
|
||||
|
||||
lastWriteResult = writer.write(record);
|
||||
recordCount += lastWriteResult.getRecordCount();
|
||||
}
|
||||
|
||||
lastEventData = eventData;
|
||||
|
||||
} catch (Exception e) {
|
||||
// Write it to the parse failure relationship.
|
||||
logger.error("Failed to parse message from Azure Event Hub using configured Record Reader and Writer", e);
|
||||
FlowFile failed = session.create();
|
||||
session.write(failed, o -> o.write(eventData.getBody()));
|
||||
putEventHubAttributes(attributes, partitionContext, eventData, lastEnqueuedEventProperties);
|
||||
failed = session.putAllAttributes(failed, attributes);
|
||||
transferTo(REL_PARSE_FAILURE, session, stopWatch, partitionContext, failed);
|
||||
}
|
||||
}
|
||||
|
||||
if (lastEventData != null) {
|
||||
putEventHubAttributes(attributes, partitionContext, lastEventData, lastEnqueuedEventProperties);
|
||||
|
||||
attributes.put("record.count", String.valueOf(recordCount));
|
||||
if (writer != null) {
|
||||
writer.finishRecordSet();
|
||||
attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
|
||||
if (lastWriteResult != null) {
|
||||
attributes.putAll(lastWriteResult.getAttributes());
|
||||
}
|
||||
|
||||
try {
|
||||
writer.close();
|
||||
} catch (final IOException e) {
|
||||
logger.warn("Failed to close Record Writer", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (lastEventData == null) {
|
||||
session.remove(flowFile);
|
||||
} else {
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
transferTo(REL_SUCCESS, session, stopWatch, partitionContext, flowFile);
|
||||
}
|
||||
}
|
||||
|
||||
private void transferTo(
|
||||
final Relationship relationship,
|
||||
final ProcessSession session,
|
||||
final StopWatch stopWatch,
|
||||
final PartitionContext partitionContext,
|
||||
final FlowFile flowFile
|
||||
) {
|
||||
session.transfer(flowFile, relationship);
|
||||
final String transitUri = getTransitUri(partitionContext);
|
||||
session.getProvenanceReporter().receive(flowFile, transitUri, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
||||
}
|
||||
|
||||
private String createStorageConnectionString(final ProcessContext context) {
|
||||
final String storageAccountName = context.getProperty(STORAGE_ACCOUNT_NAME).evaluateAttributeExpressions().getValue();
|
||||
validateRequiredProperty(STORAGE_ACCOUNT_NAME, storageAccountName);
|
||||
|
||||
serviceBusEndpoint = context.getProperty(SERVICE_BUS_ENDPOINT).getValue();
|
||||
final String domainName = serviceBusEndpoint.replace(".servicebus.", "");
|
||||
|
@ -700,13 +660,22 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
|
|||
return String.format(FORMAT_STORAGE_CONNECTION_STRING_FOR_SAS_TOKEN, storageAccountName, domainName, storageSasToken);
|
||||
}
|
||||
|
||||
private String orDefault(String value, String defaultValue) {
|
||||
return isEmpty(value) ? defaultValue : value;
|
||||
}
|
||||
private Map<String, EventPosition> getLegacyPartitionEventPosition(
|
||||
final BlobContainerAsyncClient blobContainerAsyncClient,
|
||||
final String consumerGroup
|
||||
) {
|
||||
final LegacyBlobStorageEventPositionProvider legacyBlobStorageEventPositionProvider = new LegacyBlobStorageEventPositionProvider(
|
||||
blobContainerAsyncClient,
|
||||
consumerGroup
|
||||
);
|
||||
final Map<String, EventPosition> partitionEventPosition = legacyBlobStorageEventPositionProvider.getInitialPartitionEventPosition();
|
||||
|
||||
private void validateRequiredProperty(PropertyDescriptor property, String value) {
|
||||
if (isEmpty(value)) {
|
||||
throw new IllegalArgumentException(String.format("'%s' is required, but not specified.", property.getDisplayName()));
|
||||
for (final Map.Entry<String, EventPosition> partition : partitionEventPosition.entrySet()) {
|
||||
final String partitionId = partition.getKey();
|
||||
final EventPosition eventPosition = partition.getValue();
|
||||
getLogger().info("Loaded Event Position [{}] for Partition [{}] from Legacy Checkpoint Storage", eventPosition, partitionId);
|
||||
}
|
||||
|
||||
return partitionEventPosition;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,35 +16,29 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.azure.eventhub;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
|
||||
import com.microsoft.azure.eventhubs.EventData;
|
||||
import com.microsoft.azure.eventhubs.EventHubClient;
|
||||
import com.microsoft.azure.eventhubs.EventHubException;
|
||||
import com.microsoft.azure.eventhubs.EventPosition;
|
||||
import com.microsoft.azure.eventhubs.PartitionReceiver;
|
||||
import com.microsoft.azure.eventhubs.impl.EventHubClientImpl;
|
||||
import com.azure.core.credential.AzureNamedKeyCredential;
|
||||
import com.azure.identity.ManagedIdentityCredential;
|
||||
import com.azure.identity.ManagedIdentityCredentialBuilder;
|
||||
import com.azure.messaging.eventhubs.EventData;
|
||||
import com.azure.messaging.eventhubs.EventHubClientBuilder;
|
||||
import com.azure.messaging.eventhubs.EventHubConsumerClient;
|
||||
import com.azure.messaging.eventhubs.models.EventPosition;
|
||||
import com.azure.messaging.eventhubs.models.PartitionContext;
|
||||
import com.azure.messaging.eventhubs.models.PartitionEvent;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
|
@ -55,6 +49,7 @@ import org.apache.nifi.annotation.documentation.Tags;
|
|||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.PropertyValue;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.expression.ExpressionLanguageScope;
|
||||
|
@ -81,18 +76,17 @@ import org.apache.nifi.processors.azure.eventhub.utils.AzureEventHubUtils;
|
|||
@WritesAttribute(attribute = "eventhub.property.*", description = "The application properties of this message. IE: 'application' would be 'eventhub.property.application'")
|
||||
})
|
||||
public class GetAzureEventHub extends AbstractProcessor {
|
||||
private static final String TRANSIT_URI_FORMAT_STRING = "amqps://%s%s/%s/ConsumerGroups/%s/Partitions/%s";
|
||||
private static final String FORMAT_STRING_FOR_CONECTION_BUILDER = "amqps://%s%s";
|
||||
private static final String TRANSIT_URI_FORMAT_STRING = "amqps://%s/%s/ConsumerGroups/%s/Partitions/%s";
|
||||
|
||||
static final PropertyDescriptor EVENT_HUB_NAME = new PropertyDescriptor.Builder()
|
||||
.name("Event Hub Name")
|
||||
.description("The name of the event hub to pull messages from")
|
||||
.description("Name of Azure Event Hubs source")
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.required(true)
|
||||
.build();
|
||||
static final PropertyDescriptor NAMESPACE = new PropertyDescriptor.Builder()
|
||||
.name("Event Hub Namespace")
|
||||
.description("The namespace that the event hub is assigned to. This is generally equal to <Event Hubs Name>-ns")
|
||||
.description("Namespace of Azure Event Hubs prefixed to Service Bus Endpoint domain")
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
|
||||
.required(true)
|
||||
|
@ -108,13 +102,13 @@ public class GetAzureEventHub extends AbstractProcessor {
|
|||
static final PropertyDescriptor POLICY_PRIMARY_KEY = AzureEventHubUtils.POLICY_PRIMARY_KEY;
|
||||
static final PropertyDescriptor USE_MANAGED_IDENTITY = AzureEventHubUtils.USE_MANAGED_IDENTITY;
|
||||
|
||||
@Deprecated
|
||||
static final PropertyDescriptor NUM_PARTITIONS = new PropertyDescriptor.Builder()
|
||||
.name("Number of Event Hub Partitions")
|
||||
.description("The number of partitions that the event hub has. Only this number of partitions will be used, "
|
||||
+ "so it is important to ensure that if the number of partitions changes that this value be updated. Otherwise, some messages may not be consumed.")
|
||||
.description("This property is deprecated and no longer used.")
|
||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
|
||||
.required(true)
|
||||
.required(false)
|
||||
.build();
|
||||
static final PropertyDescriptor CONSUMER_GROUP = new PropertyDescriptor.Builder()
|
||||
.name("Event Hub Consumer Group")
|
||||
|
@ -138,7 +132,7 @@ public class GetAzureEventHub extends AbstractProcessor {
|
|||
static final PropertyDescriptor RECEIVER_FETCH_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("Partition Recivier Fetch Size")
|
||||
.displayName("Partition Receiver Fetch Size")
|
||||
.description("The number of events that a receiver should fetch from an Event Hubs partition before returning. Default(100)")
|
||||
.description("The number of events that a receiver should fetch from an Event Hubs partition before returning. The default is 100")
|
||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
|
||||
.required(false)
|
||||
|
@ -146,7 +140,8 @@ public class GetAzureEventHub extends AbstractProcessor {
|
|||
static final PropertyDescriptor RECEIVER_FETCH_TIMEOUT = new PropertyDescriptor.Builder()
|
||||
.name("Partiton Receiver Timeout (millseconds)")
|
||||
.name("Partition Receiver Timeout (millseconds)")
|
||||
.description("The amount of time a Partition Receiver should wait to receive the Fetch Size before returning. Default(60000)")
|
||||
.displayName("Partition Receiver Timeout")
|
||||
.description("The amount of time in milliseconds a Partition Receiver should wait to receive the Fetch Size before returning. The default is 60000")
|
||||
.addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
|
||||
.required(false)
|
||||
|
@ -157,42 +152,37 @@ public class GetAzureEventHub extends AbstractProcessor {
|
|||
.description("Any FlowFile that is successfully received from the event hub will be transferred to this Relationship.")
|
||||
.build();
|
||||
|
||||
|
||||
private final ConcurrentMap<String, PartitionReceiver> partitionToReceiverMap = new ConcurrentHashMap<>();
|
||||
private volatile BlockingQueue<String> partitionNames = new LinkedBlockingQueue<>();
|
||||
private volatile Instant configuredEnqueueTime;
|
||||
private volatile int receiverFetchSize;
|
||||
private volatile Duration receiverFetchTimeout;
|
||||
private EventHubClient eventHubClient;
|
||||
|
||||
private final static List<PropertyDescriptor> propertyDescriptors;
|
||||
private final static Set<Relationship> relationships;
|
||||
|
||||
/*
|
||||
* Will ensure that the list of property descriptors is build only once.
|
||||
* Will also create a Set of relationships
|
||||
*/
|
||||
static {
|
||||
List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
|
||||
_propertyDescriptors.add(EVENT_HUB_NAME);
|
||||
_propertyDescriptors.add(SERVICE_BUS_ENDPOINT);
|
||||
_propertyDescriptors.add(NAMESPACE);
|
||||
_propertyDescriptors.add(ACCESS_POLICY);
|
||||
_propertyDescriptors.add(POLICY_PRIMARY_KEY);
|
||||
_propertyDescriptors.add(USE_MANAGED_IDENTITY);
|
||||
_propertyDescriptors.add(NUM_PARTITIONS);
|
||||
_propertyDescriptors.add(CONSUMER_GROUP);
|
||||
_propertyDescriptors.add(ENQUEUE_TIME);
|
||||
_propertyDescriptors.add(RECEIVER_FETCH_SIZE);
|
||||
_propertyDescriptors.add(RECEIVER_FETCH_TIMEOUT);
|
||||
|
||||
propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
|
||||
|
||||
Set<Relationship> _relationships = new HashSet<>();
|
||||
_relationships.add(REL_SUCCESS);
|
||||
relationships = Collections.unmodifiableSet(_relationships);
|
||||
propertyDescriptors = Collections.unmodifiableList(Arrays.asList(
|
||||
EVENT_HUB_NAME,
|
||||
SERVICE_BUS_ENDPOINT,
|
||||
NAMESPACE,
|
||||
ACCESS_POLICY,
|
||||
POLICY_PRIMARY_KEY,
|
||||
USE_MANAGED_IDENTITY,
|
||||
NUM_PARTITIONS,
|
||||
CONSUMER_GROUP,
|
||||
ENQUEUE_TIME,
|
||||
RECEIVER_FETCH_SIZE,
|
||||
RECEIVER_FETCH_TIMEOUT
|
||||
));
|
||||
relationships = Collections.singleton(REL_SUCCESS);
|
||||
}
|
||||
|
||||
private static final Duration DEFAULT_FETCH_TIMEOUT = Duration.ofSeconds(60);
|
||||
private static final int DEFAULT_FETCH_SIZE = 100;
|
||||
|
||||
private final Map<String, EventPosition> partitionEventPositions = new ConcurrentHashMap<>();
|
||||
|
||||
private volatile BlockingQueue<String> partitionIds = new LinkedBlockingQueue<>();
|
||||
private volatile int receiverFetchSize;
|
||||
private volatile Duration receiverFetchTimeout;
|
||||
|
||||
private EventHubConsumerClient eventHubConsumerClient;
|
||||
|
||||
@Override
|
||||
public Set<Relationship> getRelationships() {
|
||||
return relationships;
|
||||
|
@ -205,197 +195,168 @@ public class GetAzureEventHub extends AbstractProcessor {
|
|||
|
||||
@Override
|
||||
protected Collection<ValidationResult> customValidate(ValidationContext context) {
|
||||
List<ValidationResult> retVal = AzureEventHubUtils.customValidate(ACCESS_POLICY, POLICY_PRIMARY_KEY, context);
|
||||
return retVal;
|
||||
}
|
||||
|
||||
protected void setupReceiver(final String connectionString, final ScheduledExecutorService executor) throws ProcessException {
|
||||
try {
|
||||
EventHubClientImpl.USER_AGENT = "ApacheNiFi-azureeventhub/3.1.1";
|
||||
eventHubClient = EventHubClient.createFromConnectionStringSync(connectionString, executor);
|
||||
} catch (IOException | EventHubException e) {
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
}
|
||||
|
||||
PartitionReceiver getReceiver(final ProcessContext context, final String partitionId) throws IOException, EventHubException, ExecutionException, InterruptedException {
|
||||
PartitionReceiver existingReceiver = partitionToReceiverMap.get(partitionId);
|
||||
if (existingReceiver != null) {
|
||||
return existingReceiver;
|
||||
}
|
||||
|
||||
// we want to avoid allowing multiple threads to create Receivers simultaneously because that could result in
|
||||
// having multiple Receivers for the same partition. So if the map does not contain a receiver for this partition,
|
||||
// we will enter a synchronized block and check again (because once we enter the synchronized block, we know that no
|
||||
// other thread is creating a client). If within the synchronized block, we still do not have an entry in the map,
|
||||
// it is up to use to create the receiver, initialize it, and then put it into the map.
|
||||
// We do not use the putIfAbsent method in order to do a CAS operation here because we want to also initialize the
|
||||
// receiver if and only if it is not present in the map. As a result, we need to initialize the receiver and add it
|
||||
// to the map atomically. Hence, the synchronized block.
|
||||
synchronized (this) {
|
||||
existingReceiver = partitionToReceiverMap.get(partitionId);
|
||||
if (existingReceiver != null) {
|
||||
return existingReceiver;
|
||||
}
|
||||
|
||||
final String consumerGroupName = context.getProperty(CONSUMER_GROUP).getValue();
|
||||
|
||||
final PartitionReceiver receiver = eventHubClient.createReceiver(
|
||||
consumerGroupName,
|
||||
partitionId,
|
||||
EventPosition.fromEnqueuedTime(
|
||||
configuredEnqueueTime == null ? Instant.now() : configuredEnqueueTime)).get();
|
||||
|
||||
receiver.setReceiveTimeout(receiverFetchTimeout == null ? Duration.ofMillis(60000) : receiverFetchTimeout);
|
||||
partitionToReceiverMap.put(partitionId, receiver);
|
||||
return receiver;
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is here to try and isolate the Azure related code as the PartitionReceiver cannot be mocked
|
||||
* with PowerMock due to it being final. Unfortunately it extends a base class and does not implement an interface
|
||||
* so even if we create a MockPartitionReciver, it will not work as the two classes are orthogonal.
|
||||
*
|
||||
* @param context - The processcontext for this processor
|
||||
* @param partitionId - The partition ID to retrieve a receiver by.
|
||||
* @return - Returns the events received from the EventBus.
|
||||
* @throws ProcessException -- If any exception is encountered, receiving events it is wrapped in a ProcessException
|
||||
* and then that exception is thrown.
|
||||
*/
|
||||
protected Iterable<EventData> receiveEvents(final ProcessContext context, final String partitionId) throws ProcessException {
|
||||
final PartitionReceiver receiver;
|
||||
try {
|
||||
receiver = getReceiver(context, partitionId);
|
||||
return receiver.receive(receiverFetchSize).get();
|
||||
} catch (final EventHubException | IOException | ExecutionException | InterruptedException e) {
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
return AzureEventHubUtils.customValidate(ACCESS_POLICY, POLICY_PRIMARY_KEY, context);
|
||||
}
|
||||
|
||||
@OnStopped
|
||||
public void tearDown() throws ProcessException {
|
||||
for (final PartitionReceiver receiver : partitionToReceiverMap.values()) {
|
||||
if (null != receiver) {
|
||||
receiver.close();
|
||||
}
|
||||
}
|
||||
public void closeClient() {
|
||||
partitionEventPositions.clear();
|
||||
|
||||
partitionToReceiverMap.clear();
|
||||
try {
|
||||
if (null != eventHubClient) {
|
||||
eventHubClient.closeSync();
|
||||
}
|
||||
executor.shutdown();
|
||||
} catch (final EventHubException e) {
|
||||
throw new ProcessException(e);
|
||||
if (eventHubConsumerClient == null) {
|
||||
getLogger().info("Azure Event Hub Consumer Client not configured");
|
||||
} else {
|
||||
eventHubConsumerClient.close();
|
||||
}
|
||||
}
|
||||
|
||||
private ScheduledExecutorService executor;
|
||||
|
||||
@OnScheduled
|
||||
public void onScheduled(final ProcessContext context) throws ProcessException, URISyntaxException {
|
||||
final BlockingQueue<String> partitionNames = new LinkedBlockingQueue<>();
|
||||
for (int i = 0; i < context.getProperty(NUM_PARTITIONS).asInteger(); i++) {
|
||||
partitionNames.add(String.valueOf(i));
|
||||
}
|
||||
this.partitionNames = partitionNames;
|
||||
public void onScheduled(final ProcessContext context) {
|
||||
eventHubConsumerClient = createEventHubConsumerClient(context);
|
||||
|
||||
final String namespace = context.getProperty(NAMESPACE).getValue();
|
||||
final String eventHubName = context.getProperty(EVENT_HUB_NAME).getValue();
|
||||
final String serviceBusEndpoint = context.getProperty(SERVICE_BUS_ENDPOINT).getValue();
|
||||
final boolean useManagedIdentity = context.getProperty(USE_MANAGED_IDENTITY).asBoolean();
|
||||
final String connectionString;
|
||||
|
||||
if(useManagedIdentity){
|
||||
connectionString = AzureEventHubUtils.getManagedIdentityConnectionString(namespace, serviceBusEndpoint, eventHubName);
|
||||
} else {
|
||||
final String policyName = context.getProperty(ACCESS_POLICY).getValue();
|
||||
final String policyKey = context.getProperty(POLICY_PRIMARY_KEY).getValue();
|
||||
connectionString = new ConnectionStringBuilder()
|
||||
.setEndpoint(new URI(String.format(FORMAT_STRING_FOR_CONECTION_BUILDER, namespace, serviceBusEndpoint)))
|
||||
.setEventHubName(eventHubName)
|
||||
.setSasKeyName(policyName)
|
||||
.setSasKey(policyKey).toString();
|
||||
}
|
||||
|
||||
if(context.getProperty(ENQUEUE_TIME).isSet()) {
|
||||
configuredEnqueueTime = Instant.parse(context.getProperty(ENQUEUE_TIME).toString());
|
||||
} else {
|
||||
configuredEnqueueTime = null;
|
||||
}
|
||||
if(context.getProperty(RECEIVER_FETCH_SIZE).isSet()) {
|
||||
if (context.getProperty(RECEIVER_FETCH_SIZE).isSet()) {
|
||||
receiverFetchSize = context.getProperty(RECEIVER_FETCH_SIZE).asInteger();
|
||||
} else {
|
||||
receiverFetchSize = 100;
|
||||
receiverFetchSize = DEFAULT_FETCH_SIZE;
|
||||
}
|
||||
if(context.getProperty(RECEIVER_FETCH_TIMEOUT).isSet()) {
|
||||
if (context.getProperty(RECEIVER_FETCH_TIMEOUT).isSet()) {
|
||||
receiverFetchTimeout = Duration.ofMillis(context.getProperty(RECEIVER_FETCH_TIMEOUT).asLong());
|
||||
} else {
|
||||
receiverFetchTimeout = null;
|
||||
receiverFetchTimeout = DEFAULT_FETCH_TIMEOUT;
|
||||
}
|
||||
|
||||
executor = Executors.newScheduledThreadPool(4);
|
||||
setupReceiver(connectionString, executor);
|
||||
this.partitionIds = getPartitionIds();
|
||||
|
||||
final PropertyValue enqueuedTimeProperty = context.getProperty(ENQUEUE_TIME);
|
||||
final Instant initialEnqueuedTime;
|
||||
if (enqueuedTimeProperty.isSet()) {
|
||||
initialEnqueuedTime = Instant.parse(enqueuedTimeProperty.getValue());
|
||||
} else {
|
||||
initialEnqueuedTime = Instant.now();
|
||||
}
|
||||
final EventPosition initialEventPosition = EventPosition.fromEnqueuedTime(initialEnqueuedTime);
|
||||
for (final String partitionId : partitionIds) {
|
||||
partitionEventPositions.put(partitionId, initialEventPosition);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
final BlockingQueue<String> partitionIds = this.partitionNames;
|
||||
final String partitionId = partitionIds.poll();
|
||||
if (partitionId == null) {
|
||||
getLogger().debug("No partitions available");
|
||||
return;
|
||||
}
|
||||
|
||||
Long lastSequenceNumber = null;
|
||||
final StopWatch stopWatch = new StopWatch(true);
|
||||
try {
|
||||
final Iterable<PartitionEvent> events = receiveEvents(partitionId);
|
||||
|
||||
final Iterable<EventData> receivedEvents = receiveEvents(context, partitionId);
|
||||
if (receivedEvents == null) {
|
||||
return;
|
||||
for (final PartitionEvent partitionEvent : events) {
|
||||
final Map<String, String> attributes = getAttributes(partitionEvent);
|
||||
|
||||
FlowFile flowFile = session.create();
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
|
||||
final EventData eventData = partitionEvent.getData();
|
||||
final byte[] body = eventData.getBody();
|
||||
flowFile = session.write(flowFile, outputStream -> outputStream.write(body));
|
||||
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
|
||||
final String transitUri = getTransitUri(partitionId);
|
||||
session.getProvenanceReporter().receive(flowFile, transitUri, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
||||
|
||||
lastSequenceNumber = eventData.getSequenceNumber();
|
||||
}
|
||||
|
||||
for (final EventData eventData : receivedEvents) {
|
||||
if (null != eventData) {
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
FlowFile flowFile = session.create();
|
||||
final EventData.SystemProperties systemProperties = eventData.getSystemProperties();
|
||||
|
||||
if (null != systemProperties) {
|
||||
attributes.put("eventhub.enqueued.timestamp", String.valueOf(systemProperties.getEnqueuedTime()));
|
||||
attributes.put("eventhub.offset", systemProperties.getOffset());
|
||||
attributes.put("eventhub.sequence", String.valueOf(systemProperties.getSequenceNumber()));
|
||||
}
|
||||
|
||||
final Map<String,String> applicationProperties = AzureEventHubUtils.getApplicationProperties(eventData);
|
||||
attributes.putAll(applicationProperties);
|
||||
|
||||
attributes.put("eventhub.name", context.getProperty(EVENT_HUB_NAME).getValue());
|
||||
attributes.put("eventhub.partition", partitionId);
|
||||
|
||||
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
flowFile = session.write(flowFile, out -> {
|
||||
out.write(eventData.getBytes());
|
||||
});
|
||||
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
|
||||
final String namespace = context.getProperty(NAMESPACE).getValue();
|
||||
final String eventHubName = context.getProperty(EVENT_HUB_NAME).getValue();
|
||||
final String consumerGroup = context.getProperty(CONSUMER_GROUP).getValue();
|
||||
final String serviceBusEndPoint = context.getProperty(SERVICE_BUS_ENDPOINT).getValue();
|
||||
final String transitUri = String.format(TRANSIT_URI_FORMAT_STRING,
|
||||
namespace, serviceBusEndPoint, eventHubName, consumerGroup, partitionId);
|
||||
session.getProvenanceReporter().receive(flowFile, transitUri, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
||||
}
|
||||
if (lastSequenceNumber == null) {
|
||||
getLogger().debug("Partition [{}] Event Position not updated: Last Sequence Number not found", partitionId);
|
||||
} else {
|
||||
final EventPosition eventPosition = EventPosition.fromSequenceNumber(lastSequenceNumber);
|
||||
partitionEventPositions.put(partitionId, eventPosition);
|
||||
getLogger().debug("Partition [{}] Event Position updated: Sequence Number [{}]", partitionId, lastSequenceNumber);
|
||||
}
|
||||
} finally {
|
||||
partitionIds.offer(partitionId);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Partition Identifiers from Event Hub Consumer Client for polling
|
||||
*
|
||||
* @return Queue of Partition Identifiers
|
||||
*/
|
||||
protected BlockingQueue<String> getPartitionIds() {
|
||||
final BlockingQueue<String> configuredPartitionIds = new LinkedBlockingQueue<>();
|
||||
for (final String partitionId : eventHubConsumerClient.getPartitionIds()) {
|
||||
configuredPartitionIds.add(partitionId);
|
||||
}
|
||||
return configuredPartitionIds;
|
||||
}
|
||||
|
||||
/**
|
||||
* Receive Events from specified partition is synchronized to avoid concurrent requests for the same partition
|
||||
*
|
||||
* @param partitionId Partition Identifier
|
||||
* @return Iterable of Partition Events or empty when none received
|
||||
*/
|
||||
protected synchronized Iterable<PartitionEvent> receiveEvents(final String partitionId) {
|
||||
final EventPosition eventPosition = partitionEventPositions.getOrDefault(partitionId, EventPosition.fromEnqueuedTime(Instant.now()));
|
||||
getLogger().debug("Receiving Events for Partition [{}] from Position [{}]", partitionId, eventPosition);
|
||||
return eventHubConsumerClient.receiveFromPartition(partitionId, receiverFetchSize, eventPosition, receiverFetchTimeout);
|
||||
}
|
||||
|
||||
private EventHubConsumerClient createEventHubConsumerClient(final ProcessContext context) {
|
||||
final String namespace = context.getProperty(NAMESPACE).getValue();
|
||||
final String eventHubName = context.getProperty(EVENT_HUB_NAME).getValue();
|
||||
final String serviceBusEndpoint = context.getProperty(SERVICE_BUS_ENDPOINT).getValue();
|
||||
final boolean useManagedIdentity = context.getProperty(USE_MANAGED_IDENTITY).asBoolean();
|
||||
final String fullyQualifiedNamespace = String.format("%s%s", namespace, serviceBusEndpoint);
|
||||
|
||||
final EventHubClientBuilder eventHubClientBuilder = new EventHubClientBuilder();
|
||||
|
||||
final String consumerGroup = context.getProperty(CONSUMER_GROUP).getValue();
|
||||
eventHubClientBuilder.consumerGroup(consumerGroup);
|
||||
|
||||
if (useManagedIdentity) {
|
||||
final ManagedIdentityCredentialBuilder managedIdentityCredentialBuilder = new ManagedIdentityCredentialBuilder();
|
||||
final ManagedIdentityCredential managedIdentityCredential = managedIdentityCredentialBuilder.build();
|
||||
eventHubClientBuilder.credential(fullyQualifiedNamespace, eventHubName, managedIdentityCredential);
|
||||
} else {
|
||||
final String policyName = context.getProperty(ACCESS_POLICY).getValue();
|
||||
final String policyKey = context.getProperty(POLICY_PRIMARY_KEY).getValue();
|
||||
final AzureNamedKeyCredential azureNamedKeyCredential = new AzureNamedKeyCredential(policyName, policyKey);
|
||||
eventHubClientBuilder.credential(fullyQualifiedNamespace, eventHubName, azureNamedKeyCredential);
|
||||
}
|
||||
return eventHubClientBuilder.buildConsumerClient();
|
||||
}
|
||||
|
||||
private String getTransitUri(final String partitionId) {
|
||||
return String.format(TRANSIT_URI_FORMAT_STRING,
|
||||
eventHubConsumerClient.getFullyQualifiedNamespace(),
|
||||
eventHubConsumerClient.getEventHubName(),
|
||||
eventHubConsumerClient.getConsumerGroup(),
|
||||
partitionId
|
||||
);
|
||||
}
|
||||
|
||||
private Map<String, String> getAttributes(final PartitionEvent partitionEvent) {
|
||||
final Map<String, String> attributes = new LinkedHashMap<>();
|
||||
|
||||
final EventData eventData = partitionEvent.getData();
|
||||
|
||||
attributes.put("eventhub.enqueued.timestamp", String.valueOf(eventData.getEnqueuedTime()));
|
||||
attributes.put("eventhub.offset", String.valueOf(eventData.getOffset()));
|
||||
attributes.put("eventhub.sequence", String.valueOf(eventData.getSequenceNumber()));
|
||||
|
||||
final PartitionContext partitionContext = partitionEvent.getPartitionContext();
|
||||
attributes.put("eventhub.name", partitionContext.getEventHubName());
|
||||
attributes.put("eventhub.partition", partitionContext.getPartitionId());
|
||||
|
||||
final Map<String,String> applicationProperties = AzureEventHubUtils.getApplicationProperties(eventData.getProperties());
|
||||
attributes.putAll(applicationProperties);
|
||||
|
||||
return attributes;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,34 +16,25 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.azure.eventhub;
|
||||
|
||||
import com.azure.core.credential.AzureNamedKeyCredential;
|
||||
import com.azure.identity.ManagedIdentityCredential;
|
||||
import com.azure.identity.ManagedIdentityCredentialBuilder;
|
||||
import com.azure.messaging.eventhubs.EventData;
|
||||
import com.azure.messaging.eventhubs.EventHubClientBuilder;
|
||||
import com.azure.messaging.eventhubs.EventHubProducerClient;
|
||||
import com.azure.messaging.eventhubs.models.SendOptions;
|
||||
import org.apache.nifi.processors.azure.eventhub.utils.AzureEventHubUtils;
|
||||
import java.io.IOException;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.CancellationException;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionException;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import com.microsoft.azure.eventhubs.EventData;
|
||||
import com.microsoft.azure.eventhubs.EventHubClient;
|
||||
import com.microsoft.azure.eventhubs.EventHubException;
|
||||
import com.microsoft.azure.eventhubs.IllegalConnectionStringFormatException;
|
||||
import com.microsoft.azure.eventhubs.impl.EventHubClientImpl;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.commons.lang3.math.NumberUtils;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
|
@ -71,22 +62,20 @@ import org.apache.nifi.util.StopWatch;
|
|||
@SupportsBatching
|
||||
@Tags({"microsoft", "azure", "cloud", "eventhub", "events", "streams", "streaming"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Sends the contents of a FlowFile to Windows Azure Event Hubs. Note: the content of the FlowFile will be buffered into memory before being sent, "
|
||||
+ "so care should be taken to avoid sending FlowFiles to this Processor that exceed the amount of Java Heap Space available. "
|
||||
+ "Also please be aware that this processor creates a thread pool of 4 threads for Event Hub Client. They will be extra threads other than the concurrent tasks scheduled for this processor.")
|
||||
@SystemResourceConsideration(resource = SystemResource.MEMORY)
|
||||
@CapabilityDescription("Send FlowFile contents to Azure Event Hubs")
|
||||
@SystemResourceConsideration(resource = SystemResource.MEMORY, description = "The Processor buffers FlowFile contents in memory before sending")
|
||||
public class PutAzureEventHub extends AbstractProcessor {
|
||||
private static final String TRANSIT_URI_FORMAT_STRING = "amqps://%s%s/%s";
|
||||
|
||||
static final PropertyDescriptor EVENT_HUB_NAME = new PropertyDescriptor.Builder()
|
||||
.name("Event Hub Name")
|
||||
.description("The name of the event hub to send to")
|
||||
.description("Name of Azure Event Hubs destination")
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.required(true)
|
||||
.build();
|
||||
static final PropertyDescriptor NAMESPACE = new PropertyDescriptor.Builder()
|
||||
.name("Event Hub Namespace")
|
||||
.description("The namespace that the event hub is assigned to. This is generally equal to <Event Hubs Name>-ns")
|
||||
.description("Namespace of Azure Event Hubs prefixed to Service Bus Endpoint domain")
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
|
||||
.required(true)
|
||||
|
@ -113,8 +102,8 @@ public class PutAzureEventHub extends AbstractProcessor {
|
|||
.build();
|
||||
static final PropertyDescriptor MAX_BATCH_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("max-batch-size")
|
||||
.displayName("Maximum batch size")
|
||||
.description("Maximum count of flow files being processed in one batch.")
|
||||
.displayName("Maximum Batch Size")
|
||||
.description("Maximum number of FlowFiles processed for each Processor invocation")
|
||||
.required(true)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
|
||||
.addValidator(StandardValidators.NUMBER_VALIDATOR)
|
||||
|
@ -130,32 +119,29 @@ public class PutAzureEventHub extends AbstractProcessor {
|
|||
.description("Any FlowFile that could not be sent to the event hub will be transferred to this Relationship.")
|
||||
.build();
|
||||
|
||||
private volatile BlockingQueue<EventHubClient> senderQueue = new LinkedBlockingQueue<>();
|
||||
|
||||
private final static List<PropertyDescriptor> propertyDescriptors;
|
||||
private final static Set<Relationship> relationships;
|
||||
|
||||
/*
|
||||
* Will ensure that the list of property descriptors is build only once.
|
||||
* Will also create a Set of relationships
|
||||
*/
|
||||
static {
|
||||
List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
|
||||
_propertyDescriptors.add(EVENT_HUB_NAME);
|
||||
_propertyDescriptors.add(NAMESPACE);
|
||||
_propertyDescriptors.add(SERVICE_BUS_ENDPOINT);
|
||||
_propertyDescriptors.add(ACCESS_POLICY);
|
||||
_propertyDescriptors.add(POLICY_PRIMARY_KEY);
|
||||
_propertyDescriptors.add(USE_MANAGED_IDENTITY);
|
||||
_propertyDescriptors.add(PARTITIONING_KEY_ATTRIBUTE_NAME);
|
||||
_propertyDescriptors.add(MAX_BATCH_SIZE);
|
||||
propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
|
||||
final List<PropertyDescriptor> configuredDescriptors = new ArrayList<>();
|
||||
configuredDescriptors.add(EVENT_HUB_NAME);
|
||||
configuredDescriptors.add(NAMESPACE);
|
||||
configuredDescriptors.add(SERVICE_BUS_ENDPOINT);
|
||||
configuredDescriptors.add(ACCESS_POLICY);
|
||||
configuredDescriptors.add(POLICY_PRIMARY_KEY);
|
||||
configuredDescriptors.add(USE_MANAGED_IDENTITY);
|
||||
configuredDescriptors.add(PARTITIONING_KEY_ATTRIBUTE_NAME);
|
||||
configuredDescriptors.add(MAX_BATCH_SIZE);
|
||||
propertyDescriptors = Collections.unmodifiableList(configuredDescriptors);
|
||||
|
||||
Set<Relationship> _relationships = new HashSet<>();
|
||||
_relationships.add(REL_SUCCESS);
|
||||
_relationships.add(REL_FAILURE);
|
||||
relationships = Collections.unmodifiableSet(_relationships);
|
||||
final Set<Relationship> configuredRelationships = new HashSet<>();
|
||||
configuredRelationships.add(REL_SUCCESS);
|
||||
configuredRelationships.add(REL_FAILURE);
|
||||
relationships = Collections.unmodifiableSet(configuredRelationships);
|
||||
}
|
||||
|
||||
private EventHubProducerClient eventHubProducerClient;
|
||||
|
||||
@Override
|
||||
public Set<Relationship> getRelationships() {
|
||||
return relationships;
|
||||
|
@ -166,124 +152,103 @@ public class PutAzureEventHub extends AbstractProcessor {
|
|||
return propertyDescriptors;
|
||||
}
|
||||
|
||||
private ScheduledExecutorService executor;
|
||||
|
||||
@OnScheduled
|
||||
public final void setupClient(final ProcessContext context) throws ProcessException{
|
||||
public final void createClient(final ProcessContext context) {
|
||||
eventHubProducerClient = createEventHubProducerClient(context);
|
||||
}
|
||||
|
||||
@OnStopped
|
||||
public void tearDown() {
|
||||
EventHubClient sender;
|
||||
while ((sender = senderQueue.poll()) != null) {
|
||||
sender.close();
|
||||
public void closeClient() {
|
||||
if (eventHubProducerClient == null) {
|
||||
getLogger().info("Azure Event Hub Producer Client not configured");
|
||||
} else {
|
||||
eventHubProducerClient.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<ValidationResult> customValidate(ValidationContext context) {
|
||||
List<ValidationResult> retVal = AzureEventHubUtils.customValidate(ACCESS_POLICY, POLICY_PRIMARY_KEY, context);
|
||||
return retVal;
|
||||
return AzureEventHubUtils.customValidate(ACCESS_POLICY, POLICY_PRIMARY_KEY, context);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
try {
|
||||
populateSenderQueue(context);
|
||||
} catch (ProcessException e) {
|
||||
context.yield();
|
||||
throw e;
|
||||
}
|
||||
|
||||
final StopWatch stopWatch = new StopWatch(true);
|
||||
|
||||
final String partitioningKeyAttributeName = context.getProperty(PARTITIONING_KEY_ATTRIBUTE_NAME).getValue();
|
||||
|
||||
// Get N flow files
|
||||
final int maxBatchSize = NumberUtils.toInt(context.getProperty(MAX_BATCH_SIZE).getValue(), 100);
|
||||
final List<FlowFile> flowFileList = session.get(maxBatchSize);
|
||||
final int maxBatchSize = context.getProperty(MAX_BATCH_SIZE).asInteger();
|
||||
final List<FlowFile> flowFileBatch = session.get(maxBatchSize);
|
||||
|
||||
// Convert and send each flow file
|
||||
final BlockingQueue<CompletableFuture<FlowFileResultCarrier<Relationship>>> futureQueue = new LinkedBlockingQueue<>();
|
||||
for (FlowFile flowFile : flowFileList) {
|
||||
if (flowFile == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
futureQueue.offer(handleFlowFile(flowFile, partitioningKeyAttributeName, session));
|
||||
final List<FlowFileResultCarrier<Relationship>> flowFileResults = new ArrayList<>();
|
||||
for (final FlowFile flowFile : flowFileBatch) {
|
||||
final FlowFileResultCarrier<Relationship> flowFileResult = handleFlowFile(flowFile, partitioningKeyAttributeName, session);
|
||||
flowFileResults.add(flowFileResult);
|
||||
}
|
||||
|
||||
waitForAllFutures(context, session, stopWatch, futureQueue);
|
||||
processFlowFileResults(context, session, stopWatch, flowFileResults);
|
||||
}
|
||||
|
||||
/**
|
||||
* Joins all the futures so it can determine which flow files from given batch were sent successfully and which were not.
|
||||
*
|
||||
* @param context of this instance of the processor
|
||||
* @param session that handles all flow files sent within the future queue
|
||||
* @param stopWatch for time measurements
|
||||
* @param futureQueue a list of futures of messages that had been sent within above context and session before this method was called.
|
||||
*/
|
||||
protected void waitForAllFutures(
|
||||
protected EventHubProducerClient createEventHubProducerClient(final ProcessContext context) throws ProcessException {
|
||||
final boolean useManagedIdentity = context.getProperty(USE_MANAGED_IDENTITY).asBoolean();
|
||||
final String namespace = context.getProperty(NAMESPACE).getValue();
|
||||
final String serviceBusEndpoint = context.getProperty(SERVICE_BUS_ENDPOINT).getValue();
|
||||
final String eventHubName = context.getProperty(EVENT_HUB_NAME).getValue();
|
||||
|
||||
try {
|
||||
final EventHubClientBuilder eventHubClientBuilder = new EventHubClientBuilder();
|
||||
|
||||
final String fullyQualifiedNamespace = String.format("%s%s", namespace, serviceBusEndpoint);
|
||||
if (useManagedIdentity) {
|
||||
final ManagedIdentityCredentialBuilder managedIdentityCredentialBuilder = new ManagedIdentityCredentialBuilder();
|
||||
final ManagedIdentityCredential managedIdentityCredential = managedIdentityCredentialBuilder.build();
|
||||
eventHubClientBuilder.credential(fullyQualifiedNamespace, eventHubName, managedIdentityCredential);
|
||||
} else {
|
||||
final String policyName = context.getProperty(ACCESS_POLICY).getValue();
|
||||
final String policyKey = context.getProperty(POLICY_PRIMARY_KEY).getValue();
|
||||
final AzureNamedKeyCredential azureNamedKeyCredential = new AzureNamedKeyCredential(policyName, policyKey);
|
||||
eventHubClientBuilder.credential(fullyQualifiedNamespace, eventHubName, azureNamedKeyCredential);
|
||||
}
|
||||
|
||||
return eventHubClientBuilder.buildProducerClient();
|
||||
} catch (final Exception e) {
|
||||
throw new ProcessException("EventHubClient creation failed", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void processFlowFileResults(
|
||||
final ProcessContext context,
|
||||
final ProcessSession session,
|
||||
final StopWatch stopWatch,
|
||||
final BlockingQueue<CompletableFuture<FlowFileResultCarrier<Relationship>>> futureQueue){
|
||||
|
||||
final List<FlowFileResultCarrier<Relationship>> flowFileResults
|
||||
) {
|
||||
try {
|
||||
for (CompletableFuture<FlowFileResultCarrier<Relationship>> completableFuture : futureQueue) {
|
||||
completableFuture.join();
|
||||
|
||||
final FlowFileResultCarrier<Relationship> flowFileResult = completableFuture.get();
|
||||
if(flowFileResult == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
for (final FlowFileResultCarrier<Relationship> flowFileResult : flowFileResults) {
|
||||
final FlowFile flowFile = flowFileResult.getFlowFile();
|
||||
|
||||
if(flowFileResult.getResult() == REL_SUCCESS) {
|
||||
if (flowFileResult.getResult() == REL_SUCCESS) {
|
||||
final String namespace = context.getProperty(NAMESPACE).getValue();
|
||||
final String eventHubName = context.getProperty(EVENT_HUB_NAME).getValue();
|
||||
final String serviceBusEndpoint = context.getProperty(SERVICE_BUS_ENDPOINT).getValue();
|
||||
final String transitUri = String.format(TRANSIT_URI_FORMAT_STRING, namespace, serviceBusEndpoint, eventHubName);
|
||||
session.getProvenanceReporter().send(flowFile, transitUri, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
|
||||
} else {
|
||||
final Throwable processException = flowFileResult.getException();
|
||||
getLogger().error("Failed to send {} to EventHub due to {}; routing to failure", new Object[]{flowFile, processException}, processException);
|
||||
getLogger().error("Send failed {}", flowFile, processException);
|
||||
session.transfer(session.penalize(flowFile), REL_FAILURE);
|
||||
}
|
||||
}
|
||||
} catch (InterruptedException | ExecutionException | CancellationException | CompletionException e) {
|
||||
getLogger().error("Batch processing failed", e);
|
||||
} catch (final Exception e) {
|
||||
session.rollback();
|
||||
|
||||
if(e instanceof InterruptedException) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
|
||||
throw new ProcessException("Batch processing failed", e);
|
||||
getLogger().error("FlowFile Batch Size [{}] processing failed", flowFileResults.size());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert flow file to eventhub message entities (and send)!!!
|
||||
*
|
||||
* @param flowFile to be converted to a message and sent to Eventhub (Body = content, User Properties = attributes, partitioning key = value configured attribute)
|
||||
* @param partitioningKeyAttributeName where the partitioning is saved within each flow file
|
||||
* @param session under which is this flow file being managed
|
||||
*
|
||||
* @return Completable future carrying the context of flowfile used as a base for message being send. Never Null.
|
||||
* */
|
||||
protected CompletableFuture<FlowFileResultCarrier<Relationship>> handleFlowFile(FlowFile flowFile, final String partitioningKeyAttributeName, final ProcessSession session) {
|
||||
|
||||
// Read message body
|
||||
private FlowFileResultCarrier<Relationship> handleFlowFile(FlowFile flowFile, final String partitioningKeyAttributeName, final ProcessSession session) {
|
||||
final byte[] buffer = new byte[(int) flowFile.getSize()];
|
||||
session.read(flowFile, in -> StreamUtils.fillBuffer(in, buffer));
|
||||
|
||||
// Lift partitioning key
|
||||
final String partitioningKey;
|
||||
if (StringUtils.isNotBlank(partitioningKeyAttributeName)) {
|
||||
partitioningKey = flowFile.getAttribute(partitioningKeyAttributeName);
|
||||
|
@ -291,132 +256,25 @@ public class PutAzureEventHub extends AbstractProcessor {
|
|||
partitioningKey = null;
|
||||
}
|
||||
|
||||
// Prepare user properties
|
||||
final Map<String, Object> userProperties;
|
||||
Map<String, String> attributes = flowFile.getAttributes();
|
||||
if(attributes == null) {
|
||||
userProperties = Collections.emptyMap();
|
||||
}else {
|
||||
userProperties = new HashMap<>(attributes);
|
||||
}
|
||||
|
||||
// Send the message
|
||||
try {
|
||||
return sendMessage(buffer, partitioningKey, userProperties)
|
||||
.thenApplyAsync(param -> {
|
||||
return new FlowFileResultCarrier<Relationship>(flowFile, REL_SUCCESS);
|
||||
})
|
||||
.exceptionally(processException -> {
|
||||
return new FlowFileResultCarrier<Relationship>(flowFile, REL_FAILURE, processException);
|
||||
});
|
||||
|
||||
} catch (final ProcessException processException) {
|
||||
return CompletableFuture.completedFuture(new FlowFileResultCarrier<Relationship>(flowFile, REL_FAILURE, processException));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Prepare at least one Event hub sender based on this instance of processor.
|
||||
*
|
||||
* @param context of this processor instance from which all connectivity information properties are taken.
|
||||
*/
|
||||
protected void populateSenderQueue(ProcessContext context) {
|
||||
if(senderQueue.size() == 0){
|
||||
final int numThreads = context.getMaxConcurrentTasks();
|
||||
senderQueue = new LinkedBlockingQueue<>(numThreads);
|
||||
executor = Executors.newScheduledThreadPool(4);
|
||||
final boolean useManagedIdentiy = context.getProperty(USE_MANAGED_IDENTITY).asBoolean();
|
||||
final String policyName, policyKey;
|
||||
if(useManagedIdentiy) {
|
||||
policyName = AzureEventHubUtils.MANAGED_IDENTITY_POLICY;
|
||||
policyKey =null;
|
||||
} else {
|
||||
policyName = context.getProperty(ACCESS_POLICY).getValue();
|
||||
policyKey = context.getProperty(POLICY_PRIMARY_KEY).getValue();
|
||||
}
|
||||
final String namespace = context.getProperty(NAMESPACE).getValue();
|
||||
final String serviceBusEndpoint = context.getProperty(SERVICE_BUS_ENDPOINT).getValue();
|
||||
final String eventHubName = context.getProperty(EVENT_HUB_NAME).getValue();
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
final EventHubClient client = createEventHubClient(namespace, serviceBusEndpoint, eventHubName, policyName, policyKey, executor);
|
||||
if(null != client) {
|
||||
senderQueue.offer(client);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param namespace name of the Eventhub namespace (part of the domain name)
|
||||
* @param eventHubName name of the eventhub, a message broker entity. Like topic.
|
||||
* @param policyName technically it is username bound to eventhub namespace or hub and privileges.
|
||||
* @param policyKey password belonging to the above policy
|
||||
* @param executor thread executor to perform the client connection.
|
||||
* @return An initialized eventhub client based on supplied parameters.
|
||||
* @throws ProcessException when creation of event hub fails due to formatting of conection string. Authorization or even network connectivity.
|
||||
*/
|
||||
protected EventHubClient createEventHubClient(
|
||||
final String namespace,
|
||||
final String serviceBusEndpoint,
|
||||
final String eventHubName,
|
||||
final String policyName,
|
||||
final String policyKey,
|
||||
final ScheduledExecutorService executor)
|
||||
throws ProcessException{
|
||||
final Map<String, String> attributes = flowFile.getAttributes();
|
||||
final Map<String, ?> userProperties = attributes == null ? Collections.emptyMap() : attributes;
|
||||
|
||||
try {
|
||||
EventHubClientImpl.USER_AGENT = "ApacheNiFi-azureeventhub/3.1.1";
|
||||
final String connectionString;
|
||||
if(policyName == AzureEventHubUtils.MANAGED_IDENTITY_POLICY) {
|
||||
connectionString = AzureEventHubUtils.getManagedIdentityConnectionString(namespace, serviceBusEndpoint, eventHubName);
|
||||
} else{
|
||||
connectionString = getConnectionString(namespace, serviceBusEndpoint, eventHubName, policyName, policyKey);
|
||||
}
|
||||
return EventHubClient.createFromConnectionStringSync(connectionString, executor);
|
||||
} catch (IOException | EventHubException | IllegalConnectionStringFormatException e) {
|
||||
getLogger().error("Failed to create EventHubClient due to {}", new Object[]{e.getMessage()}, e);
|
||||
throw new ProcessException(e);
|
||||
sendMessage(buffer, partitioningKey, userProperties);
|
||||
return new FlowFileResultCarrier<>(flowFile, REL_SUCCESS);
|
||||
} catch (final Exception processException) {
|
||||
return new FlowFileResultCarrier<>(flowFile, REL_FAILURE, processException);
|
||||
}
|
||||
}
|
||||
|
||||
protected String getConnectionString(final String namespace, final String serviceBusEndpoint, final String eventHubName, final String policyName, final String policyKey){
|
||||
return AzureEventHubUtils.getSharedAccessSignatureConnectionString(namespace, serviceBusEndpoint, eventHubName, policyName, policyKey);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param buffer Block of data to be sent as a message body. Entire array is used. See Event hub limits for body size.
|
||||
* @param partitioningKey A hint for Eventhub message broker how to distribute messages consistently amongst multiple partitions.
|
||||
* @param userProperties A key value set of customary information that is attached in User defined properties part of the message.
|
||||
* @return future object for referencing a success/failure of this message sending.
|
||||
* @throws ProcessException
|
||||
*
|
||||
* @see <a href="https://docs.microsoft.com/en-us/azure/event-hubs/event-hubs-quotas">Event Hubs Quotas</a>
|
||||
*/
|
||||
protected CompletableFuture<Void> sendMessage(final byte[] buffer, String partitioningKey, Map<String, Object> userProperties) throws ProcessException {
|
||||
|
||||
final EventHubClient sender = senderQueue.poll();
|
||||
if(sender == null) {
|
||||
throw new ProcessException("No EventHubClients are configured for sending");
|
||||
private void sendMessage(final byte[] buffer, String partitioningKey, Map<String, ?> userProperties) {
|
||||
final EventData eventData = new EventData(buffer);
|
||||
eventData.getProperties().putAll(userProperties);
|
||||
final SendOptions sendOptions = new SendOptions();
|
||||
if (StringUtils.isNotBlank(partitioningKey)) {
|
||||
sendOptions.setPartitionKey(partitioningKey);
|
||||
}
|
||||
|
||||
// Create message with properties
|
||||
final EventData eventData = EventData.create(buffer);
|
||||
final Map<String, Object> properties = eventData.getProperties();
|
||||
if(userProperties != null && properties != null) {
|
||||
properties.putAll(userProperties);
|
||||
}
|
||||
|
||||
// Send with optional partition key
|
||||
final CompletableFuture<Void> eventFuture;
|
||||
if(StringUtils.isNotBlank(partitioningKey)) {
|
||||
eventFuture = sender.send(eventData, partitioningKey);
|
||||
}else {
|
||||
eventFuture = sender.send(eventData);
|
||||
}
|
||||
|
||||
senderQueue.offer(sender);
|
||||
|
||||
return eventFuture;
|
||||
eventHubProducerClient.send(Collections.singleton(eventData), sendOptions);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,63 @@
|
|||
/*
|
||||
* 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.nifi.processors.azure.eventhub.position;
|
||||
|
||||
import com.azure.messaging.eventhubs.models.EventPosition;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Earliest Event Position provides a workaround for implementing a start position other than EventPosition.latest()
|
||||
* The initial number of partitions is not known. This should be replaced pending implementation of
|
||||
* <a href="https://github.com/Azure/azure-sdk-for-java/issues/11431">Azure SDK for Java Issue 11431</a>
|
||||
*/
|
||||
public class EarliestEventPositionProvider implements EventPositionProvider {
|
||||
/**
|
||||
* Get Initial Partition Event Position using earliest available strategy
|
||||
*
|
||||
* @return Map of Partition to earliest Event Position
|
||||
*/
|
||||
@Override
|
||||
public Map<String, EventPosition> getInitialPartitionEventPosition() {
|
||||
return new EarliestEventPosition();
|
||||
}
|
||||
|
||||
private static class EarliestEventPosition extends HashMap<String, EventPosition> {
|
||||
/**
|
||||
* Contains Key returns true in order for PartitionPumpManager to request the EventPosition
|
||||
*
|
||||
* @param partitionId Partition Identifier requested
|
||||
* @return Returns true for all invocations
|
||||
*/
|
||||
@Override
|
||||
public boolean containsKey(final Object partitionId) {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get EventPosition.earliest() for PartitionPumpManager.startPartitionPump()
|
||||
*
|
||||
* @param partitionId Partition Identifier requested
|
||||
* @return EventPosition.earliest()
|
||||
*/
|
||||
@Override
|
||||
public EventPosition get(final Object partitionId) {
|
||||
return EventPosition.earliest();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
* 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.nifi.processors.azure.eventhub.position;
|
||||
|
||||
import com.azure.messaging.eventhubs.models.EventPosition;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Provider for initial Event Position
|
||||
*/
|
||||
public interface EventPositionProvider {
|
||||
/**
|
||||
* Get Map of Partition to Initial Event Position
|
||||
*
|
||||
* @return Map of Partition to Initial Event Position
|
||||
*/
|
||||
Map<String, EventPosition> getInitialPartitionEventPosition();
|
||||
}
|
|
@ -0,0 +1,151 @@
|
|||
/*
|
||||
* 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.nifi.processors.azure.eventhub.position;
|
||||
|
||||
import com.azure.core.util.BinaryData;
|
||||
import com.azure.messaging.eventhubs.models.EventPosition;
|
||||
import com.azure.storage.blob.BlobAsyncClient;
|
||||
import com.azure.storage.blob.BlobContainerAsyncClient;
|
||||
import com.azure.storage.blob.models.BlobItem;
|
||||
import com.azure.storage.blob.models.BlobListDetails;
|
||||
import com.azure.storage.blob.models.ListBlobsOptions;
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Event Position Provider using Azure Blob Storage implemented in Azure Event Hubs SDK Version 3
|
||||
*/
|
||||
public class LegacyBlobStorageEventPositionProvider implements EventPositionProvider {
|
||||
private static final String LEASE_SEQUENCE_NUMBER_FIELD = "sequenceNumber";
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(LegacyBlobStorageEventPositionProvider.class);
|
||||
|
||||
private static final ObjectMapper objectMapper = new ObjectMapper();
|
||||
|
||||
private final BlobContainerAsyncClient blobContainerAsyncClient;
|
||||
|
||||
private final String consumerGroup;
|
||||
|
||||
public LegacyBlobStorageEventPositionProvider(
|
||||
final BlobContainerAsyncClient blobContainerAsyncClient,
|
||||
final String consumerGroup
|
||||
) {
|
||||
this.blobContainerAsyncClient = Objects.requireNonNull(blobContainerAsyncClient, "Client required");
|
||||
this.consumerGroup = Objects.requireNonNull(consumerGroup, "Consumer Group required");
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Initial Partition Event Position using Azure Blob Storage as persisted in
|
||||
* com.microsoft.azure.eventprocessorhost.AzureStorageCheckpointLeaseManager
|
||||
*
|
||||
* @return Map of Partition and Event Position or empty when no checkpoints found
|
||||
*/
|
||||
@Override
|
||||
public Map<String, EventPosition> getInitialPartitionEventPosition() {
|
||||
final Map<String, EventPosition> partitionEventPosition;
|
||||
|
||||
if (containerExists()) {
|
||||
final BlobListDetails blobListDetails = new BlobListDetails().setRetrieveMetadata(true);
|
||||
final ListBlobsOptions listBlobsOptions = new ListBlobsOptions().setPrefix(consumerGroup).setDetails(blobListDetails);
|
||||
final Iterable<BlobItem> blobItems = blobContainerAsyncClient.listBlobs(listBlobsOptions).toIterable();
|
||||
partitionEventPosition = getPartitionEventPosition(blobItems);
|
||||
} else {
|
||||
partitionEventPosition = Collections.emptyMap();
|
||||
}
|
||||
|
||||
return partitionEventPosition;
|
||||
}
|
||||
|
||||
private Map<String, EventPosition> getPartitionEventPosition(final Iterable<BlobItem> blobItems) {
|
||||
final Map<String, EventPosition> partitionEventPosition = new LinkedHashMap<>();
|
||||
|
||||
for (final BlobItem blobItem : blobItems) {
|
||||
if (Boolean.TRUE.equals(blobItem.isPrefix())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
final String partitionId = getPartitionId(blobItem);
|
||||
final EventPosition eventPosition = getEventPosition(blobItem);
|
||||
if (eventPosition == null) {
|
||||
logger.info("Legacy Event Position not found for Partition [{}] Blob [{}]", partitionId, blobItem.getName());
|
||||
} else {
|
||||
partitionEventPosition.put(partitionId, eventPosition);
|
||||
}
|
||||
}
|
||||
|
||||
return partitionEventPosition;
|
||||
}
|
||||
|
||||
private String getPartitionId(final BlobItem blobItem) {
|
||||
final String blobItemName = blobItem.getName();
|
||||
final Path blobItemPath = Paths.get(blobItemName);
|
||||
final Path blobItemFileName = blobItemPath.getFileName();
|
||||
return blobItemFileName.toString();
|
||||
}
|
||||
|
||||
private EventPosition getEventPosition(final BlobItem blobItem) {
|
||||
final EventPosition eventPosition;
|
||||
|
||||
final String blobName = blobItem.getName();
|
||||
final BlobAsyncClient blobAsyncClient = blobContainerAsyncClient.getBlobAsyncClient(blobName);
|
||||
|
||||
if (itemExists(blobAsyncClient)) {
|
||||
final BinaryData content = blobAsyncClient.downloadContent().block();
|
||||
if (content == null) {
|
||||
throw new IllegalStateException(String.format("Legacy Event Position content not found [%s]", blobName));
|
||||
}
|
||||
|
||||
try {
|
||||
// Read com.microsoft.azure.eventprocessorhost.AzureBlobLease from JSON
|
||||
final JsonNode lease = objectMapper.readTree(content.toBytes());
|
||||
if (lease.hasNonNull(LEASE_SEQUENCE_NUMBER_FIELD)) {
|
||||
final JsonNode sequenceNumberField = lease.get(LEASE_SEQUENCE_NUMBER_FIELD);
|
||||
final long sequenceNumber = sequenceNumberField.asLong();
|
||||
eventPosition = EventPosition.fromSequenceNumber(sequenceNumber);
|
||||
} else {
|
||||
eventPosition = null;
|
||||
}
|
||||
} catch (final IOException e) {
|
||||
throw new UncheckedIOException(String.format("Reading Legacy Event Position Failed [%s]", blobName), e);
|
||||
}
|
||||
} else {
|
||||
logger.info("Legacy Event Position not found [{}]", blobName);
|
||||
eventPosition = null;
|
||||
}
|
||||
|
||||
return eventPosition;
|
||||
}
|
||||
|
||||
private boolean containerExists() {
|
||||
return Boolean.TRUE.equals(blobContainerAsyncClient.exists().block());
|
||||
}
|
||||
|
||||
private boolean itemExists(final BlobAsyncClient blobAsyncClient) {
|
||||
return Boolean.TRUE.equals(blobAsyncClient.exists().block());
|
||||
}
|
||||
}
|
|
@ -21,9 +21,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.HashMap;
|
||||
|
||||
import com.microsoft.azure.eventhubs.ConnectionStringBuilder;
|
||||
import com.microsoft.azure.eventhubs.EventData;
|
||||
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
|
@ -33,7 +30,6 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
|
||||
public final class AzureEventHubUtils {
|
||||
|
||||
public static final String MANAGED_IDENTITY_POLICY = ConnectionStringBuilder.MANAGED_IDENTITY_AUTHENTICATION;
|
||||
public static final AllowableValue AZURE_ENDPOINT = new AllowableValue(".servicebus.windows.net","Azure", "Servicebus endpoint for general use");
|
||||
public static final AllowableValue AZURE_CHINA_ENDPOINT = new AllowableValue(".servicebus.chinacloudapi.cn", "Azure China", "Servicebus endpoint for China");
|
||||
public static final AllowableValue AZURE_GERMANY_ENDPOINT = new AllowableValue(".servicebus.cloudapi.de", "Azure Germany", "Servicebus endpoint for Germany");
|
||||
|
@ -95,35 +91,15 @@ public final class AzureEventHubUtils {
|
|||
return retVal;
|
||||
}
|
||||
|
||||
public static String getManagedIdentityConnectionString(final String namespace, final String domainName, final String eventHubName){
|
||||
return new ConnectionStringBuilder()
|
||||
.setEndpoint(namespace, removeStartingDotFrom(domainName))
|
||||
.setEventHubName(eventHubName)
|
||||
.setAuthentication(MANAGED_IDENTITY_POLICY).toString();
|
||||
}
|
||||
public static String getSharedAccessSignatureConnectionString(final String namespace, final String domainName, final String eventHubName, final String sasName, final String sasKey) {
|
||||
return new ConnectionStringBuilder()
|
||||
.setEndpoint(namespace, removeStartingDotFrom(domainName))
|
||||
.setEventHubName(eventHubName)
|
||||
.setSasKeyName(sasName)
|
||||
.setSasKey(sasKey).toString();
|
||||
}
|
||||
|
||||
public static Map<String, String> getApplicationProperties(EventData eventData) {
|
||||
public static Map<String, String> getApplicationProperties(final Map<String,Object> eventProperties) {
|
||||
final Map<String, String> properties = new HashMap<>();
|
||||
|
||||
final Map<String,Object> applicationProperties = eventData.getProperties();
|
||||
if (null != applicationProperties) {
|
||||
for (Map.Entry<String, Object> property : applicationProperties.entrySet()) {
|
||||
if (eventProperties != null) {
|
||||
for (Map.Entry<String, Object> property : eventProperties.entrySet()) {
|
||||
properties.put(String.format("eventhub.property.%s", property.getKey()), property.getValue().toString());
|
||||
}
|
||||
}
|
||||
|
||||
return properties;
|
||||
}
|
||||
|
||||
private static String removeStartingDotFrom(final String domainName) {
|
||||
return domainName.replaceFirst("^\\.", "");
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -12,6 +12,7 @@
|
|||
# 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.
|
||||
org.apache.nifi.services.azure.eventhub.AzureEventHubRecordSink
|
||||
org.apache.nifi.services.azure.storage.AzureStorageCredentialsControllerService
|
||||
org.apache.nifi.services.azure.storage.AzureStorageCredentialsControllerServiceLookup
|
||||
org.apache.nifi.services.azure.storage.ADLSCredentialsControllerService
|
||||
|
|
|
@ -20,11 +20,10 @@ import com.azure.cosmos.ConsistencyLevel;
|
|||
import com.azure.cosmos.CosmosClient;
|
||||
import com.azure.cosmos.CosmosContainer;
|
||||
import com.azure.cosmos.CosmosException;
|
||||
import com.google.gson.Gson;
|
||||
import com.google.gson.GsonBuilder;
|
||||
import com.google.gson.JsonArray;
|
||||
import com.google.gson.JsonObject;
|
||||
import net.minidev.json.JSONObject;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.node.ArrayNode;
|
||||
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.nifi.avro.AvroTypeUtil;
|
||||
import org.apache.nifi.json.JsonTreeReader;
|
||||
|
@ -183,47 +182,49 @@ public class PutAzureCosmosDBRecordTest extends MockTestBase {
|
|||
|
||||
@Test
|
||||
public void testArrayConversion() throws Exception {
|
||||
Gson gson = new GsonBuilder().setPrettyPrinting().serializeNulls().create();
|
||||
final ObjectMapper objectMapper = new ObjectMapper();
|
||||
final JsonNodeFactory nodeFactory = objectMapper.getNodeFactory();
|
||||
|
||||
// schema creation for test
|
||||
JsonObject schemaDef = new JsonObject();
|
||||
schemaDef.addProperty("type", "record");
|
||||
schemaDef.addProperty("name", "Test");
|
||||
JsonArray schemaArray = new JsonArray();
|
||||
JsonObject f1 = new JsonObject();
|
||||
f1.addProperty("type", "string");
|
||||
f1.addProperty("name", "id");
|
||||
final ObjectNode schemaDef = nodeFactory.objectNode();
|
||||
schemaDef.put("type", "record");
|
||||
schemaDef.put("name", "Test");
|
||||
final ArrayNode schemaArray = nodeFactory.arrayNode();
|
||||
final ObjectNode f1 = nodeFactory.objectNode();
|
||||
f1.put("type", "string");
|
||||
f1.put("name", "id");
|
||||
schemaArray.add(f1);
|
||||
JsonObject f2 = new JsonObject();
|
||||
f2.addProperty("type", "string");
|
||||
f2.addProperty("name", "name");
|
||||
final ObjectNode f2 = nodeFactory.objectNode();
|
||||
f2.put("type", "string");
|
||||
f2.put("name", "name");
|
||||
schemaArray.add(f2);
|
||||
JsonObject f3 = new JsonObject();
|
||||
f3.addProperty("type", "string");
|
||||
f3.addProperty("name", "sport");
|
||||
final ObjectNode f3 = nodeFactory.objectNode();
|
||||
f3.put("type", "string");
|
||||
f3.put("name", "sport");
|
||||
schemaArray.add(f3);
|
||||
JsonObject arrayDef = new JsonObject();
|
||||
arrayDef.addProperty("type", "array");
|
||||
arrayDef.addProperty("items", "string");
|
||||
JsonObject f4 = new JsonObject();
|
||||
f4.add("type", arrayDef);
|
||||
f4.addProperty("name", "arrayTest");
|
||||
final ObjectNode arrayDef = nodeFactory.objectNode();
|
||||
arrayDef.put("type", "array");
|
||||
arrayDef.put("items", "string");
|
||||
final ObjectNode f4 = nodeFactory.objectNode();
|
||||
f4.set("type", arrayDef);
|
||||
f4.put("name", "arrayTest");
|
||||
schemaArray.add(f4);
|
||||
schemaDef.add("fields", schemaArray);
|
||||
schemaDef.set("fields", schemaArray);
|
||||
|
||||
// test data generation
|
||||
JsonObject testData = new JsonObject();
|
||||
testData.addProperty("id", UUID.randomUUID().toString());
|
||||
testData.addProperty("name", "John Doe");
|
||||
testData.addProperty("sport", "Soccer");
|
||||
JsonArray jarray = new JsonArray();
|
||||
final ObjectNode testData = nodeFactory.objectNode();
|
||||
testData.put("id", UUID.randomUUID().toString());
|
||||
testData.put("name", "John Doe");
|
||||
testData.put("sport", "Soccer");
|
||||
final ArrayNode jarray = nodeFactory.arrayNode();
|
||||
jarray.add("a");
|
||||
jarray.add("b");
|
||||
jarray.add("c");
|
||||
testData.add("arrayTest", jarray);
|
||||
testData.set("arrayTest", jarray);
|
||||
|
||||
// setup registry and reader
|
||||
MockSchemaRegistry registry = new MockSchemaRegistry();
|
||||
RecordSchema rschema = AvroTypeUtil.createSchema(new Schema.Parser().parse(gson.toJson(schemaDef)));
|
||||
RecordSchema rschema = AvroTypeUtil.createSchema(new Schema.Parser().parse(schemaDef.toPrettyString()));
|
||||
registry.addSchema("test", rschema);
|
||||
JsonTreeReader reader = new JsonTreeReader();
|
||||
testRunner.addControllerService("registry", registry);
|
||||
|
@ -239,7 +240,7 @@ public class PutAzureCosmosDBRecordTest extends MockTestBase {
|
|||
Map<String, String> attrs = new HashMap<>();
|
||||
attrs.put("schema.name", "test");
|
||||
|
||||
testRunner.enqueue(gson.toJson(testData), attrs);
|
||||
testRunner.enqueue(testData.toPrettyString(), attrs);
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertTransferCount(PutAzureCosmosDBRecord.REL_FAILURE, 0);
|
||||
|
@ -247,8 +248,7 @@ public class PutAzureCosmosDBRecordTest extends MockTestBase {
|
|||
List<Map<String, Object>> backendData = processor.getTestResults();
|
||||
assertEquals(1, backendData.size());
|
||||
//validate array data
|
||||
JSONObject arrayTestResult = new JSONObject();
|
||||
arrayTestResult.putAll(backendData.get(0));
|
||||
final Map<?, ?> arrayTestResult = backendData.get(0);
|
||||
Object[] check = (Object []) arrayTestResult.get("arrayTest");
|
||||
assertArrayEquals(new Object[]{"a", "b", "c"}, check);
|
||||
}
|
||||
|
@ -262,7 +262,7 @@ class MockPutAzureCosmosDBRecord extends PutAzureCosmosDBRecord {
|
|||
|
||||
static CosmosClient mockClient = mock(CosmosClient.class);
|
||||
static CosmosContainer mockContainer = mock(CosmosContainer.class);
|
||||
private List<Map<String, Object>> mockBackend = new ArrayList<>();
|
||||
private final List<Map<String, Object>> mockBackend = new ArrayList<>();
|
||||
|
||||
@Override
|
||||
protected void createCosmosClient(final String uri, final String accessKey, final ConsistencyLevel clevel) {
|
||||
|
@ -281,9 +281,4 @@ class MockPutAzureCosmosDBRecord extends PutAzureCosmosDBRecord {
|
|||
public List<Map<String, Object>> getTestResults() {
|
||||
return mockBackend;
|
||||
}
|
||||
|
||||
|
||||
public CosmosContainer getMockConainer() {
|
||||
return mockContainer;
|
||||
}
|
||||
}
|
|
@ -16,236 +16,149 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.azure.eventhub;
|
||||
|
||||
import com.microsoft.azure.eventhubs.EventData;
|
||||
import com.microsoft.azure.eventhubs.EventData.SystemProperties;
|
||||
import com.microsoft.azure.eventhubs.EventHubException;
|
||||
import com.microsoft.azure.eventhubs.PartitionReceiver;
|
||||
import com.microsoft.azure.eventhubs.impl.AmqpConstants;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import com.azure.messaging.eventhubs.EventData;
|
||||
import com.azure.messaging.eventhubs.models.LastEnqueuedEventProperties;
|
||||
import com.azure.messaging.eventhubs.models.PartitionContext;
|
||||
import com.azure.messaging.eventhubs.models.PartitionEvent;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Clock;
|
||||
import java.time.Instant;
|
||||
import java.time.ZoneId;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
||||
public class GetAzureEventHubTest {
|
||||
private static final String namespaceName = "nifi-azure-hub";
|
||||
private static final String eventHubName = "get-test";
|
||||
private static final String sasKeyName = "bogus-policy";
|
||||
private static final String sasKey = "9rHmHqxoOVWOb8wS09dvqXYxnNiLqxNMCbmt6qMaQyU!";
|
||||
private static final Date ENQUEUED_TIME_VALUE = Date.from(Clock.fixed(Instant.now(), ZoneId.systemDefault()).instant());
|
||||
public static final long SEQUENCE_NUMBER_VALUE = 13L;
|
||||
public static final String OFFSET_VALUE = "100";
|
||||
public static final String PARTITION_KEY_VALUE = "0";
|
||||
private static final String DOMAIN_NAME = "servicebus";
|
||||
private static final String EVENT_HUB_NAMESPACE = "NAMESPACE";
|
||||
private static final String EVENT_HUB_NAME = "NAME";
|
||||
private static final String POLICY_NAME = "POLICY";
|
||||
private static final String POLICY_KEY = "POLICY-KEY";
|
||||
private static final String CONSUMER_GROUP = "$Default";
|
||||
private static final Instant ENQUEUED_TIME = Instant.now();
|
||||
private static final long SEQUENCE_NUMBER = 32;
|
||||
private static final long OFFSET = 64;
|
||||
private static final String PARTITION_ID = "0";
|
||||
private static final String CONTENT = String.class.getSimpleName();
|
||||
|
||||
private List<PartitionEvent> partitionEvents;
|
||||
|
||||
private TestRunner testRunner;
|
||||
private MockGetAzureEventHub processor;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
processor = new MockGetAzureEventHub();
|
||||
testRunner = TestRunners.newTestRunner(processor);
|
||||
partitionEvents = new ArrayList<>();
|
||||
testRunner = TestRunners.newTestRunner(new MockGetAzureEventHub());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessorConfigValidity() {
|
||||
testRunner.setProperty(GetAzureEventHub.EVENT_HUB_NAME,eventHubName);
|
||||
public void testProperties() {
|
||||
testRunner.setProperty(GetAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(GetAzureEventHub.NAMESPACE,namespaceName);
|
||||
testRunner.setProperty(GetAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(GetAzureEventHub.ACCESS_POLICY,sasKeyName);
|
||||
testRunner.setProperty(GetAzureEventHub.ACCESS_POLICY, POLICY_NAME);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(GetAzureEventHub.POLICY_PRIMARY_KEY,sasKey);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(GetAzureEventHub.NUM_PARTITIONS,"4");
|
||||
testRunner.setProperty(GetAzureEventHub.POLICY_PRIMARY_KEY, POLICY_KEY);
|
||||
testRunner.assertValid();
|
||||
testRunner.setProperty(GetAzureEventHub.ENQUEUE_TIME,"2015-12-22T21:55:10.000Z");
|
||||
testRunner.setProperty(GetAzureEventHub.ENQUEUE_TIME, ENQUEUED_TIME.toString());
|
||||
testRunner.assertValid();
|
||||
testRunner.setProperty(GetAzureEventHub.RECEIVER_FETCH_SIZE, "5");
|
||||
testRunner.assertValid();
|
||||
testRunner.setProperty(GetAzureEventHub.RECEIVER_FETCH_TIMEOUT,"10000");
|
||||
testRunner.setProperty(GetAzureEventHub.RECEIVER_FETCH_TIMEOUT, "10000");
|
||||
testRunner.assertValid();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessorConfigValidityWithManagedIdentityFlag() {
|
||||
testRunner.setProperty(PutAzureEventHub.EVENT_HUB_NAME,eventHubName);
|
||||
public void testPropertiesManagedIdentity() {
|
||||
testRunner.setProperty(GetAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(PutAzureEventHub.NAMESPACE,namespaceName);
|
||||
testRunner.setProperty(GetAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(PutAzureEventHub.USE_MANAGED_IDENTITY,"true");
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(GetAzureEventHub.NUM_PARTITIONS,"4");
|
||||
testRunner.setProperty(PutAzureEventHub.USE_MANAGED_IDENTITY, Boolean.TRUE.toString());
|
||||
testRunner.assertValid();
|
||||
testRunner.setProperty(GetAzureEventHub.ENQUEUE_TIME,"2015-12-22T21:55:10.000Z");
|
||||
testRunner.assertValid();
|
||||
testRunner.setProperty(GetAzureEventHub.RECEIVER_FETCH_SIZE, "5");
|
||||
testRunner.assertValid();
|
||||
testRunner.setProperty(GetAzureEventHub.RECEIVER_FETCH_TIMEOUT,"10000");
|
||||
testRunner.assertValid();
|
||||
}
|
||||
@Test
|
||||
public void verifyRelationships(){
|
||||
assert(1 == processor.getRelationships().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoPartitions(){
|
||||
MockGetAzureEventHubNoPartitions mockProcessor = new MockGetAzureEventHubNoPartitions();
|
||||
testRunner = TestRunners.newTestRunner(mockProcessor);
|
||||
setUpStandardTestConfig();
|
||||
testRunner.run(1, true);
|
||||
public void testRunNoEventsReceived(){
|
||||
setProperties();
|
||||
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(GetAzureEventHub.REL_SUCCESS, 0);
|
||||
testRunner.clearTransferState();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNullRecieve(){
|
||||
setUpStandardTestConfig();
|
||||
processor.nullReceive = true;
|
||||
testRunner.run(1, true);
|
||||
testRunner.assertAllFlowFilesTransferred(GetAzureEventHub.REL_SUCCESS, 0);
|
||||
testRunner.clearTransferState();
|
||||
public void testRunEventsReceived() {
|
||||
setProperties();
|
||||
|
||||
final PartitionEvent partitionEvent = createPartitionEvent();
|
||||
partitionEvents.add(partitionEvent);
|
||||
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(GetAzureEventHub.REL_SUCCESS, 1);
|
||||
|
||||
final MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(GetAzureEventHub.REL_SUCCESS).get(0);
|
||||
flowFile.assertContentEquals(CONTENT);
|
||||
flowFile.assertAttributeEquals("eventhub.enqueued.timestamp", ENQUEUED_TIME.toString());
|
||||
flowFile.assertAttributeEquals("eventhub.offset", Long.toString(OFFSET));
|
||||
flowFile.assertAttributeEquals("eventhub.sequence", Long.toString(SEQUENCE_NUMBER));
|
||||
flowFile.assertAttributeEquals("eventhub.name", EVENT_HUB_NAME);
|
||||
}
|
||||
|
||||
@Test//(expected = AssertionError.class)
|
||||
public void testThrowGetReceiver(){
|
||||
setUpStandardTestConfig();
|
||||
processor.getReceiverThrow = true;
|
||||
assertThrows(AssertionError.class, () -> testRunner.run(1, true));
|
||||
testRunner.assertAllFlowFilesTransferred(GetAzureEventHub.REL_SUCCESS, 0);
|
||||
testRunner.clearTransferState();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNormalFlow() throws Exception {
|
||||
setUpStandardTestConfig();
|
||||
testRunner.run(1, true);
|
||||
testRunner.assertAllFlowFilesTransferred(GetAzureEventHub.REL_SUCCESS, 10);
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(GetAzureEventHub.REL_SUCCESS).get(0);
|
||||
flowFile.assertContentEquals("test event number: 0");
|
||||
flowFile.assertAttributeEquals("eventhub.enqueued.timestamp", ENQUEUED_TIME_VALUE.toInstant().toString());
|
||||
flowFile.assertAttributeEquals("eventhub.offset", OFFSET_VALUE);
|
||||
flowFile.assertAttributeEquals("eventhub.sequence", String.valueOf(SEQUENCE_NUMBER_VALUE));
|
||||
flowFile.assertAttributeEquals("eventhub.name", eventHubName);
|
||||
|
||||
testRunner.clearTransferState();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNormalFlowWithApplicationProperties() throws Exception {
|
||||
setUpStandardTestConfig();
|
||||
testRunner.run(1, true);
|
||||
testRunner.assertAllFlowFilesTransferred(GetAzureEventHub.REL_SUCCESS, 10);
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(GetAzureEventHub.REL_SUCCESS).get(0);
|
||||
flowFile.assertAttributeEquals("eventhub.property.event-sender", "Apache NiFi");
|
||||
flowFile.assertAttributeEquals("eventhub.property.application", "TestApp");
|
||||
|
||||
testRunner.clearTransferState();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNormalNotReceivedEventsFlow() throws Exception {
|
||||
setUpStandardTestConfig();
|
||||
processor.received = false;
|
||||
testRunner.run(1, true);
|
||||
testRunner.assertAllFlowFilesTransferred(GetAzureEventHub.REL_SUCCESS, 10);
|
||||
|
||||
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(GetAzureEventHub.REL_SUCCESS).get(0);
|
||||
flowFile.assertContentEquals("test event number: 0");
|
||||
flowFile.assertAttributeNotExists("eventhub.enqueued.timestamp");
|
||||
flowFile.assertAttributeNotExists("eventhub.offset");
|
||||
flowFile.assertAttributeNotExists("eventhub.sequence");
|
||||
flowFile.assertAttributeEquals("eventhub.name", eventHubName);
|
||||
|
||||
testRunner.clearTransferState();
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a stubbed processor instance for testing
|
||||
*/
|
||||
public static class MockGetAzureEventHub extends GetAzureEventHub{
|
||||
|
||||
boolean nullReceive = false;
|
||||
boolean getReceiverThrow = false;
|
||||
boolean received = true;
|
||||
private class MockGetAzureEventHub extends GetAzureEventHub {
|
||||
|
||||
@Override
|
||||
protected void setupReceiver(final String connectionString, final ScheduledExecutorService executor) throws ProcessException{
|
||||
//do nothing
|
||||
}
|
||||
@Override
|
||||
protected PartitionReceiver getReceiver(final ProcessContext context, final String partitionId) throws IOException, EventHubException, ExecutionException, InterruptedException {
|
||||
if(getReceiverThrow){
|
||||
throw new IOException("Could not create receiver");
|
||||
}
|
||||
return null;
|
||||
protected BlockingQueue<String> getPartitionIds() {
|
||||
return new LinkedBlockingQueue<>(Collections.singleton(PARTITION_ID));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Iterable<EventData> receiveEvents(final ProcessContext context, final String partitionId) throws ProcessException{
|
||||
if(nullReceive){
|
||||
return null;
|
||||
}
|
||||
if(getReceiverThrow){
|
||||
throw new ProcessException("Could not create receiver");
|
||||
}
|
||||
final LinkedList<EventData> receivedEvents = new LinkedList<>();
|
||||
for(int i = 0; i < 10; i++){
|
||||
EventData eventData = EventData.create(String.format("test event number: %d", i).getBytes());
|
||||
eventData.getProperties().put("event-sender", "Apache NiFi");
|
||||
eventData.getProperties().put("application", "TestApp");
|
||||
if (received) {
|
||||
HashMap<String, Object> properties = new HashMap<>();
|
||||
properties.put(AmqpConstants.PARTITION_KEY_ANNOTATION_NAME, PARTITION_KEY_VALUE);
|
||||
properties.put(AmqpConstants.OFFSET_ANNOTATION_NAME, OFFSET_VALUE);
|
||||
properties.put(AmqpConstants.SEQUENCE_NUMBER_ANNOTATION_NAME, SEQUENCE_NUMBER_VALUE);
|
||||
properties.put(AmqpConstants.ENQUEUED_TIME_UTC_ANNOTATION_NAME, ENQUEUED_TIME_VALUE);
|
||||
|
||||
SystemProperties systemProperties = new SystemProperties(properties);
|
||||
eventData.setSystemProperties(systemProperties);
|
||||
}
|
||||
receivedEvents.add(eventData);
|
||||
}
|
||||
|
||||
return receivedEvents;
|
||||
protected Iterable<PartitionEvent> receiveEvents(final String partitionId) {
|
||||
return partitionEvents;
|
||||
}
|
||||
}
|
||||
|
||||
public static class MockGetAzureEventHubNoPartitions extends GetAzureEventHub{
|
||||
@Override
|
||||
protected void setupReceiver(final String connectionString, final ScheduledExecutorService executor) throws ProcessException{
|
||||
//do nothing
|
||||
}
|
||||
private PartitionEvent createPartitionEvent() {
|
||||
final PartitionContext partitionContext = new PartitionContext(DOMAIN_NAME, EVENT_HUB_NAME, CONSUMER_GROUP, PARTITION_ID);
|
||||
final EventData eventData = new MockEventData();
|
||||
|
||||
@Override
|
||||
public void onScheduled(final ProcessContext context) throws ProcessException {
|
||||
|
||||
}
|
||||
@Override
|
||||
public void tearDown() throws ProcessException {
|
||||
}
|
||||
final LastEnqueuedEventProperties lastEnqueuedEventProperties = new LastEnqueuedEventProperties(SEQUENCE_NUMBER, OFFSET, ENQUEUED_TIME, ENQUEUED_TIME);
|
||||
return new PartitionEvent(partitionContext, eventData, lastEnqueuedEventProperties);
|
||||
}
|
||||
private void setUpStandardTestConfig() {
|
||||
testRunner.setProperty(GetAzureEventHub.EVENT_HUB_NAME,eventHubName);
|
||||
testRunner.setProperty(GetAzureEventHub.NAMESPACE,namespaceName);
|
||||
testRunner.setProperty(GetAzureEventHub.ACCESS_POLICY,sasKeyName);
|
||||
testRunner.setProperty(GetAzureEventHub.POLICY_PRIMARY_KEY,sasKey);
|
||||
testRunner.setProperty(GetAzureEventHub.NUM_PARTITIONS,"4");
|
||||
|
||||
private void setProperties() {
|
||||
testRunner.setProperty(GetAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.setProperty(GetAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.setProperty(GetAzureEventHub.ACCESS_POLICY, POLICY_NAME);
|
||||
testRunner.setProperty(GetAzureEventHub.POLICY_PRIMARY_KEY, POLICY_KEY);
|
||||
testRunner.assertValid();
|
||||
}
|
||||
|
||||
private static class MockEventData extends EventData {
|
||||
|
||||
private MockEventData() {
|
||||
super(CONTENT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getOffset() {
|
||||
return OFFSET;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getSequenceNumber() {
|
||||
return SEQUENCE_NUMBER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Instant getEnqueuedTime() {
|
||||
return ENQUEUED_TIME;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,376 +16,156 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.azure.eventhub;
|
||||
|
||||
import com.microsoft.azure.eventhubs.EventData;
|
||||
import com.microsoft.azure.eventhubs.EventHubClient;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import com.azure.messaging.eventhubs.EventHubProducerClient;
|
||||
import com.azure.messaging.eventhubs.models.SendOptions;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processors.azure.storage.utils.FlowFileResultCarrier;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.StopWatch;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.ArgumentCaptor;
|
||||
import org.mockito.Captor;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.MockitoAnnotations;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyString;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.Mockito.doNothing;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.never;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.ArgumentMatchers.anyIterable;
|
||||
import static org.mockito.Mockito.doThrow;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
public class PutAzureEventHubTest {
|
||||
private static final String namespaceName = "nifi-azure-hub";
|
||||
private static final String eventHubName = "get-test";
|
||||
private static final String sasKeyName = "bogus-policy";
|
||||
private static final String sasKey = "9rHmHqxoOVWOb8wS09dvqXYxnNiLqxNMCbmt6qMaQyU!";
|
||||
private static final String TEST_PARTITIONING_KEY_ATTRIBUTE_NAME = "x-opt-partition-key";
|
||||
private static final String TEST_PARTITIONING_KEY = "some-partitioning-key";
|
||||
private static final String EVENT_HUB_NAMESPACE = "NAMESPACE";
|
||||
private static final String EVENT_HUB_NAME = "NAME";
|
||||
private static final String POLICY_NAME = "POLICY";
|
||||
private static final String POLICY_KEY = "POLICY-KEY";
|
||||
private static final String PARTITION_KEY_ATTRIBUTE_NAME = "eventPartitionKey";
|
||||
private static final String PARTITION_KEY = "partition";
|
||||
private static final String CONTENT = String.class.getSimpleName();
|
||||
|
||||
@Mock
|
||||
EventHubProducerClient eventHubProducerClient;
|
||||
|
||||
private TestRunner testRunner;
|
||||
private PutAzureEventHubTest.MockPutAzureEventHub processor;
|
||||
@Captor
|
||||
ArgumentCaptor<SendOptions> sendOptionsArgumentCaptor;
|
||||
|
||||
TestRunner testRunner;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
processor = new PutAzureEventHubTest.MockPutAzureEventHub();
|
||||
testRunner = TestRunners.newTestRunner(processor);
|
||||
testRunner = TestRunners.newTestRunner(new MockPutAzureEventHub());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessorConfigValidity() {
|
||||
testRunner.setProperty(PutAzureEventHub.EVENT_HUB_NAME,eventHubName);
|
||||
public void testProperties() {
|
||||
testRunner.setProperty(PutAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(PutAzureEventHub.NAMESPACE,namespaceName);
|
||||
testRunner.setProperty(PutAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(PutAzureEventHub.ACCESS_POLICY,sasKeyName);
|
||||
testRunner.setProperty(PutAzureEventHub.ACCESS_POLICY, POLICY_NAME);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(PutAzureEventHub.POLICY_PRIMARY_KEY,sasKey);
|
||||
testRunner.setProperty(PutAzureEventHub.POLICY_PRIMARY_KEY, POLICY_KEY);
|
||||
testRunner.assertValid();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessorConfigValidityWithManagedIdentityFlag() {
|
||||
testRunner.setProperty(PutAzureEventHub.EVENT_HUB_NAME,eventHubName);
|
||||
public void testPropertiesManagedIdentityEnabled() {
|
||||
testRunner.setProperty(PutAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(PutAzureEventHub.NAMESPACE,namespaceName);
|
||||
testRunner.setProperty(PutAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(PutAzureEventHub.USE_MANAGED_IDENTITY,"true");
|
||||
testRunner.setProperty(PutAzureEventHub.USE_MANAGED_IDENTITY, Boolean.TRUE.toString());
|
||||
testRunner.assertValid();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void verifyRelationships(){
|
||||
assert(2 == processor.getRelationships().size());
|
||||
}
|
||||
@Test
|
||||
public void testNoFlow() {
|
||||
setUpStandardTestConfig();
|
||||
testRunner.run(1, true);
|
||||
public void testRunNoFlowFiles() {
|
||||
setProperties();
|
||||
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(PutAzureEventHub.REL_SUCCESS, 0);
|
||||
testRunner.clearTransferState();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNormalFlow(){
|
||||
setUpStandardTestConfig();
|
||||
String flowFileContents = "TEST MESSAGE";
|
||||
testRunner.enqueue(flowFileContents);
|
||||
testRunner.run(1, true);
|
||||
assert(flowFileContents.contentEquals(new String(processor.getReceivedBuffer())));
|
||||
public void testRunSuccess(){
|
||||
setProperties();
|
||||
|
||||
testRunner.enqueue(CONTENT);
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(PutAzureEventHub.REL_SUCCESS, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunFailure() {
|
||||
setProperties();
|
||||
|
||||
doThrow(new RuntimeException()).when(eventHubProducerClient).send(anyIterable(), any(SendOptions.class));
|
||||
|
||||
testRunner.enqueue(CONTENT);
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(PutAzureEventHub.REL_FAILURE, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunBatchSuccess(){
|
||||
setProperties();
|
||||
|
||||
final int batchSize = 2;
|
||||
|
||||
testRunner.setProperty(PutAzureEventHub.MAX_BATCH_SIZE, Integer.toString(batchSize));
|
||||
|
||||
testRunner.enqueue(CONTENT);
|
||||
testRunner.enqueue(CONTENT);
|
||||
testRunner.enqueue(CONTENT);
|
||||
testRunner.enqueue(CONTENT);
|
||||
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(PutAzureEventHub.REL_SUCCESS, batchSize);
|
||||
testRunner.clearTransferState();
|
||||
}
|
||||
@Test
|
||||
public void testSendMessageThrows() {
|
||||
PutAzureEventHubTest.OnSendThrowingMockPutAzureEventHub throwingProcessor = new PutAzureEventHubTest.OnSendThrowingMockPutAzureEventHub();
|
||||
testRunner = TestRunners.newTestRunner(throwingProcessor);
|
||||
setUpStandardTestConfig();
|
||||
String flowFileContents = "TEST MESSAGE";
|
||||
testRunner.enqueue(flowFileContents);
|
||||
testRunner.run(1, true);
|
||||
testRunner.assertAllFlowFilesTransferred(PutAzureEventHub.REL_FAILURE);
|
||||
testRunner.clearTransferState();
|
||||
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(PutAzureEventHub.REL_SUCCESS, batchSize);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBadConnectionString() {
|
||||
PutAzureEventHubTest.BogusConnectionStringMockPutAzureEventHub badConnectionStringProcessor = new PutAzureEventHubTest.BogusConnectionStringMockPutAzureEventHub();
|
||||
testRunner = TestRunners.newTestRunner(badConnectionStringProcessor);
|
||||
setUpStandardTestConfig();
|
||||
assertThrows(AssertionError.class, () -> testRunner.run(1, true));
|
||||
public void testRunSuccessPartitionKey(){
|
||||
setProperties();
|
||||
|
||||
final Map<String, String> attributes = Collections.singletonMap(PARTITION_KEY_ATTRIBUTE_NAME, PARTITION_KEY);
|
||||
testRunner.setProperty(PutAzureEventHub.PARTITIONING_KEY_ATTRIBUTE_NAME, PARTITION_KEY_ATTRIBUTE_NAME);
|
||||
|
||||
testRunner.enqueue(CONTENT, attributes);
|
||||
testRunner.run();
|
||||
|
||||
testRunner.assertAllFlowFilesTransferred(PutAzureEventHub.REL_SUCCESS, 1);
|
||||
|
||||
verify(eventHubProducerClient).send(anyIterable(), sendOptionsArgumentCaptor.capture());
|
||||
|
||||
final SendOptions sendOptions = sendOptionsArgumentCaptor.getValue();
|
||||
assertEquals(PARTITION_KEY, sendOptions.getPartitionKey());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMessageIsSentWithPartitioningKeyIfSpecifiedAndPopulated() {
|
||||
MockedEventhubClientMockPutAzureEventHub processor = new PutAzureEventHubTest.MockedEventhubClientMockPutAzureEventHub();
|
||||
MockitoAnnotations.initMocks(processor);
|
||||
|
||||
EventHubClient eventHubClient = processor.getEventHubClient();
|
||||
when(eventHubClient.send(any(EventData.class), anyString()))
|
||||
.thenReturn(CompletableFuture.completedFuture(null));
|
||||
|
||||
when(eventHubClient.send(any(EventData.class)))
|
||||
.thenThrow(new RuntimeException("Partition-key-less method called despite key is defined and required."));
|
||||
|
||||
testRunner = TestRunners.newTestRunner(processor);
|
||||
setUpStandardTestConfig();
|
||||
testRunner.setProperty(PutAzureEventHub.PARTITIONING_KEY_ATTRIBUTE_NAME, TEST_PARTITIONING_KEY_ATTRIBUTE_NAME);
|
||||
|
||||
MockFlowFile flowFile = new MockFlowFile(1234);
|
||||
flowFile.putAttributes(Collections.singletonMap(TEST_PARTITIONING_KEY_ATTRIBUTE_NAME, TEST_PARTITIONING_KEY));
|
||||
testRunner.enqueue(flowFile);
|
||||
testRunner.run(1, true);
|
||||
|
||||
Mockito.verify(eventHubClient).send(any(EventData.class), eq(TEST_PARTITIONING_KEY));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMessageIsSentWithoutPartitioningKeyIfNotSpecifiedOrNotPopulated() {
|
||||
MockedEventhubClientMockPutAzureEventHub processor = new PutAzureEventHubTest.MockedEventhubClientMockPutAzureEventHub();
|
||||
MockitoAnnotations.initMocks(processor);
|
||||
|
||||
EventHubClient eventHubClient = processor.getEventHubClient();
|
||||
when(eventHubClient.send(any(EventData.class), anyString()))
|
||||
.thenThrow(new RuntimeException("Partition-key-full method called despite key is Not required or not populated."));
|
||||
|
||||
when(eventHubClient.send(any(EventData.class)))
|
||||
.thenReturn(CompletableFuture.completedFuture(null));
|
||||
|
||||
testRunner = TestRunners.newTestRunner(processor);
|
||||
setUpStandardTestConfig();
|
||||
|
||||
MockFlowFile flowFile = new MockFlowFile(1234);
|
||||
flowFile.putAttributes(Collections.singletonMap(TEST_PARTITIONING_KEY_ATTRIBUTE_NAME, TEST_PARTITIONING_KEY));
|
||||
|
||||
// Key not specified
|
||||
testRunner.enqueue(flowFile);
|
||||
testRunner.run(1, true);
|
||||
|
||||
Mockito.verify(eventHubClient, never()).send(any(EventData.class), eq(TEST_PARTITIONING_KEY));
|
||||
Mockito.verify(eventHubClient).send(any(EventData.class));
|
||||
|
||||
// Key wanted but not available
|
||||
testRunner.setProperty(PutAzureEventHub.PARTITIONING_KEY_ATTRIBUTE_NAME, "Non-existing-attribute");
|
||||
|
||||
testRunner.enqueue(flowFile);
|
||||
testRunner.run(1, true);
|
||||
|
||||
Mockito.verify(eventHubClient, never()).send(any(EventData.class), eq(TEST_PARTITIONING_KEY));
|
||||
Mockito.verify(eventHubClient, times(2)).send(any(EventData.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAllAttributesAreLiftedToProperties() {
|
||||
MockedEventhubClientMockPutAzureEventHub processor = new PutAzureEventHubTest.MockedEventhubClientMockPutAzureEventHub();
|
||||
MockitoAnnotations.initMocks(processor);
|
||||
|
||||
EventHubClient eventHubClient = processor.getEventHubClient();
|
||||
when(eventHubClient.send(any(EventData.class)))
|
||||
.thenReturn(CompletableFuture.completedFuture(null));
|
||||
|
||||
testRunner = TestRunners.newTestRunner(processor);
|
||||
setUpStandardTestConfig();
|
||||
|
||||
MockFlowFile flowFile = new MockFlowFile(1234);
|
||||
final Map<String, String> demoAttributes = new LinkedHashMap<>();
|
||||
demoAttributes.put("A", "a");
|
||||
demoAttributes.put("B", "b");
|
||||
demoAttributes.put("D", "d");
|
||||
demoAttributes.put("C", "c");
|
||||
flowFile.putAttributes(demoAttributes);
|
||||
|
||||
testRunner.enqueue(flowFile);
|
||||
testRunner.run(1, true);
|
||||
ArgumentCaptor<EventData> eventDataCaptor = ArgumentCaptor.forClass(EventData.class);
|
||||
|
||||
Mockito.verify(eventHubClient).send(eventDataCaptor.capture());
|
||||
|
||||
EventData event = eventDataCaptor.getValue();
|
||||
assertTrue(event.getProperties().entrySet().containsAll(demoAttributes.entrySet()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBatchProcessesUptoMaximum() {
|
||||
MockedEventhubClientMockPutAzureEventHub processor = new PutAzureEventHubTest.MockedEventhubClientMockPutAzureEventHub();
|
||||
MockitoAnnotations.initMocks(processor);
|
||||
|
||||
EventHubClient eventHubClient = processor.getEventHubClient();
|
||||
|
||||
CompletableFuture<Void> failedFuture = new CompletableFuture<Void>();
|
||||
failedFuture.completeExceptionally(new IllegalArgumentException());
|
||||
|
||||
when(eventHubClient.send(any(EventData.class)))
|
||||
.thenReturn(failedFuture)
|
||||
.thenReturn(CompletableFuture.completedFuture(null));
|
||||
|
||||
testRunner = TestRunners.newTestRunner(processor);
|
||||
setUpStandardTestConfig();
|
||||
|
||||
List<MockFlowFile> flowFiles = Arrays.asList(new MockFlowFile(1), new MockFlowFile(2), new MockFlowFile(3),
|
||||
new MockFlowFile(4), new MockFlowFile(5), new MockFlowFile(6));
|
||||
|
||||
flowFiles.stream().forEachOrdered(ff -> testRunner.enqueue(ff));
|
||||
|
||||
testRunner.setProperty(PutAzureEventHub.MAX_BATCH_SIZE, "4");
|
||||
testRunner.run(1, true);
|
||||
|
||||
Mockito.verify(eventHubClient, times(4)).send(any(EventData.class));
|
||||
testRunner.assertTransferCount(PutAzureEventHub.REL_SUCCESS, 3);
|
||||
testRunner.assertTransferCount(PutAzureEventHub.REL_FAILURE, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailedBatchProcessesRollsBackTransactions() throws InterruptedException, ExecutionException {
|
||||
MockedEventhubClientMockPutAzureEventHub processor = new PutAzureEventHubTest.MockedEventhubClientMockPutAzureEventHub();
|
||||
MockitoAnnotations.initMocks(processor);
|
||||
|
||||
final BlockingQueue<CompletableFuture<FlowFileResultCarrier<Relationship>>> futureQueue = new LinkedBlockingQueue<CompletableFuture<FlowFileResultCarrier<Relationship>>>();
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
CompletableFuture<FlowFileResultCarrier<Relationship>> throwingFuture = (CompletableFuture<FlowFileResultCarrier<Relationship>>)mock(CompletableFuture.class);
|
||||
|
||||
when(throwingFuture.get()).thenThrow(new ExecutionException(new IllegalArgumentException()));
|
||||
|
||||
MockFlowFile flowFile1 = new MockFlowFile(1);
|
||||
MockFlowFile flowFile2 = new MockFlowFile(2);
|
||||
|
||||
futureQueue.offer(CompletableFuture.completedFuture(null));
|
||||
futureQueue.offer(CompletableFuture.completedFuture(new FlowFileResultCarrier<Relationship>(flowFile1, PutAzureEventHub.REL_SUCCESS)));
|
||||
futureQueue.offer(CompletableFuture.completedFuture(new FlowFileResultCarrier<Relationship>(flowFile2, PutAzureEventHub.REL_FAILURE, new IllegalArgumentException())));
|
||||
futureQueue.offer(throwingFuture);
|
||||
|
||||
testRunner = TestRunners.newTestRunner(processor);
|
||||
setUpStandardTestConfig();
|
||||
testRunner.enqueue(flowFile1);
|
||||
testRunner.enqueue(flowFile2);
|
||||
|
||||
|
||||
final ProcessContext context = testRunner.getProcessContext();
|
||||
final ProcessSession session = spy(testRunner.getProcessSessionFactory().createSession());
|
||||
doNothing().when(session).transfer(any(FlowFile.class), any());
|
||||
doReturn(flowFile2).when(session).penalize(any());
|
||||
|
||||
try {
|
||||
processor.waitForAllFutures(context, session, new StopWatch(true), futureQueue);
|
||||
assertFalse(true);
|
||||
}catch(ProcessException pe) {
|
||||
assertTrue(true);
|
||||
assertFalse(Thread.currentThread().isInterrupted());
|
||||
}
|
||||
|
||||
verify(session).transfer(flowFile1, PutAzureEventHub.REL_SUCCESS);
|
||||
verify(session).transfer(flowFile2, PutAzureEventHub.REL_FAILURE);
|
||||
verify(session).rollback();
|
||||
|
||||
//Second run to test interrupted exception
|
||||
Mockito.reset(throwingFuture, session);
|
||||
when(throwingFuture.get()).thenThrow(new InterruptedException());
|
||||
doNothing().when(session).transfer(any(FlowFile.class), any());
|
||||
doReturn(flowFile2).when(session).penalize(any());
|
||||
|
||||
try {
|
||||
processor.waitForAllFutures(context, session, new StopWatch(true), futureQueue);
|
||||
assertFalse(true);
|
||||
}catch(ProcessException pe) {
|
||||
assertTrue(true);
|
||||
assertTrue(Thread.currentThread().isInterrupted());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class MockPutAzureEventHub extends PutAzureEventHub{
|
||||
byte[] receivedBuffer = null;
|
||||
|
||||
byte[] getReceivedBuffer(){
|
||||
return receivedBuffer;
|
||||
}
|
||||
|
||||
private class MockPutAzureEventHub extends PutAzureEventHub {
|
||||
|
||||
@Override
|
||||
protected EventHubClient createEventHubClient(
|
||||
final String namespace,
|
||||
final String serviceBusEndpoint,
|
||||
final String eventHubName,
|
||||
final String policyName,
|
||||
final String policyKey,
|
||||
final ScheduledExecutorService executor) throws ProcessException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected CompletableFuture<Void> sendMessage(final byte[] buffer, String partitioningKey, Map<String, Object> userProperties) throws ProcessException {
|
||||
receivedBuffer = buffer;
|
||||
|
||||
return CompletableFuture.completedFuture(null);
|
||||
protected EventHubProducerClient createEventHubProducerClient(final ProcessContext context) {
|
||||
return eventHubProducerClient;
|
||||
}
|
||||
}
|
||||
private static class OnSendThrowingMockPutAzureEventHub extends PutAzureEventHub{
|
||||
@Override
|
||||
protected EventHubClient createEventHubClient(
|
||||
final String namespace,
|
||||
final String serviceBusEndpoint,
|
||||
final String eventHubName,
|
||||
final String policyName,
|
||||
final String policyKey,
|
||||
final ScheduledExecutorService executor) throws ProcessException {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
private static class BogusConnectionStringMockPutAzureEventHub extends PutAzureEventHub{
|
||||
@Override
|
||||
protected String getConnectionString(final String namespace, final String serviceBusEndpoint, final String eventHubName, final String policyName, final String policyKey){
|
||||
return "Bogus Connection String";
|
||||
}
|
||||
}
|
||||
private static class MockedEventhubClientMockPutAzureEventHub extends PutAzureEventHub{
|
||||
|
||||
@Mock
|
||||
private EventHubClient client;
|
||||
|
||||
public EventHubClient getEventHubClient() {
|
||||
return client;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected EventHubClient createEventHubClient(
|
||||
final String namespace,
|
||||
final String serviceBusEndpoint,
|
||||
final String eventHubName,
|
||||
final String policyName,
|
||||
final String policyKey,
|
||||
final ScheduledExecutorService executor) throws ProcessException {
|
||||
return client;
|
||||
}
|
||||
}
|
||||
private void setUpStandardTestConfig() {
|
||||
testRunner.setProperty(PutAzureEventHub.EVENT_HUB_NAME,eventHubName);
|
||||
testRunner.setProperty(PutAzureEventHub.NAMESPACE,namespaceName);
|
||||
testRunner.setProperty(PutAzureEventHub.ACCESS_POLICY,sasKeyName);
|
||||
testRunner.setProperty(PutAzureEventHub.POLICY_PRIMARY_KEY,sasKey);
|
||||
private void setProperties() {
|
||||
testRunner.setProperty(PutAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.setProperty(PutAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.setProperty(PutAzureEventHub.ACCESS_POLICY, POLICY_NAME);
|
||||
testRunner.setProperty(PutAzureEventHub.POLICY_PRIMARY_KEY, POLICY_KEY);
|
||||
testRunner.assertValid();
|
||||
}
|
||||
}
|
|
@ -16,15 +16,17 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.azure.eventhub;
|
||||
|
||||
import com.microsoft.azure.eventhubs.EventData;
|
||||
import com.microsoft.azure.eventprocessorhost.PartitionContext;
|
||||
import com.azure.messaging.eventhubs.CheckpointStore;
|
||||
import com.azure.messaging.eventhubs.EventData;
|
||||
import com.azure.messaging.eventhubs.EventProcessorClient;
|
||||
import com.azure.messaging.eventhubs.models.Checkpoint;
|
||||
import com.azure.messaging.eventhubs.models.EventBatchContext;
|
||||
import com.azure.messaging.eventhubs.models.PartitionContext;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.ProcessorInitializationContext;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.schema.access.SchemaNotFoundException;
|
||||
import org.apache.nifi.serialization.MalformedRecordException;
|
||||
import org.apache.nifi.serialization.RecordReader;
|
||||
import org.apache.nifi.serialization.RecordReaderFactory;
|
||||
|
@ -38,15 +40,15 @@ import org.apache.nifi.serialization.record.MockRecordWriter;
|
|||
import org.apache.nifi.serialization.record.Record;
|
||||
import org.apache.nifi.serialization.record.RecordField;
|
||||
import org.apache.nifi.serialization.record.RecordFieldType;
|
||||
import org.apache.nifi.util.MockComponentLog;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.MockProcessSession;
|
||||
import org.apache.nifi.util.SharedSessionState;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.mockito.Mockito;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
import reactor.core.publisher.Mono;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
|
@ -56,7 +58,6 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
@ -66,62 +67,71 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
|
|||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyLong;
|
||||
import static org.mockito.ArgumentMatchers.anyMap;
|
||||
import static org.mockito.Mockito.doThrow;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
public class TestConsumeAzureEventHub {
|
||||
private static final String namespaceName = "nifi-azure-hub";
|
||||
private static final String eventHubName = "get-test";
|
||||
private static final String policyName = "test-pn";
|
||||
private static final String policyKey = "test-pk";
|
||||
private static final String storageAccountName = "test-sa";
|
||||
private static final String storageAccountKey = "test-sa-key";
|
||||
private static final String storageSasToken = "?test-sa-token";
|
||||
private static final String serviceBusEndpoint = ".endpoint";
|
||||
private static final String EVENT_HUB_NAMESPACE = "NAMESPACE";
|
||||
private static final String EVENT_HUB_NAME = "NAME";
|
||||
private static final String POLICY_NAME = "POLICY";
|
||||
private static final String POLICY_KEY = "POLICY_KEY";
|
||||
private static final String STORAGE_ACCOUNT_NAME = "STORAGE";
|
||||
private static final String STORAGE_ACCOUNT_KEY = "STORAGE_KEY";
|
||||
private static final String STORAGE_TOKEN = "?TOKEN";
|
||||
private static final String SERVICE_BUS_ENDPOINT = ".endpoint";
|
||||
private static final String CONSUMER_GROUP = "CONSUMER";
|
||||
private static final String PARTITION_ID = "0";
|
||||
private static final String FIRST_CONTENT = "CONTENT-1";
|
||||
private static final String SECOND_CONTENT = "CONTENT-2";
|
||||
private static final String THIRD_CONTENT = "CONTENT-3";
|
||||
private static final String FOURTH_CONTENT = "CONTENT-4";
|
||||
private static final String APPLICATION_PROPERTY = "application";
|
||||
private static final String APPLICATION_ATTRIBUTE_NAME = String.format("eventhub.property.%s", APPLICATION_PROPERTY);
|
||||
|
||||
private static final String EXPECTED_TRANSIT_URI = "amqps://namespace" + serviceBusEndpoint + "/" +
|
||||
"eventhub-name/ConsumerGroups/consumer-group/Partitions/partition-id";
|
||||
private static final String EXPECTED_TRANSIT_URI = String.format("amqps://%s%s/%s/ConsumerGroups/%s/Partitions/%s",
|
||||
EVENT_HUB_NAMESPACE,
|
||||
SERVICE_BUS_ENDPOINT,
|
||||
EVENT_HUB_NAME,
|
||||
CONSUMER_GROUP,
|
||||
PARTITION_ID
|
||||
);
|
||||
|
||||
private ConsumeAzureEventHub.EventProcessor eventProcessor;
|
||||
private MockProcessSession processSession;
|
||||
private SharedSessionState sharedState;
|
||||
private PartitionContext partitionContext;
|
||||
private ConsumeAzureEventHub processor;
|
||||
@Mock
|
||||
EventProcessorClient eventProcessorClient;
|
||||
|
||||
@Mock
|
||||
PartitionContext partitionContext;
|
||||
|
||||
@Mock
|
||||
CheckpointStore checkpointStore;
|
||||
|
||||
@Mock
|
||||
RecordSetWriterFactory writerFactory;
|
||||
|
||||
@Mock
|
||||
RecordSetWriter writer;
|
||||
|
||||
@Mock
|
||||
RecordReaderFactory readerFactory;
|
||||
|
||||
@Mock
|
||||
RecordReader reader;
|
||||
|
||||
private MockConsumeAzureEventHub processor;
|
||||
|
||||
private TestRunner testRunner;
|
||||
|
||||
@BeforeEach
|
||||
public void setupProcessor() {
|
||||
processor = new ConsumeAzureEventHub();
|
||||
final ProcessorInitializationContext initContext = Mockito.mock(ProcessorInitializationContext.class);
|
||||
final String componentId = "componentId";
|
||||
when(initContext.getIdentifier()).thenReturn(componentId);
|
||||
MockComponentLog componentLog = new MockComponentLog(componentId, processor);
|
||||
when(initContext.getLogger()).thenReturn(componentLog);
|
||||
processor.initialize(initContext);
|
||||
|
||||
final ProcessSessionFactory processSessionFactory = Mockito.mock(ProcessSessionFactory.class);
|
||||
processor.setProcessSessionFactory(processSessionFactory);
|
||||
processor.setNamespaceName("namespace");
|
||||
processor.setServiceBusEndpoint(serviceBusEndpoint);
|
||||
|
||||
sharedState = new SharedSessionState(processor, new AtomicLong(0));
|
||||
processSession = new MockProcessSession(sharedState, processor);
|
||||
when(processSessionFactory.createSession()).thenReturn(processSession);
|
||||
|
||||
eventProcessor = processor.new EventProcessor();
|
||||
|
||||
partitionContext = Mockito.mock(PartitionContext.class);
|
||||
when(partitionContext.getEventHubPath()).thenReturn("eventhub-name");
|
||||
when(partitionContext.getPartitionId()).thenReturn("partition-id");
|
||||
when(partitionContext.getConsumerGroupName()).thenReturn("consumer-group");
|
||||
processor = new MockConsumeAzureEventHub();
|
||||
testRunner = TestRunners.newTestRunner(processor);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessorConfigValidityWithManagedIdentityFlag() throws InitializationException {
|
||||
TestRunner testRunner = TestRunners.newTestRunner(processor);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.EVENT_HUB_NAME,eventHubName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.NAMESPACE,namespaceName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.assertNotValid();
|
||||
final MockRecordParser reader = new MockRecordParser();
|
||||
final MockRecordWriter writer = new MockRecordWriter();
|
||||
|
@ -132,8 +142,8 @@ public class TestConsumeAzureEventHub {
|
|||
testRunner.setProperty(ConsumeAzureEventHub.RECORD_WRITER, "writer");
|
||||
testRunner.setProperty(ConsumeAzureEventHub.RECORD_READER, "reader");
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_NAME, storageAccountName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_KEY, storageAccountKey);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_NAME, STORAGE_ACCOUNT_NAME);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_KEY, STORAGE_ACCOUNT_KEY);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.USE_MANAGED_IDENTITY,"true");
|
||||
testRunner.assertValid();
|
||||
|
@ -141,94 +151,76 @@ public class TestConsumeAzureEventHub {
|
|||
|
||||
@Test
|
||||
public void testProcessorConfigValidityWithNeitherStorageKeyNorTokenSet() {
|
||||
TestRunner testRunner = TestRunners.newTestRunner(processor);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.EVENT_HUB_NAME,eventHubName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.NAMESPACE,namespaceName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.ACCESS_POLICY_NAME, policyName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.POLICY_PRIMARY_KEY, policyKey);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.ACCESS_POLICY_NAME, POLICY_NAME);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.POLICY_PRIMARY_KEY, POLICY_KEY);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_NAME, storageAccountName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_NAME, STORAGE_ACCOUNT_NAME);
|
||||
testRunner.assertNotValid();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessorConfigValidityWithBothStorageKeyAndTokenSet() {
|
||||
TestRunner testRunner = TestRunners.newTestRunner(processor);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.EVENT_HUB_NAME,eventHubName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.NAMESPACE,namespaceName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.ACCESS_POLICY_NAME, policyName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.POLICY_PRIMARY_KEY, policyKey);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.ACCESS_POLICY_NAME, POLICY_NAME);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.POLICY_PRIMARY_KEY, POLICY_KEY);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_NAME, storageAccountName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_KEY, storageAccountKey);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_SAS_TOKEN, storageSasToken);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_NAME, STORAGE_ACCOUNT_NAME);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_KEY, STORAGE_ACCOUNT_KEY);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_SAS_TOKEN, STORAGE_TOKEN);
|
||||
testRunner.assertNotValid();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessorConfigValidityWithTokenSet() {
|
||||
TestRunner testRunner = TestRunners.newTestRunner(processor);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.EVENT_HUB_NAME,eventHubName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.NAMESPACE,namespaceName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.ACCESS_POLICY_NAME, policyName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.POLICY_PRIMARY_KEY, policyKey);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.ACCESS_POLICY_NAME, POLICY_NAME);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.POLICY_PRIMARY_KEY, POLICY_KEY);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_NAME, storageAccountName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_SAS_TOKEN, storageSasToken);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_NAME, STORAGE_ACCOUNT_NAME);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_SAS_TOKEN, STORAGE_TOKEN);
|
||||
testRunner.assertValid();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessorConfigValidityWithStorageKeySet() {
|
||||
TestRunner testRunner = TestRunners.newTestRunner(processor);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.EVENT_HUB_NAME,eventHubName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.NAMESPACE,namespaceName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.ACCESS_POLICY_NAME, policyName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.POLICY_PRIMARY_KEY, policyKey);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.ACCESS_POLICY_NAME, POLICY_NAME);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.POLICY_PRIMARY_KEY, POLICY_KEY);
|
||||
testRunner.assertNotValid();
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_NAME, storageAccountName);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_KEY, storageAccountKey);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_NAME, STORAGE_ACCOUNT_NAME);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_KEY, STORAGE_ACCOUNT_KEY);
|
||||
testRunner.assertValid();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReceivedApplicationProperties() throws Exception {
|
||||
final EventData singleEvent = EventData.create("one".getBytes(StandardCharsets.UTF_8));
|
||||
singleEvent.getProperties().put("event-sender", "Apache NiFi");
|
||||
singleEvent.getProperties().put("application", "TestApp");
|
||||
final Iterable<EventData> eventDataList = Collections.singletonList(singleEvent);
|
||||
eventProcessor.onEvents(partitionContext, eventDataList);
|
||||
public void testReceiveOne() {
|
||||
setProperties();
|
||||
testRunner.run(1, false);
|
||||
final List<EventData> events = getEvents(FIRST_CONTENT);
|
||||
|
||||
processSession.assertCommitted();
|
||||
final List<MockFlowFile> flowFiles = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
final EventBatchContext eventBatchContext = new EventBatchContext(partitionContext, events, checkpointStore, null);
|
||||
processor.eventBatchProcessor.accept(eventBatchContext);
|
||||
|
||||
final List<MockFlowFile> flowFiles = testRunner.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(1, flowFiles.size());
|
||||
final MockFlowFile msg1 = flowFiles.get(0);
|
||||
msg1.assertAttributeEquals("eventhub.property.event-sender", "Apache NiFi");
|
||||
msg1.assertAttributeEquals("eventhub.property.application", "TestApp");
|
||||
}
|
||||
final MockFlowFile flowFile = flowFiles.get(0);
|
||||
flowFile.assertContentEquals(FIRST_CONTENT);
|
||||
assertEventHubAttributesFound(flowFile);
|
||||
|
||||
@Test
|
||||
public void testReceiveOne() throws Exception {
|
||||
final Iterable<EventData> eventDataList = Collections.singletonList(EventData.create("one".getBytes(StandardCharsets.UTF_8)));
|
||||
eventProcessor.onEvents(partitionContext, eventDataList);
|
||||
|
||||
processSession.assertCommitted();
|
||||
final List<MockFlowFile> flowFiles = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(1, flowFiles.size());
|
||||
final MockFlowFile msg1 = flowFiles.get(0);
|
||||
msg1.assertContentEquals("one");
|
||||
msg1.assertAttributeEquals("eventhub.name", "eventhub-name");
|
||||
msg1.assertAttributeEquals("eventhub.partition", "partition-id");
|
||||
|
||||
final List<ProvenanceEventRecord> provenanceEvents = sharedState.getProvenanceEvents();
|
||||
final List<ProvenanceEventRecord> provenanceEvents = testRunner.getProvenanceEvents();
|
||||
assertEquals(1, provenanceEvents.size());
|
||||
final ProvenanceEventRecord provenanceEvent1 = provenanceEvents.get(0);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent1.getEventType());
|
||||
|
@ -236,45 +228,168 @@ public class TestConsumeAzureEventHub {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testReceiveTwo() throws Exception {
|
||||
final Iterable<EventData> eventDataList = Arrays.asList(
|
||||
EventData.create("one".getBytes(StandardCharsets.UTF_8)),
|
||||
EventData.create("two".getBytes(StandardCharsets.UTF_8))
|
||||
);
|
||||
eventProcessor.onEvents(partitionContext, eventDataList);
|
||||
public void testReceiveTwo() {
|
||||
setProperties();
|
||||
testRunner.run(1, false);
|
||||
final List<EventData> events = getEvents(FIRST_CONTENT, SECOND_CONTENT);
|
||||
|
||||
processSession.assertCommitted();
|
||||
final List<MockFlowFile> flowFiles = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
final EventBatchContext eventBatchContext = new EventBatchContext(partitionContext, events, checkpointStore, null);
|
||||
processor.eventBatchProcessor.accept(eventBatchContext);
|
||||
|
||||
final List<MockFlowFile> flowFiles = testRunner.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(2, flowFiles.size());
|
||||
final MockFlowFile msg1 = flowFiles.get(0);
|
||||
msg1.assertContentEquals("one");
|
||||
msg1.assertContentEquals(FIRST_CONTENT);
|
||||
final MockFlowFile msg2 = flowFiles.get(1);
|
||||
msg2.assertContentEquals("two");
|
||||
msg2.assertContentEquals(SECOND_CONTENT);
|
||||
|
||||
final List<ProvenanceEventRecord> provenanceEvents = sharedState.getProvenanceEvents();
|
||||
final List<ProvenanceEventRecord> provenanceEvents = testRunner.getProvenanceEvents();
|
||||
assertEquals(2, provenanceEvents.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpointFailure() throws Exception {
|
||||
final Iterable<EventData> eventDataList = Arrays.asList(
|
||||
EventData.create("one".getBytes(StandardCharsets.UTF_8)),
|
||||
EventData.create("two".getBytes(StandardCharsets.UTF_8))
|
||||
);
|
||||
doThrow(new RuntimeException("Failed to create a checkpoint.")).when(partitionContext).checkpoint();
|
||||
eventProcessor.onEvents(partitionContext, eventDataList);
|
||||
public void testReceiveRecords() throws Exception {
|
||||
setProperties();
|
||||
|
||||
// Even if it fails to create a checkpoint, these FlowFiles are already committed.
|
||||
processSession.assertCommitted();
|
||||
final List<MockFlowFile> flowFiles = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(2, flowFiles.size());
|
||||
final MockFlowFile msg1 = flowFiles.get(0);
|
||||
msg1.assertContentEquals("one");
|
||||
final MockFlowFile msg2 = flowFiles.get(1);
|
||||
msg2.assertContentEquals("two");
|
||||
final List<EventData> events = getEvents(FIRST_CONTENT, SECOND_CONTENT);
|
||||
setupRecordReader(events);
|
||||
setupRecordWriter();
|
||||
|
||||
final List<ProvenanceEventRecord> provenanceEvents = sharedState.getProvenanceEvents();
|
||||
testRunner.run(1, false);
|
||||
|
||||
final EventBatchContext eventBatchContext = new EventBatchContext(partitionContext, events, checkpointStore, null);
|
||||
processor.eventBatchProcessor.accept(eventBatchContext);
|
||||
|
||||
final List<MockFlowFile> flowFiles = testRunner.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(1, flowFiles.size());
|
||||
final MockFlowFile ff1 = flowFiles.get(0);
|
||||
ff1.assertContentEquals(FIRST_CONTENT + SECOND_CONTENT);
|
||||
assertEventHubAttributesFound(ff1);
|
||||
|
||||
final List<ProvenanceEventRecord> provenanceEvents = testRunner.getProvenanceEvents();
|
||||
assertEquals(1, provenanceEvents.size());
|
||||
final ProvenanceEventRecord provenanceEvent1 = provenanceEvents.get(0);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent1.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent1.getTransitUri());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReceiveRecordReaderFailure() throws Exception {
|
||||
setProperties();
|
||||
|
||||
final List<EventData> events = getEvents(FIRST_CONTENT, SECOND_CONTENT, THIRD_CONTENT, FOURTH_CONTENT);
|
||||
setupRecordReader(events, 2, null);
|
||||
setupRecordWriter();
|
||||
|
||||
testRunner.run(1, false);
|
||||
|
||||
final EventBatchContext eventBatchContext = new EventBatchContext(partitionContext, events, checkpointStore, null);
|
||||
processor.eventBatchProcessor.accept(eventBatchContext);
|
||||
|
||||
final List<MockFlowFile> flowFiles = testRunner.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(1, flowFiles.size());
|
||||
final MockFlowFile ff1 = flowFiles.get(0);
|
||||
ff1.assertContentEquals(FIRST_CONTENT + SECOND_CONTENT + FOURTH_CONTENT);
|
||||
assertEventHubAttributesFound(ff1);
|
||||
|
||||
final List<MockFlowFile> failedFFs = testRunner.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_PARSE_FAILURE);
|
||||
assertEquals(1, failedFFs.size());
|
||||
final MockFlowFile failed1 = failedFFs.get(0);
|
||||
failed1.assertContentEquals(THIRD_CONTENT);
|
||||
assertEventHubAttributesFound(failed1);
|
||||
|
||||
final List<ProvenanceEventRecord> provenanceEvents = testRunner.getProvenanceEvents();
|
||||
assertEquals(2, provenanceEvents.size());
|
||||
|
||||
final ProvenanceEventRecord provenanceEvent1 = provenanceEvents.get(0);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent1.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent1.getTransitUri());
|
||||
|
||||
final ProvenanceEventRecord provenanceEvent2 = provenanceEvents.get(1);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent2.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent2.getTransitUri());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReceiveAllRecordFailure() throws Exception {
|
||||
setProperties();
|
||||
|
||||
final List<EventData> events = getEvents(FIRST_CONTENT);
|
||||
setupRecordReader(events, 0, null);
|
||||
setRecordWriterProperty();
|
||||
|
||||
testRunner.run(1, false);
|
||||
|
||||
final EventBatchContext eventBatchContext = new EventBatchContext(partitionContext, events, checkpointStore, null);
|
||||
processor.eventBatchProcessor.accept(eventBatchContext);
|
||||
|
||||
final List<MockFlowFile> flowFiles = testRunner.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(0, flowFiles.size());
|
||||
|
||||
final List<MockFlowFile> failedFFs = testRunner.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_PARSE_FAILURE);
|
||||
assertEquals(1, failedFFs.size());
|
||||
final MockFlowFile failed1 = failedFFs.get(0);
|
||||
failed1.assertContentEquals(FIRST_CONTENT);
|
||||
assertEventHubAttributesFound(failed1);
|
||||
|
||||
final List<ProvenanceEventRecord> provenanceEvents = testRunner.getProvenanceEvents();
|
||||
assertEquals(1, provenanceEvents.size());
|
||||
|
||||
final ProvenanceEventRecord provenanceEvent1 = provenanceEvents.get(0);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent1.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent1.getTransitUri());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReceiveRecordWriterFailure() throws Exception {
|
||||
setProperties();
|
||||
|
||||
final List<EventData> events = getEvents(FIRST_CONTENT, SECOND_CONTENT, THIRD_CONTENT, FOURTH_CONTENT);
|
||||
setupRecordReader(events, -1, SECOND_CONTENT);
|
||||
setupRecordWriter(SECOND_CONTENT);
|
||||
|
||||
testRunner.run(1, false);
|
||||
|
||||
final EventBatchContext eventBatchContext = new EventBatchContext(partitionContext, events, checkpointStore, null);
|
||||
processor.eventBatchProcessor.accept(eventBatchContext);
|
||||
|
||||
final List<MockFlowFile> flowFiles = testRunner.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(1, flowFiles.size());
|
||||
final MockFlowFile ff1 = flowFiles.get(0);
|
||||
ff1.assertContentEquals(FIRST_CONTENT + THIRD_CONTENT + FOURTH_CONTENT);
|
||||
assertEventHubAttributesFound(ff1);
|
||||
|
||||
final List<MockFlowFile> failedFFs = testRunner.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_PARSE_FAILURE);
|
||||
assertEquals(1, failedFFs.size());
|
||||
final MockFlowFile failed1 = failedFFs.get(0);
|
||||
failed1.assertContentEquals(SECOND_CONTENT);
|
||||
assertEventHubAttributesFound(failed1);
|
||||
|
||||
final List<ProvenanceEventRecord> provenanceEvents = testRunner.getProvenanceEvents();
|
||||
assertEquals(2, provenanceEvents.size());
|
||||
|
||||
final ProvenanceEventRecord provenanceEvent1 = provenanceEvents.get(0);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent1.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent1.getTransitUri());
|
||||
|
||||
final ProvenanceEventRecord provenanceEvent2 = provenanceEvents.get(1);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent2.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent2.getTransitUri());
|
||||
}
|
||||
|
||||
private void setProperties() {
|
||||
testRunner.setProperty(ConsumeAzureEventHub.EVENT_HUB_NAME, EVENT_HUB_NAME);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.NAMESPACE, EVENT_HUB_NAMESPACE);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.ACCESS_POLICY_NAME, POLICY_NAME);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.POLICY_PRIMARY_KEY, POLICY_KEY);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_ACCOUNT_NAME, STORAGE_ACCOUNT_NAME);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.STORAGE_SAS_TOKEN, STORAGE_TOKEN);
|
||||
|
||||
when(partitionContext.getEventHubName()).thenReturn(EVENT_HUB_NAME);
|
||||
when(partitionContext.getConsumerGroup()).thenReturn(CONSUMER_GROUP);
|
||||
when(partitionContext.getPartitionId()).thenReturn(PARTITION_ID);
|
||||
|
||||
when(checkpointStore.updateCheckpoint(any(Checkpoint.class))).thenReturn(Mono.empty());
|
||||
}
|
||||
|
||||
private Record toRecord(String value) {
|
||||
|
@ -284,14 +399,22 @@ public class TestConsumeAzureEventHub {
|
|||
new RecordField("value", RecordFieldType.STRING.getDataType()))), map);
|
||||
}
|
||||
|
||||
private void setupRecordWriter() throws SchemaNotFoundException, IOException {
|
||||
private void setupRecordWriter() throws Exception {
|
||||
setupRecordWriter(null);
|
||||
}
|
||||
|
||||
private void setupRecordWriter(String throwErrorWith) throws SchemaNotFoundException, IOException {
|
||||
final RecordSetWriterFactory writerFactory = mock(RecordSetWriterFactory.class);
|
||||
processor.setWriterFactory(writerFactory);
|
||||
final RecordSetWriter writer = mock(RecordSetWriter.class);
|
||||
private RecordSetWriterFactory setRecordWriterProperty() throws InitializationException {
|
||||
when(writerFactory.getIdentifier()).thenReturn(RecordSetWriterFactory.class.getName());
|
||||
|
||||
testRunner.addControllerService(RecordSetWriterFactory.class.getName(), writerFactory);
|
||||
testRunner.enableControllerService(writerFactory);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.RECORD_WRITER, RecordSetWriterFactory.class.getName());
|
||||
|
||||
return writerFactory;
|
||||
}
|
||||
|
||||
private void setupRecordWriter(String throwErrorWith) throws Exception {
|
||||
final RecordSetWriterFactory writerFactory = setRecordWriterProperty();
|
||||
final AtomicReference<OutputStream> outRef = new AtomicReference<>();
|
||||
when(writerFactory.createWriter(any(), any(), any(), any(FlowFile.class))).thenAnswer(invocation -> {
|
||||
outRef.set(invocation.getArgument(2));
|
||||
|
@ -300,25 +423,27 @@ public class TestConsumeAzureEventHub {
|
|||
when(writer.write(any(Record.class))).thenAnswer(invocation -> {
|
||||
final String value = (String) invocation.<Record>getArgument(0).getValue("value");
|
||||
if (throwErrorWith != null && throwErrorWith.equals(value)) {
|
||||
throw new IOException("Simulating record write failure.");
|
||||
throw new IOException(MockConsumeAzureEventHub.class.getSimpleName());
|
||||
}
|
||||
outRef.get().write(value.getBytes(StandardCharsets.UTF_8));
|
||||
return WriteResult.of(1, Collections.emptyMap());
|
||||
});
|
||||
}
|
||||
|
||||
private void setupRecordReader(List<EventData> eventDataList) throws MalformedRecordException, IOException, SchemaNotFoundException {
|
||||
private void setupRecordReader(List<EventData> eventDataList) throws Exception {
|
||||
setupRecordReader(eventDataList, -1, null);
|
||||
}
|
||||
|
||||
private void setupRecordReader(List<EventData> eventDataList, int throwExceptionAt, String writeFailureWith)
|
||||
throws MalformedRecordException, IOException, SchemaNotFoundException {
|
||||
final RecordReaderFactory readerFactory = mock(RecordReaderFactory.class);
|
||||
processor.setReaderFactory(readerFactory);
|
||||
final RecordReader reader = mock(RecordReader.class);
|
||||
private void setupRecordReader(List<EventData> eventDataList, int throwExceptionAt, String writeFailureWith) throws Exception {
|
||||
when(readerFactory.getIdentifier()).thenReturn(RecordReaderFactory.class.getName());
|
||||
|
||||
testRunner.addControllerService(RecordReaderFactory.class.getName(), readerFactory);
|
||||
testRunner.enableControllerService(readerFactory);
|
||||
testRunner.setProperty(ConsumeAzureEventHub.RECORD_READER, RecordReaderFactory.class.getName());
|
||||
|
||||
when(readerFactory.createRecordReader(anyMap(), any(), anyLong(), any())).thenReturn(reader);
|
||||
final List<Record> recordList = eventDataList.stream()
|
||||
.map(eventData -> toRecord(new String(eventData.getBytes())))
|
||||
.map(eventData -> toRecord(eventData.getBodyAsString()))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// Add null to indicate the end of records.
|
||||
|
@ -337,161 +462,47 @@ public class TestConsumeAzureEventHub {
|
|||
break;
|
||||
case 0:
|
||||
when(reader.nextRecord())
|
||||
.thenThrow(new MalformedRecordException("Simulating Record parse failure."))
|
||||
.thenThrow(new MalformedRecordException(MockConsumeAzureEventHub.class.getSimpleName()))
|
||||
.thenReturn(records[0], Arrays.copyOfRange(records, 1, records.length));
|
||||
break;
|
||||
default:
|
||||
final List<Record> recordList1 = addEndRecord.apply(recordList.subList(0, throwExceptionAt));
|
||||
final List<Record> recordList2 = addEndRecord.apply(recordList.subList(throwExceptionAt + 1, recordList.size()));
|
||||
final Record[] records1 = recordList1.toArray(new Record[recordList1.size()]);
|
||||
final Record[] records2 = recordList2.toArray(new Record[recordList2.size()]);
|
||||
final Record[] records1 = recordList1.toArray(new Record[0]);
|
||||
final Record[] records2 = recordList2.toArray(new Record[0]);
|
||||
when(reader.nextRecord())
|
||||
.thenReturn(records1[0], Arrays.copyOfRange(records1, 1, records1.length))
|
||||
.thenThrow(new MalformedRecordException("Simulating Record parse failure."))
|
||||
.thenThrow(new MalformedRecordException(MockConsumeAzureEventHub.class.getSimpleName()))
|
||||
.thenReturn(records2[0], Arrays.copyOfRange(records2, 1, records2.length));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReceiveRecords() throws Exception {
|
||||
final List<EventData> eventDataList = Arrays.asList(
|
||||
EventData.create("one".getBytes(StandardCharsets.UTF_8)),
|
||||
EventData.create("two".getBytes(StandardCharsets.UTF_8))
|
||||
);
|
||||
|
||||
setupRecordReader(eventDataList);
|
||||
|
||||
setupRecordWriter();
|
||||
|
||||
eventProcessor.onEvents(partitionContext, eventDataList);
|
||||
|
||||
processSession.assertCommitted();
|
||||
final List<MockFlowFile> flowFiles = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(1, flowFiles.size());
|
||||
final MockFlowFile ff1 = flowFiles.get(0);
|
||||
ff1.assertContentEquals("onetwo");
|
||||
ff1.assertAttributeEquals("eventhub.name", "eventhub-name");
|
||||
ff1.assertAttributeEquals("eventhub.partition", "partition-id");
|
||||
|
||||
final List<ProvenanceEventRecord> provenanceEvents = sharedState.getProvenanceEvents();
|
||||
assertEquals(1, provenanceEvents.size());
|
||||
final ProvenanceEventRecord provenanceEvent1 = provenanceEvents.get(0);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent1.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent1.getTransitUri());
|
||||
private void assertEventHubAttributesFound(final MockFlowFile flowFile) {
|
||||
flowFile.assertAttributeEquals("eventhub.name", EVENT_HUB_NAME);
|
||||
flowFile.assertAttributeEquals("eventhub.partition", PARTITION_ID);
|
||||
flowFile.assertAttributeEquals(APPLICATION_ATTRIBUTE_NAME, MockConsumeAzureEventHub.class.getSimpleName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReceiveRecordReaderFailure() throws Exception {
|
||||
final List<EventData> eventDataList = Arrays.asList(
|
||||
EventData.create("one".getBytes(StandardCharsets.UTF_8)),
|
||||
EventData.create("two".getBytes(StandardCharsets.UTF_8)),
|
||||
EventData.create("three".getBytes(StandardCharsets.UTF_8)),
|
||||
EventData.create("four".getBytes(StandardCharsets.UTF_8))
|
||||
);
|
||||
|
||||
setupRecordReader(eventDataList, 2, null);
|
||||
|
||||
setupRecordWriter();
|
||||
|
||||
eventProcessor.onEvents(partitionContext, eventDataList);
|
||||
|
||||
processSession.assertCommitted();
|
||||
final List<MockFlowFile> flowFiles = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(1, flowFiles.size());
|
||||
final MockFlowFile ff1 = flowFiles.get(0);
|
||||
ff1.assertContentEquals("onetwofour");
|
||||
ff1.assertAttributeEquals("eventhub.name", "eventhub-name");
|
||||
ff1.assertAttributeEquals("eventhub.partition", "partition-id");
|
||||
|
||||
final List<MockFlowFile> failedFFs = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_PARSE_FAILURE);
|
||||
assertEquals(1, failedFFs.size());
|
||||
final MockFlowFile failed1 = failedFFs.get(0);
|
||||
failed1.assertContentEquals("three");
|
||||
failed1.assertAttributeEquals("eventhub.name", "eventhub-name");
|
||||
failed1.assertAttributeEquals("eventhub.partition", "partition-id");
|
||||
|
||||
final List<ProvenanceEventRecord> provenanceEvents = sharedState.getProvenanceEvents();
|
||||
assertEquals(2, provenanceEvents.size());
|
||||
|
||||
final ProvenanceEventRecord provenanceEvent1 = provenanceEvents.get(0);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent1.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent1.getTransitUri());
|
||||
|
||||
final ProvenanceEventRecord provenanceEvent2 = provenanceEvents.get(1);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent2.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent2.getTransitUri());
|
||||
private List<EventData> getEvents(final String... contents) {
|
||||
return Arrays.stream(contents)
|
||||
.map(content -> {
|
||||
final EventData eventData = new EventData(content);
|
||||
eventData.getProperties().put(APPLICATION_PROPERTY, MockConsumeAzureEventHub.class.getSimpleName());
|
||||
return eventData;
|
||||
})
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReceiveAllRecordFailure() throws Exception {
|
||||
final List<EventData> eventDataList = Collections.singletonList(
|
||||
EventData.create("one".getBytes(StandardCharsets.UTF_8))
|
||||
);
|
||||
private class MockConsumeAzureEventHub extends ConsumeAzureEventHub {
|
||||
|
||||
setupRecordReader(eventDataList, 0, null);
|
||||
@Override
|
||||
protected EventProcessorClient createClient(final ProcessContext context) {
|
||||
return eventProcessorClient;
|
||||
}
|
||||
|
||||
setupRecordWriter();
|
||||
|
||||
eventProcessor.onEvents(partitionContext, eventDataList);
|
||||
|
||||
processSession.assertCommitted();
|
||||
final List<MockFlowFile> flowFiles = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(0, flowFiles.size());
|
||||
|
||||
final List<MockFlowFile> failedFFs = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_PARSE_FAILURE);
|
||||
assertEquals(1, failedFFs.size());
|
||||
final MockFlowFile failed1 = failedFFs.get(0);
|
||||
failed1.assertContentEquals("one");
|
||||
failed1.assertAttributeEquals("eventhub.name", "eventhub-name");
|
||||
failed1.assertAttributeEquals("eventhub.partition", "partition-id");
|
||||
|
||||
final List<ProvenanceEventRecord> provenanceEvents = sharedState.getProvenanceEvents();
|
||||
assertEquals(1, provenanceEvents.size());
|
||||
|
||||
final ProvenanceEventRecord provenanceEvent1 = provenanceEvents.get(0);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent1.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent1.getTransitUri());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReceiveRecordWriterFailure() throws Exception {
|
||||
final List<EventData> eventDataList = Arrays.asList(
|
||||
EventData.create("one".getBytes(StandardCharsets.UTF_8)),
|
||||
EventData.create("two".getBytes(StandardCharsets.UTF_8)),
|
||||
EventData.create("three".getBytes(StandardCharsets.UTF_8)),
|
||||
EventData.create("four".getBytes(StandardCharsets.UTF_8))
|
||||
);
|
||||
|
||||
setupRecordReader(eventDataList, -1, "two");
|
||||
|
||||
setupRecordWriter("two");
|
||||
|
||||
eventProcessor.onEvents(partitionContext, eventDataList);
|
||||
|
||||
processSession.assertCommitted();
|
||||
final List<MockFlowFile> flowFiles = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS);
|
||||
assertEquals(1, flowFiles.size());
|
||||
final MockFlowFile ff1 = flowFiles.get(0);
|
||||
ff1.assertContentEquals("onethreefour");
|
||||
ff1.assertAttributeEquals("eventhub.name", "eventhub-name");
|
||||
ff1.assertAttributeEquals("eventhub.partition", "partition-id");
|
||||
|
||||
final List<MockFlowFile> failedFFs = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_PARSE_FAILURE);
|
||||
assertEquals(1, failedFFs.size());
|
||||
final MockFlowFile failed1 = failedFFs.get(0);
|
||||
failed1.assertContentEquals("two");
|
||||
failed1.assertAttributeEquals("eventhub.name", "eventhub-name");
|
||||
failed1.assertAttributeEquals("eventhub.partition", "partition-id");
|
||||
|
||||
final List<ProvenanceEventRecord> provenanceEvents = sharedState.getProvenanceEvents();
|
||||
assertEquals(2, provenanceEvents.size());
|
||||
|
||||
final ProvenanceEventRecord provenanceEvent1 = provenanceEvents.get(0);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent1.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent1.getTransitUri());
|
||||
|
||||
final ProvenanceEventRecord provenanceEvent2 = provenanceEvents.get(1);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent2.getEventType());
|
||||
assertEquals(EXPECTED_TRANSIT_URI, provenanceEvent2.getTransitUri());
|
||||
@Override
|
||||
protected String getTransitUri(final PartitionContext partitionContext) {
|
||||
return EXPECTED_TRANSIT_URI;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,145 @@
|
|||
/*
|
||||
* 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.nifi.processors.azure.eventhub.position;
|
||||
|
||||
import com.azure.core.http.rest.PagedFlux;
|
||||
import com.azure.core.http.rest.PagedResponse;
|
||||
import com.azure.core.util.BinaryData;
|
||||
import com.azure.core.util.IterableStream;
|
||||
import com.azure.messaging.eventhubs.models.EventPosition;
|
||||
import com.azure.storage.blob.BlobAsyncClient;
|
||||
import com.azure.storage.blob.BlobContainerAsyncClient;
|
||||
import com.azure.storage.blob.models.BlobItem;
|
||||
import com.azure.storage.blob.models.ListBlobsOptions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
import reactor.core.publisher.Mono;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
class LegacyBlobStorageEventPositionProviderTest {
|
||||
private static final String CONSUMER_GROUP = "$Default";
|
||||
|
||||
private static final String PARTITION_ID = "1";
|
||||
|
||||
private static final String BLOB_NAME = String.format("/Partitions/%s", PARTITION_ID);
|
||||
|
||||
private static final String EMPTY_OBJECT = "{}";
|
||||
|
||||
private static final long SEQUENCE_NUMBER = 10;
|
||||
|
||||
private static final String SEQUENCE_NUMBER_OBJECT = String.format("{\"sequenceNumber\":%d}", SEQUENCE_NUMBER);
|
||||
|
||||
@Mock
|
||||
BlobContainerAsyncClient blobContainerAsyncClient;
|
||||
|
||||
@Mock
|
||||
BlobAsyncClient blobAsyncClient;
|
||||
|
||||
@Mock
|
||||
PagedResponse<BlobItem> pagedResponse;
|
||||
|
||||
LegacyBlobStorageEventPositionProvider provider;
|
||||
|
||||
@BeforeEach
|
||||
void setProvider() {
|
||||
provider = new LegacyBlobStorageEventPositionProvider(blobContainerAsyncClient, CONSUMER_GROUP);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testContainerNotFound() {
|
||||
when(blobContainerAsyncClient.exists()).thenReturn(Mono.just(Boolean.FALSE));
|
||||
|
||||
final Map<String, EventPosition> partitionEventPosition = provider.getInitialPartitionEventPosition();
|
||||
|
||||
assertNotNull(partitionEventPosition);
|
||||
assertTrue(partitionEventPosition.isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testContainerEmpty() {
|
||||
when(blobContainerAsyncClient.exists()).thenReturn(Mono.just(Boolean.TRUE));
|
||||
|
||||
final PagedFlux<BlobItem> blobItems = new PagedFlux<>(() -> Mono.just(pagedResponse));
|
||||
when(blobContainerAsyncClient.listBlobs(any(ListBlobsOptions.class))).thenReturn(blobItems);
|
||||
|
||||
final Map<String, EventPosition> partitionEventPosition = provider.getInitialPartitionEventPosition();
|
||||
|
||||
assertNotNull(partitionEventPosition);
|
||||
assertTrue(partitionEventPosition.isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSequenceNumberNotFound() {
|
||||
setBlobData(EMPTY_OBJECT);
|
||||
|
||||
final Map<String, EventPosition> partitionEventPosition = provider.getInitialPartitionEventPosition();
|
||||
|
||||
assertNotNull(partitionEventPosition);
|
||||
assertTrue(partitionEventPosition.isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSequenceNumberFound() {
|
||||
setBlobData(SEQUENCE_NUMBER_OBJECT);
|
||||
|
||||
final Map<String, EventPosition> partitionEventPosition = provider.getInitialPartitionEventPosition();
|
||||
|
||||
assertNotNull(partitionEventPosition);
|
||||
assertFalse(partitionEventPosition.isEmpty());
|
||||
|
||||
final EventPosition eventPosition = partitionEventPosition.get(PARTITION_ID);
|
||||
assertNotNull(eventPosition);
|
||||
|
||||
assertEquals(SEQUENCE_NUMBER, eventPosition.getSequenceNumber());
|
||||
}
|
||||
|
||||
private void setBlobData(final String blobData) {
|
||||
when(blobContainerAsyncClient.exists()).thenReturn(Mono.just(Boolean.TRUE));
|
||||
|
||||
final BlobItem directoryBlobItem = new BlobItem();
|
||||
directoryBlobItem.setIsPrefix(true);
|
||||
|
||||
final BlobItem blobItem = new BlobItem();
|
||||
blobItem.setIsPrefix(false);
|
||||
blobItem.setName(BLOB_NAME);
|
||||
|
||||
final IterableStream<BlobItem> blobItems = IterableStream.of(Arrays.asList(directoryBlobItem, blobItem));
|
||||
when(pagedResponse.getElements()).thenReturn(blobItems);
|
||||
final PagedFlux<BlobItem> pagedItems = new PagedFlux<>(() -> Mono.just(pagedResponse));
|
||||
when(blobContainerAsyncClient.listBlobs(any(ListBlobsOptions.class))).thenReturn(pagedItems);
|
||||
|
||||
when(blobContainerAsyncClient.getBlobAsyncClient(eq(BLOB_NAME))).thenReturn(blobAsyncClient);
|
||||
|
||||
when(blobAsyncClient.exists()).thenReturn(Mono.just(Boolean.TRUE));
|
||||
final BinaryData objectData = BinaryData.fromString(blobData);
|
||||
when(blobAsyncClient.downloadContent()).thenReturn(Mono.just(objectData));
|
||||
}
|
||||
}
|
|
@ -1,45 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<parent>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-azure-bundle</artifactId>
|
||||
<version>1.18.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<artifactId>nifi-azure-record-sink-nar</artifactId>
|
||||
<packaging>nar</packaging>
|
||||
<properties>
|
||||
<maven.javadoc.skip>true</maven.javadoc.skip>
|
||||
<source.skip>true</source.skip>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-azure-record-sink</artifactId>
|
||||
<version>1.18.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-standard-services-api-nar</artifactId>
|
||||
<version>1.18.0-SNAPSHOT</version>
|
||||
<type>nar</type>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
|
@ -1,73 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!-- 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. -->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>nifi-azure-bundle</artifactId>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<version>1.18.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>nifi-azure-record-sink</artifactId>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-messaging-eventhubs</artifactId>
|
||||
<version>5.12.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-identity</artifactId>
|
||||
<version>${azure.identity.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-record-sink-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-record</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-record-serialization-service-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-utils</artifactId>
|
||||
<version>1.18.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-inline</artifactId>
|
||||
<version>${mockito.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-mock</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-mock-record-utils</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
|
@ -1,15 +0,0 @@
|
|||
# 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.
|
||||
org.apache.nifi.services.azure.eventhub.AzureEventHubRecordSink
|
|
@ -20,7 +20,6 @@
|
|||
<artifactId>nifi-azure-reporting-task</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
|
@ -37,6 +36,11 @@
|
|||
<artifactId>nifi-metrics</artifactId>
|
||||
<version>1.18.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-reporting-utils</artifactId>
|
||||
<version>1.18.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.code.gson</groupId>
|
||||
<artifactId>gson</artifactId>
|
||||
|
@ -53,7 +57,6 @@
|
|||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-lang3</artifactId>
|
||||
</dependency>
|
||||
<!-- test dependencies -->
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-core</artifactId>
|
||||
|
@ -65,11 +68,5 @@
|
|||
<version>1.18.0-SNAPSHOT</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-reporting-utils</artifactId>
|
||||
<version>1.18.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
|
|
@ -27,32 +27,14 @@
|
|||
<dependency>
|
||||
<groupId>com.microsoft.azure</groupId>
|
||||
<artifactId>azure-storage</artifactId>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.google.code.findbugs</groupId>
|
||||
<artifactId>jsr305</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-core</artifactId>
|
||||
<version>${azure.core.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-cosmos</artifactId>
|
||||
<version>${azure-cosmos.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-core</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.google.code.findbugs</groupId>
|
||||
<artifactId>jsr305</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
|
|
|
@ -26,12 +26,9 @@
|
|||
<packaging>pom</packaging>
|
||||
|
||||
<properties>
|
||||
<azure-storage.version>8.6.6</azure-storage.version>
|
||||
<azure.core.version>1.26.0</azure.core.version>
|
||||
<azure.identity.version>1.4.5</azure.identity.version>
|
||||
<!-- azure-identity depends on msal4j transitively, keep these versions consistent -->
|
||||
<msal4j.version>1.11.0</msal4j.version>
|
||||
<azure-cosmos.version>4.26.0</azure-cosmos.version>
|
||||
<azure.sdk.bom.version>1.2.6</azure.sdk.bom.version>
|
||||
<microsoft.azure-storage.version>8.6.6</microsoft.azure-storage.version>
|
||||
<msal4j.version>1.13.0</msal4j.version>
|
||||
</properties>
|
||||
|
||||
<modules>
|
||||
|
@ -41,16 +38,21 @@
|
|||
<module>nifi-azure-nar</module>
|
||||
<module>nifi-azure-services-api</module>
|
||||
<module>nifi-azure-services-api-nar</module>
|
||||
<module>nifi-azure-record-sink</module>
|
||||
<module>nifi-azure-record-sink-nar</module>
|
||||
</modules>
|
||||
|
||||
<dependencyManagement>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.azure</groupId>
|
||||
<artifactId>azure-sdk-bom</artifactId>
|
||||
<version>${azure.sdk.bom.version}</version>
|
||||
<type>pom</type>
|
||||
<scope>import</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.microsoft.azure</groupId>
|
||||
<artifactId>azure-storage</artifactId>
|
||||
<version>${azure-storage.version}</version>
|
||||
<version>${microsoft.azure-storage.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
|
|
Loading…
Reference in New Issue