From 6461061e8f12b41c4d5fd9d33d389f8c93d5f92e Mon Sep 17 00:00:00 2001 From: exceptionfactory Date: Wed, 17 Aug 2022 18:55:17 -0500 Subject: [PATCH] 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 --- nifi-assembly/pom.xml | 6 - .../nifi-azure-processors/pom.xml | 82 +-- .../azure/eventhub/ConsumeAzureEventHub.java | 613 +++++++++--------- .../azure/eventhub/GetAzureEventHub.java | 387 +++++------ .../azure/eventhub/PutAzureEventHub.java | 332 +++------- .../EarliestEventPositionProvider.java | 63 ++ .../position/EventPositionProvider.java | 33 + ...egacyBlobStorageEventPositionProvider.java | 151 +++++ .../eventhub/utils/AzureEventHubUtils.java | 30 +- .../eventhub/AzureAuthenticationStrategy.java | 0 .../eventhub/AzureEventHubRecordSink.java | 0 ...g.apache.nifi.controller.ControllerService | 1 + .../document/PutAzureCosmosDBRecordTest.java | 79 ++- .../azure/eventhub/GetAzureEventHubTest.java | 275 +++----- .../azure/eventhub/PutAzureEventHubTest.java | 424 +++--------- .../eventhub/TestConsumeAzureEventHub.java | 587 +++++++++-------- ...yBlobStorageEventPositionProviderTest.java | 145 +++++ .../eventhub/TestAzureEventHubRecordSink.java | 0 .../nifi-azure-record-sink-nar/pom.xml | 45 -- .../nifi-azure-record-sink/pom.xml | 73 --- ...g.apache.nifi.controller.ControllerService | 15 - .../nifi-azure-reporting-task/pom.xml | 13 +- .../nifi-azure-services-api/pom.xml | 18 - nifi-nar-bundles/nifi-azure-bundle/pom.xml | 20 +- 24 files changed, 1530 insertions(+), 1862 deletions(-) create mode 100644 nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/EarliestEventPositionProvider.java create mode 100644 nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/EventPositionProvider.java create mode 100644 nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/LegacyBlobStorageEventPositionProvider.java rename nifi-nar-bundles/nifi-azure-bundle/{nifi-azure-record-sink => nifi-azure-processors}/src/main/java/org/apache/nifi/services/azure/eventhub/AzureAuthenticationStrategy.java (100%) rename nifi-nar-bundles/nifi-azure-bundle/{nifi-azure-record-sink => nifi-azure-processors}/src/main/java/org/apache/nifi/services/azure/eventhub/AzureEventHubRecordSink.java (100%) create mode 100644 nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/position/LegacyBlobStorageEventPositionProviderTest.java rename nifi-nar-bundles/nifi-azure-bundle/{nifi-azure-record-sink => nifi-azure-processors}/src/test/java/org/apache/nifi/services/azure/eventhub/TestAzureEventHubRecordSink.java (100%) delete mode 100644 nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink-nar/pom.xml delete mode 100644 nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/pom.xml delete mode 100644 nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index 78f67698e8..05ec98e555 100644 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -514,12 +514,6 @@ language governing permissions and limitations under the License. --> 1.18.0-SNAPSHOT nar - - org.apache.nifi - nifi-azure-record-sink-nar - 1.18.0-SNAPSHOT - nar - org.apache.nifi nifi-azure-services-api-nar diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml index a6ad604524..45679ee755 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml @@ -20,11 +20,7 @@ nifi-azure-processors jar - 3.3.0 - 3.3.0 1.2.6 - 12.7.4 - 12.14.4 @@ -45,6 +41,10 @@ org.apache.nifi nifi-record-serialization-service-api + + org.apache.nifi + nifi-record-sink-api + org.apache.nifi nifi-record @@ -62,74 +62,44 @@ com.azure azure-core - ${azure.core.version} com.azure azure-identity - ${azure.identity.version} - - - com.azure - azure-core - - - com.microsoft.azure - azure-eventhubs - ${azure-eventhubs.version} + com.azure + azure-messaging-eventhubs + + com.azure + azure-messaging-eventhubs-checkpointstore-blob + + + com.azure + azure-cosmos + + + com.azure + azure-storage-file-datalake + + + com.azure + azure-storage-blob + + com.microsoft.azure azure-keyvault ${azure-keyvault.version} - - com.microsoft.azure - azure-eventhubs-eph - ${azure-eventhubs-eph.version} - com.microsoft.azure azure-storage - com.azure - azure-cosmos - ${azure-cosmos.version} - - - com.azure - azure-core - - - com.google.guava - guava - - - - - com.azure - azure-storage-file-datalake - ${azure-storage-file-datalake.version} - - - com.azure - azure-core - - - - - com.azure - azure-storage-blob - ${azure-storage-blob.version} - - - com.azure - azure-core - - + com.fasterxml.jackson.core + jackson-databind commons-io @@ -180,7 +150,7 @@ org.mockito mockito-inline - 3.3.3 + ${mockito.version} test diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java index 92d34ddef9..219e8403fb 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java @@ -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-' 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 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 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 { - @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 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 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 applicationProperties = AzureEventHubUtils.getApplicationProperties(eventData); - attributes.putAll(applicationProperties); - - attributes.put("eventhub.name", eventHubName); - attributes.put("eventhub.partition", partitionId); - } - - private void writeFlowFiles(PartitionContext context, Iterable 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 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 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 schemaRetrievalVariables = new HashMap<>(); - schemaRetrievalVariables.put("eventhub.name", eventHubName); - - final ComponentLog logger = getLogger(); - FlowFile flowFile = session.create(); - final Map 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 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 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 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 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 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 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 events = eventBatchContext.getEvents(); + events.forEach(eventData -> { + final Map 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 schemaRetrievalVariables = new HashMap<>(); + schemaRetrievalVariables.put("eventhub.name", partitionContext.getEventHubName()); + + final ComponentLog logger = getLogger(); + FlowFile flowFile = session.create(); + final Map attributes = new HashMap<>(); + + RecordSetWriter writer = null; + EventData lastEventData = null; + WriteResult lastWriteResult = null; + int recordCount = 0; + + final LastEnqueuedEventProperties lastEnqueuedEventProperties = eventBatchContext.getLastEnqueuedEventProperties(); + final List 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 getLegacyPartitionEventPosition( + final BlobContainerAsyncClient blobContainerAsyncClient, + final String consumerGroup + ) { + final LegacyBlobStorageEventPositionProvider legacyBlobStorageEventPositionProvider = new LegacyBlobStorageEventPositionProvider( + blobContainerAsyncClient, + consumerGroup + ); + final Map 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 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; } } diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java index 27c836ce3e..88a0ada7e7 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java @@ -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 -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 partitionToReceiverMap = new ConcurrentHashMap<>(); - private volatile BlockingQueue partitionNames = new LinkedBlockingQueue<>(); - private volatile Instant configuredEnqueueTime; - private volatile int receiverFetchSize; - private volatile Duration receiverFetchTimeout; - private EventHubClient eventHubClient; - private final static List propertyDescriptors; private final static Set relationships; - /* - * Will ensure that the list of property descriptors is build only once. - * Will also create a Set of relationships - */ static { - List _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 _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 partitionEventPositions = new ConcurrentHashMap<>(); + + private volatile BlockingQueue partitionIds = new LinkedBlockingQueue<>(); + private volatile int receiverFetchSize; + private volatile Duration receiverFetchTimeout; + + private EventHubConsumerClient eventHubConsumerClient; + @Override public Set getRelationships() { return relationships; @@ -205,197 +195,168 @@ public class GetAzureEventHub extends AbstractProcessor { @Override protected Collection customValidate(ValidationContext context) { - List 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 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 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 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 events = receiveEvents(partitionId); - final Iterable receivedEvents = receiveEvents(context, partitionId); - if (receivedEvents == null) { - return; + for (final PartitionEvent partitionEvent : events) { + final Map 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 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 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 getPartitionIds() { + final BlockingQueue 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 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 getAttributes(final PartitionEvent partitionEvent) { + final Map 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 applicationProperties = AzureEventHubUtils.getApplicationProperties(eventData.getProperties()); + attributes.putAll(applicationProperties); + + return attributes; + } } diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java index f588f76a59..83e80f7235 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java @@ -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 -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 senderQueue = new LinkedBlockingQueue<>(); - private final static List propertyDescriptors; private final static Set relationships; - /* - * Will ensure that the list of property descriptors is build only once. - * Will also create a Set of relationships - */ static { - List _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 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 _relationships = new HashSet<>(); - _relationships.add(REL_SUCCESS); - _relationships.add(REL_FAILURE); - relationships = Collections.unmodifiableSet(_relationships); + final Set configuredRelationships = new HashSet<>(); + configuredRelationships.add(REL_SUCCESS); + configuredRelationships.add(REL_FAILURE); + relationships = Collections.unmodifiableSet(configuredRelationships); } + + private EventHubProducerClient eventHubProducerClient; + @Override public Set 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 customValidate(ValidationContext context) { - List 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 flowFileList = session.get(maxBatchSize); + final int maxBatchSize = context.getProperty(MAX_BATCH_SIZE).asInteger(); + final List flowFileBatch = session.get(maxBatchSize); - // Convert and send each flow file - final BlockingQueue>> futureQueue = new LinkedBlockingQueue<>(); - for (FlowFile flowFile : flowFileList) { - if (flowFile == null) { - continue; - } - - futureQueue.offer(handleFlowFile(flowFile, partitioningKeyAttributeName, session)); + final List> flowFileResults = new ArrayList<>(); + for (final FlowFile flowFile : flowFileBatch) { + final FlowFileResultCarrier 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>> futureQueue){ - + final List> flowFileResults + ) { try { - for (CompletableFuture> completableFuture : futureQueue) { - completableFuture.join(); - - final FlowFileResultCarrier flowFileResult = completableFuture.get(); - if(flowFileResult == null) { - continue; - } - + for (final FlowFileResultCarrier 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> handleFlowFile(FlowFile flowFile, final String partitioningKeyAttributeName, final ProcessSession session) { - - // Read message body + private FlowFileResultCarrier 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 userProperties; - Map 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(flowFile, REL_SUCCESS); - }) - .exceptionally(processException -> { - return new FlowFileResultCarrier(flowFile, REL_FAILURE, processException); - }); - - } catch (final ProcessException processException) { - return CompletableFuture.completedFuture(new FlowFileResultCarrier(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 attributes = flowFile.getAttributes(); + final Map 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 Event Hubs Quotas - */ - protected CompletableFuture sendMessage(final byte[] buffer, String partitioningKey, Map 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 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 properties = eventData.getProperties(); - if(userProperties != null && properties != null) { - properties.putAll(userProperties); - } - - // Send with optional partition key - final CompletableFuture 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); } } diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/EarliestEventPositionProvider.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/EarliestEventPositionProvider.java new file mode 100644 index 0000000000..4eb517a8fb --- /dev/null +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/EarliestEventPositionProvider.java @@ -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 + * Azure SDK for Java Issue 11431 + */ +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 getInitialPartitionEventPosition() { + return new EarliestEventPosition(); + } + + private static class EarliestEventPosition extends HashMap { + /** + * 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(); + } + } +} diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/EventPositionProvider.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/EventPositionProvider.java new file mode 100644 index 0000000000..efea28f718 --- /dev/null +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/EventPositionProvider.java @@ -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 getInitialPartitionEventPosition(); +} diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/LegacyBlobStorageEventPositionProvider.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/LegacyBlobStorageEventPositionProvider.java new file mode 100644 index 0000000000..8a4c15894a --- /dev/null +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/position/LegacyBlobStorageEventPositionProvider.java @@ -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 getInitialPartitionEventPosition() { + final Map partitionEventPosition; + + if (containerExists()) { + final BlobListDetails blobListDetails = new BlobListDetails().setRetrieveMetadata(true); + final ListBlobsOptions listBlobsOptions = new ListBlobsOptions().setPrefix(consumerGroup).setDetails(blobListDetails); + final Iterable blobItems = blobContainerAsyncClient.listBlobs(listBlobsOptions).toIterable(); + partitionEventPosition = getPartitionEventPosition(blobItems); + } else { + partitionEventPosition = Collections.emptyMap(); + } + + return partitionEventPosition; + } + + private Map getPartitionEventPosition(final Iterable blobItems) { + final Map 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()); + } +} diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java index cb19f2e54b..6169178f5e 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/utils/AzureEventHubUtils.java @@ -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 getApplicationProperties(EventData eventData) { + public static Map getApplicationProperties(final Map eventProperties) { final Map properties = new HashMap<>(); - final Map applicationProperties = eventData.getProperties(); - if (null != applicationProperties) { - for (Map.Entry property : applicationProperties.entrySet()) { + if (eventProperties != null) { + for (Map.Entry 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("^\\.", ""); - } - } diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/src/main/java/org/apache/nifi/services/azure/eventhub/AzureAuthenticationStrategy.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/eventhub/AzureAuthenticationStrategy.java similarity index 100% rename from nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/src/main/java/org/apache/nifi/services/azure/eventhub/AzureAuthenticationStrategy.java rename to nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/eventhub/AzureAuthenticationStrategy.java diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/src/main/java/org/apache/nifi/services/azure/eventhub/AzureEventHubRecordSink.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/eventhub/AzureEventHubRecordSink.java similarity index 100% rename from nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/src/main/java/org/apache/nifi/services/azure/eventhub/AzureEventHubRecordSink.java rename to nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/eventhub/AzureEventHubRecordSink.java diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService index 86036bdad0..37ed2ab9f6 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -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 diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/cosmos/document/PutAzureCosmosDBRecordTest.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/cosmos/document/PutAzureCosmosDBRecordTest.java index 55ead1ad00..de2d85e2cc 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/cosmos/document/PutAzureCosmosDBRecordTest.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/cosmos/document/PutAzureCosmosDBRecordTest.java @@ -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 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> 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> mockBackend = new ArrayList<>(); + private final List> 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> getTestResults() { return mockBackend; } - - - public CosmosContainer getMockConainer() { - return mockContainer; - } } \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHubTest.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHubTest.java index a581f9155e..a46aa54558 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHubTest.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHubTest.java @@ -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 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 getPartitionIds() { + return new LinkedBlockingQueue<>(Collections.singleton(PARTITION_ID)); } @Override - protected Iterable receiveEvents(final ProcessContext context, final String partitionId) throws ProcessException{ - if(nullReceive){ - return null; - } - if(getReceiverThrow){ - throw new ProcessException("Could not create receiver"); - } - final LinkedList 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 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 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; + } + } } diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHubTest.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHubTest.java index 9f0bf5f0c8..10fdb12e8d 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHubTest.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHubTest.java @@ -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 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 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 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 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 failedFuture = new CompletableFuture(); - failedFuture.completeExceptionally(new IllegalArgumentException()); - - when(eventHubClient.send(any(EventData.class))) - .thenReturn(failedFuture) - .thenReturn(CompletableFuture.completedFuture(null)); - - testRunner = TestRunners.newTestRunner(processor); - setUpStandardTestConfig(); - - List 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>> futureQueue = new LinkedBlockingQueue>>(); - - @SuppressWarnings("unchecked") - CompletableFuture> throwingFuture = (CompletableFuture>)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(flowFile1, PutAzureEventHub.REL_SUCCESS))); - futureQueue.offer(CompletableFuture.completedFuture(new FlowFileResultCarrier(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 sendMessage(final byte[] buffer, String partitioningKey, Map 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(); } } \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/TestConsumeAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/TestConsumeAzureEventHub.java index 85dead6e13..e44197c928 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/TestConsumeAzureEventHub.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/TestConsumeAzureEventHub.java @@ -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 eventDataList = Collections.singletonList(singleEvent); - eventProcessor.onEvents(partitionContext, eventDataList); + public void testReceiveOne() { + setProperties(); + testRunner.run(1, false); + final List events = getEvents(FIRST_CONTENT); - processSession.assertCommitted(); - final List flowFiles = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS); + final EventBatchContext eventBatchContext = new EventBatchContext(partitionContext, events, checkpointStore, null); + processor.eventBatchProcessor.accept(eventBatchContext); + + final List 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 eventDataList = Collections.singletonList(EventData.create("one".getBytes(StandardCharsets.UTF_8))); - eventProcessor.onEvents(partitionContext, eventDataList); - - processSession.assertCommitted(); - final List 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 provenanceEvents = sharedState.getProvenanceEvents(); + final List 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 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 events = getEvents(FIRST_CONTENT, SECOND_CONTENT); - processSession.assertCommitted(); - final List flowFiles = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS); + final EventBatchContext eventBatchContext = new EventBatchContext(partitionContext, events, checkpointStore, null); + processor.eventBatchProcessor.accept(eventBatchContext); + + final List 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 provenanceEvents = sharedState.getProvenanceEvents(); + final List provenanceEvents = testRunner.getProvenanceEvents(); assertEquals(2, provenanceEvents.size()); } @Test - public void testCheckpointFailure() throws Exception { - final Iterable 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 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 events = getEvents(FIRST_CONTENT, SECOND_CONTENT); + setupRecordReader(events); + setupRecordWriter(); - final List provenanceEvents = sharedState.getProvenanceEvents(); + testRunner.run(1, false); + + final EventBatchContext eventBatchContext = new EventBatchContext(partitionContext, events, checkpointStore, null); + processor.eventBatchProcessor.accept(eventBatchContext); + + final List 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 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 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 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 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 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 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 flowFiles = testRunner.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS); + assertEquals(0, flowFiles.size()); + + final List 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 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 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 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 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 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 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.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 eventDataList) throws MalformedRecordException, IOException, SchemaNotFoundException { + private void setupRecordReader(List eventDataList) throws Exception { setupRecordReader(eventDataList, -1, null); } - private void setupRecordReader(List 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 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 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 recordList1 = addEndRecord.apply(recordList.subList(0, throwExceptionAt)); final List 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 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 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 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 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 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 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 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 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 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 flowFiles = processSession.getFlowFilesForRelationship(ConsumeAzureEventHub.REL_SUCCESS); - assertEquals(0, flowFiles.size()); - - final List 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 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 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 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 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 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; + } } } diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/position/LegacyBlobStorageEventPositionProviderTest.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/position/LegacyBlobStorageEventPositionProviderTest.java new file mode 100644 index 0000000000..36e244bc54 --- /dev/null +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/position/LegacyBlobStorageEventPositionProviderTest.java @@ -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 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 partitionEventPosition = provider.getInitialPartitionEventPosition(); + + assertNotNull(partitionEventPosition); + assertTrue(partitionEventPosition.isEmpty()); + } + + @Test + void testContainerEmpty() { + when(blobContainerAsyncClient.exists()).thenReturn(Mono.just(Boolean.TRUE)); + + final PagedFlux blobItems = new PagedFlux<>(() -> Mono.just(pagedResponse)); + when(blobContainerAsyncClient.listBlobs(any(ListBlobsOptions.class))).thenReturn(blobItems); + + final Map partitionEventPosition = provider.getInitialPartitionEventPosition(); + + assertNotNull(partitionEventPosition); + assertTrue(partitionEventPosition.isEmpty()); + } + + @Test + void testSequenceNumberNotFound() { + setBlobData(EMPTY_OBJECT); + + final Map partitionEventPosition = provider.getInitialPartitionEventPosition(); + + assertNotNull(partitionEventPosition); + assertTrue(partitionEventPosition.isEmpty()); + } + + @Test + void testSequenceNumberFound() { + setBlobData(SEQUENCE_NUMBER_OBJECT); + + final Map 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 blobItems = IterableStream.of(Arrays.asList(directoryBlobItem, blobItem)); + when(pagedResponse.getElements()).thenReturn(blobItems); + final PagedFlux 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)); + } +} diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/src/test/java/org/apache/nifi/services/azure/eventhub/TestAzureEventHubRecordSink.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/eventhub/TestAzureEventHubRecordSink.java similarity index 100% rename from nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/src/test/java/org/apache/nifi/services/azure/eventhub/TestAzureEventHubRecordSink.java rename to nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/eventhub/TestAzureEventHubRecordSink.java diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink-nar/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink-nar/pom.xml deleted file mode 100644 index 912ce0a1c6..0000000000 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink-nar/pom.xml +++ /dev/null @@ -1,45 +0,0 @@ - - - - 4.0.0 - - - org.apache.nifi - nifi-azure-bundle - 1.18.0-SNAPSHOT - - - nifi-azure-record-sink-nar - nar - - true - true - - - - - org.apache.nifi - nifi-azure-record-sink - 1.18.0-SNAPSHOT - - - org.apache.nifi - nifi-standard-services-api-nar - 1.18.0-SNAPSHOT - nar - - - diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/pom.xml deleted file mode 100644 index a8d6073937..0000000000 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/pom.xml +++ /dev/null @@ -1,73 +0,0 @@ - - - - - nifi-azure-bundle - org.apache.nifi - 1.18.0-SNAPSHOT - - 4.0.0 - - nifi-azure-record-sink - - - - com.azure - azure-messaging-eventhubs - 5.12.0 - - - com.azure - azure-identity - ${azure.identity.version} - - - org.apache.nifi - nifi-record-sink-api - - - org.apache.nifi - nifi-record - - - org.apache.nifi - nifi-record-serialization-service-api - - - org.apache.nifi - nifi-api - - - org.apache.nifi - nifi-utils - 1.18.0-SNAPSHOT - - - org.mockito - mockito-inline - ${mockito.version} - test - - - org.apache.nifi - nifi-mock - test - - - org.apache.nifi - nifi-mock-record-utils - test - - - \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService deleted file mode 100644 index 2684da536e..0000000000 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-record-sink/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService +++ /dev/null @@ -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 \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-reporting-task/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-reporting-task/pom.xml index 7aa41af097..8c7a87174e 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-reporting-task/pom.xml +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-reporting-task/pom.xml @@ -20,7 +20,6 @@ nifi-azure-reporting-task jar - org.apache.nifi @@ -37,6 +36,11 @@ nifi-metrics 1.18.0-SNAPSHOT + + org.apache.nifi + nifi-reporting-utils + 1.18.0-SNAPSHOT + com.google.code.gson gson @@ -53,7 +57,6 @@ org.apache.commons commons-lang3 - org.mockito mockito-core @@ -65,11 +68,5 @@ 1.18.0-SNAPSHOT test - - - org.apache.nifi - nifi-reporting-utils - 1.18.0-SNAPSHOT - diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/pom.xml index db98469db0..9e075caa2b 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/pom.xml +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/pom.xml @@ -27,32 +27,14 @@ com.microsoft.azure azure-storage - - - com.google.code.findbugs - jsr305 - - com.azure azure-core - ${azure.core.version} com.azure azure-cosmos - ${azure-cosmos.version} - - - com.azure - azure-core - - - com.google.code.findbugs - jsr305 - - org.apache.nifi diff --git a/nifi-nar-bundles/nifi-azure-bundle/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/pom.xml index f617b0a3cd..92618b7caa 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/pom.xml +++ b/nifi-nar-bundles/nifi-azure-bundle/pom.xml @@ -26,12 +26,9 @@ pom - 8.6.6 - 1.26.0 - 1.4.5 - - 1.11.0 - 4.26.0 + 1.2.6 + 8.6.6 + 1.13.0 @@ -41,16 +38,21 @@ nifi-azure-nar nifi-azure-services-api nifi-azure-services-api-nar - nifi-azure-record-sink - nifi-azure-record-sink-nar + + com.azure + azure-sdk-bom + ${azure.sdk.bom.version} + pom + import + com.microsoft.azure azure-storage - ${azure-storage.version} + ${microsoft.azure-storage.version} org.apache.commons