From 507e91dc4dd07f5a7dcc230bc784a1a71fdff7b8 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Thu, 21 Sep 2017 17:16:45 -0700 Subject: [PATCH 01/34] added gradle changes --- build.gradle | 5 +++++ gradle/wrapper/gradle-wrapper.properties | 4 ++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index 3a6f9b6d66..959ddd0493 100644 --- a/build.gradle +++ b/build.gradle @@ -192,7 +192,9 @@ project(':samza-azure') { dependencies { compile "com.microsoft.azure:azure-storage:5.3.1" + compile "com.microsoft.azure:azure-eventhubs:0.14.5" compile "com.fasterxml.jackson.core:jackson-core:2.8.8" + compile "io.dropwizard.metrics:metrics-core:3.1.2" compile project(':samza-api') compile project(":samza-core_$scalaVersion") compile "org.slf4j:slf4j-api:$slf4jVersion" @@ -202,6 +204,9 @@ project(':samza-azure') { configFile = new File(rootDir, "checkstyle/checkstyle.xml") toolVersion = "$checkstyleVersion" } + test { + exclude 'org/apache/samza/system/eventhub/**' + } } diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 55720c38e2..f742bf3ea0 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,6 @@ -#Mon Oct 31 23:13:44 PDT 2016 +#Thu Sep 07 09:55:54 PDT 2017 distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-2.8-bin.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-2.8-all.zip From 120e182626bfa518b504ff1901969e3c759e5f80 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Thu, 21 Sep 2017 17:20:11 -0700 Subject: [PATCH 02/34] initial commit, consumer, producer, admin --- .../system/eventhub/EventDataWrapper.java | 34 +++ .../eventhub/EventHubClientWrapper.java | 150 ++++++++++ .../samza/system/eventhub/EventHubConfig.java | 131 +++++++++ .../eventhub/EventHubSystemFactory.java | 50 ++++ .../eventhub/admin/EventHubSystemAdmin.java | 112 ++++++++ .../consumer/EventHubEntityConnection.java | 107 +++++++ .../EventHubEntityConnectionFactory.java | 8 + .../consumer/EventHubSystemConsumer.java | 230 +++++++++++++++ .../eventhub/metrics/SamzaHistogram.java | 43 +++ .../producer/EventHubSystemProducer.java | 266 ++++++++++++++++++ .../samza/system/eventhub/MockEventData.java | 38 +++ .../system/eventhub/TestMetricsRegistry.java | 66 +++++ .../admin/TestEventHubSystemAdmin.java | 65 +++++ .../MockEventHubEntityConnectionFactory.java | 49 ++++ .../consumer/TestEventHubSystemConsumer.java | 175 ++++++++++++ .../producer/TestEventHubSystemProducer.java | 180 ++++++++++++ 16 files changed, 1704 insertions(+) create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java new file mode 100644 index 0000000000..eb17c28e74 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java @@ -0,0 +1,34 @@ +package org.apache.samza.system.eventhub; + +import com.microsoft.azure.eventhubs.EventData; + +/** + * Simpler wrapper of {@link EventData} events with the decrypted payload + */ +public class EventDataWrapper { + private final EventData _eventData; + private final byte[] _body; + + public EventDataWrapper(EventData eventData, byte[] body) { + _eventData = eventData; + _body = body; + } + + public EventData getEventData() { + return _eventData; + } + + /** + * @return the body of decrypted body of the message. In case not encryption is setup for this topic + * just returns the body of the message. + */ + public byte[] getDecryptedBody() { + return _body; + } + + @Override + public String toString() { + return "EventDataWrapper: body: " + (new String(_body)) + ", EventData " + _eventData; + } + +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java new file mode 100644 index 0000000000..555a442dfe --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java @@ -0,0 +1,150 @@ +package org.apache.samza.system.eventhub; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.eventhubs.PartitionSender; +import com.microsoft.azure.servicebus.ClientConstants; +import com.microsoft.azure.servicebus.ConnectionStringBuilder; +import com.microsoft.azure.servicebus.ServiceBusException; +import org.apache.samza.SamzaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.*; + +public class EventHubClientWrapper { + private static final Logger LOG = LoggerFactory.getLogger(EventHubClientWrapper.class.getName()); + + public enum PartitioningMethod { + EVENT_HUB_HASHING, + + PARTITION_KEY_AS_PARTITION, + } + + private final PartitioningMethod _partitioningMethod; + private final int _numPartitions; + + public EventHubClient getEventHubClient() { + return _eventHubClient; + } + + private EventHubClient _eventHubClient; + private Map _partitionSenders = new HashMap<>(); + + public EventHubClientWrapper(PartitioningMethod partitioningMethod, int numPartitions, + String eventHubNamespace, String entityPath, String sasKeyName, String sasToken) { + String remoteHost = String.format("%s.servicebus.windows.net", eventHubNamespace); + _partitioningMethod = partitioningMethod; + _numPartitions = numPartitions; + try { + // Create a event hub connection string pointing to localhost + ConnectionStringBuilder connectionStringBuilder = + new ConnectionStringBuilder(eventHubNamespace, entityPath, sasKeyName, sasToken); + + _eventHubClient = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString()); + } catch (IOException ioe) { + throw new IllegalStateException( + "Failed to connect to remote host " + remoteHost + ":" + ClientConstants.AMQPS_PORT, ioe); + } catch (ServiceBusException e) { + String msg = String.format("Creation of event hub client failed for eventHub %s %s %s %s %s %s with exception", entityPath, partitioningMethod, numPartitions, eventHubNamespace, sasKeyName, sasToken); + LOG.error(msg, e); + throw new SamzaException(msg, e); + } + } + + public void closeSync() { + _partitionSenders.entrySet().forEach(x -> { + try { + x.getValue().closeSync(); + } catch (ServiceBusException e) { + LOG.warn("Closing the partition sender failed for partition " + x.getKey(), e); + } + }); + + try { + _eventHubClient.closeSync(); + } catch (ServiceBusException e) { + LOG.warn("Closing the event hub client failed ", e); + } + } + + /** + * Timed connection close. + * @param timeoutMS Time in Miliseconds to wait for individual components (partition senders, event hub client and + * tunnel in order) to shutdown before moving to the next stage. + * For example a timeoutMS of 30000, will result in a wait of max 30secs for a successful close of + * all partition senders followed by a max 30secs wait for a successful close of eventhub client and + * then another max 30secs wait for successful tunnel close. Tunnel close failure/timeout will result + * in RuntimeException. + */ + + public void closeSync(long timeoutMS) { + List> futures = new ArrayList<>(); + _partitionSenders.entrySet().forEach(x -> futures.add(x.getValue().close())); + CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); + try { + future.get(timeoutMS, TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + LOG.warn("Closing the partition sender failed ", e); + } + + future = _eventHubClient.close(); + try { + future.get(timeoutMS, TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + LOG.warn("Closing the event hub client failed ", e); + } + } + + + public CompletableFuture send(EventData eventData, Object partitionKey) { + if (_partitioningMethod == PartitioningMethod.EVENT_HUB_HASHING) { + return _eventHubClient.send(eventData, convertPartitionKeyToString(partitionKey)); + } else if (_partitioningMethod == PartitioningMethod.PARTITION_KEY_AS_PARTITION) { + if (!(partitionKey instanceof Integer)) { + String msg = "Partition key should be of type Integer"; + LOG.error(msg); + throw new SamzaException(msg); + } + + PartitionSender sender = getPartitionSender((int) partitionKey); + return sender.send(eventData); + } else { + throw new SamzaException("Unknown partitioning method " + _partitioningMethod); + } + } + + private String convertPartitionKeyToString(Object partitionKey) { + if (partitionKey instanceof String) { + return (String) partitionKey; + } else if (partitionKey instanceof Integer) { + return String.valueOf(partitionKey); + } else if (partitionKey instanceof byte[]) { + return new String((byte[]) partitionKey, Charset.defaultCharset()); + } else { + throw new SamzaException("Unsupported key type: " + partitionKey.getClass().toString()); + } + } + + private PartitionSender getPartitionSender(int partition) { + if (!_partitionSenders.containsKey(partition)) { + try { + PartitionSender partitionSender = + _eventHubClient.createPartitionSenderSync(String.valueOf(partition % _numPartitions)); + _partitionSenders.put(partition, partitionSender); + } catch (ServiceBusException e) { + String msg = "Creation of partition sender failed with exception"; + LOG.error(msg, e); + throw new SamzaException(msg, e); + } + } + + return _partitionSenders.get(partition); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java new file mode 100644 index 0000000000..6944a17292 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -0,0 +1,131 @@ +package org.apache.samza.system.eventhub; + +import com.microsoft.azure.eventhubs.EventHubClient; +import org.apache.commons.lang3.StringUtils; +import org.apache.samza.config.MapConfig; +import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.SerdeFactory; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +public class EventHubConfig extends MapConfig { + public static final String CONFIG_STREAM_LIST = "systems.%s.stream.list"; + + public static final String CONFIG_STREAM_NAMESPACE = "systems.%s.streams.%s.eventhubs.namespace"; + + public static final String CONFIG_STREAM_ENTITYPATH = "systems.%s.streams.%s.eventhubs.entitypath"; + + public static final String CONFIG_STREAM_SAS_KEY_NAME = "systems.%s.streams.%s.eventhubs.sas.keyname"; + + public static final String CONFIG_STREAM_SAS_TOKEN = "systems.%s.streams.%s.eventhubs.sas.token"; + + public static final String CONFIG_STREAM_SERDE_FACTORY = "systems.%s.streams.%s.eventhubs.serdeFactory"; + public static final String CONFIG_STREAM_SERDE_PREFIX = "systems.%s.streams.%s.eventhubs.serde."; + + public static final String CONFIG_STREAM_CONSUMER_GROUP = "systems.%s.streams.%s.eventhubs.consumer.group"; + public static final String DEFAULT_CONFIG_STREAM_CONSUMER_GROUP = EventHubClient.DEFAULT_CONSUMER_GROUP_NAME; + + public static final String CONFIG_STREAM_CONSUMER_START_POSITION = "systems.%s.streams.%s.eventhubs.start.position"; + public static final String DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION = StartPosition.LATEST.name(); + + public enum StartPosition { + EARLIEST, + LATEST + } + + private final String _system; + + public EventHubConfig(Map config, String systemName) { + super(config); + _system = systemName; + } + + /** + * Get the list of streams that are defined. Each stream has enough + * information for connecting to a certain EventHub entity. + * @return list of stream names + */ + public List getStreamList() { + return getList(String.format(CONFIG_STREAM_LIST, _system)); + } + + /** + * Get the EventHubs namespace for the stream + * @param streamName name of stream + * @return EventHubs namespace + */ + public String getStreamNamespace(String streamName) { + return get(String.format(CONFIG_STREAM_NAMESPACE, _system, streamName)); + } + + /** + * Get the EventHubs entity path (topic name) for the stream + * @param streamName name of stream + * @return EventHubs entity path + */ + public String getStreamEntityPath(String streamName) { + return get(String.format(CONFIG_STREAM_ENTITYPATH, _system, streamName)); + } + + /** + * Get the EventHubs SAS (Shared Access Signature) key name for the stream + * @param streamName name of stream + * @return EventHubs SAS key name + */ + public String getStreamSasKeyName(String streamName) { + return get(String.format(CONFIG_STREAM_SAS_KEY_NAME, _system, streamName)); + } + + /** + * Get the EventHubs SAS (Shared Access Signature) token for the stream + * @param streamName name of stream + * @return EventHubs SAS token + */ + public String getStreamSasToken(String streamName) { + return get(String.format(CONFIG_STREAM_SAS_TOKEN, _system, streamName)); + } + + public Optional> getSerde(String streamName) { + Serde serde = null; + String serdeFactoryClassName = this.get(String.format(CONFIG_STREAM_SERDE_FACTORY, _system, streamName)); + if (!StringUtils.isEmpty(serdeFactoryClassName)) { + SerdeFactory factory = EventHubSystemFactory.getSerdeFactory(serdeFactoryClassName); + serde = factory.getSerde(streamName, this.subset(String.format(CONFIG_STREAM_SERDE_PREFIX, _system, streamName))); + } + return Optional.ofNullable(serde); + } + + /** + * Get the EventHubs consumer group used for consumption for the stream + * @param streamName name of stream + * @return EventHubs consumer group + */ + public String getStreamConsumerGroup(String streamName) { + return get(String.format(CONFIG_STREAM_CONSUMER_GROUP, _system, streamName), DEFAULT_CONFIG_STREAM_CONSUMER_GROUP); + } + + /** + * Get the start position when there is no checkpoints. By default the consumer starts from latest (end of stream) + * @param streamName name of the stream + * @return Starting position when no checkpoints + */ + public StartPosition getStartPosition(String streamName) { + String startPositionStr = get(String.format(CONFIG_STREAM_CONSUMER_START_POSITION, _system, streamName), + DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION); + return StartPosition.valueOf(startPositionStr.toUpperCase()); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), _system); + } + + @Override + public boolean equals(Object obj) { + return super.equals(obj) && _system.equals(((EventHubConfig) obj)._system); + } + +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java new file mode 100644 index 0000000000..839ecf06fd --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java @@ -0,0 +1,50 @@ +package org.apache.samza.system.eventhub; + +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.serializers.SerdeFactory; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemConsumer; +import org.apache.samza.system.SystemFactory; +import org.apache.samza.system.SystemProducer; +import org.apache.samza.system.eventhub.admin.EventHubSystemAdmin; +import org.apache.samza.system.eventhub.consumer.EventHubEntityConnectionFactory; +import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; +import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; + +import java.lang.reflect.Constructor; +import java.util.HashMap; +import java.util.Map; + +public class EventHubSystemFactory implements SystemFactory { + + @Override + public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { + return new EventHubSystemConsumer(new EventHubConfig(config, systemName), + new EventHubEntityConnectionFactory(), registry); + } + + @Override + public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { + return new EventHubSystemProducer(systemName, config, registry); + } + + @Override + public SystemAdmin getAdmin(String systemName, Config config) { + return new EventHubSystemAdmin(systemName, new EventHubConfig(config, systemName)); + } + + @SuppressWarnings("unchecked") + public static SerdeFactory getSerdeFactory(String serdeFactoryClassName) { + SerdeFactory factory; + try { + Class> classObj = (Class>) Class.forName(serdeFactoryClassName); + Constructor> ctor = classObj.getDeclaredConstructor(); + factory = ctor.newInstance(); + } catch (Exception e) { + throw new SamzaException("Failed to create Serde Factory for: " + serdeFactoryClassName, e); + } + return factory; + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java new file mode 100644 index 0000000000..350e602ac8 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -0,0 +1,112 @@ +package org.apache.samza.system.eventhub.admin; + +import com.microsoft.azure.eventhubs.EventHubRuntimeInformation; +import com.microsoft.azure.eventhubs.PartitionReceiver; +import org.apache.samza.Partition; +import org.apache.samza.SamzaException; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemStreamMetadata; +import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.system.eventhub.EventHubClientWrapper; +import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +public class EventHubSystemAdmin implements SystemAdmin { + private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemAdmin.class); + + private String _systemName; + private EventHubConfig _config; + private Map _eventHubClients = new HashMap<>(); + + public EventHubSystemAdmin(String systemName, EventHubConfig config) { + _systemName = systemName; + _config = config; + } + + private static String getNextOffset(String currentOffset) { + // EventHub will return the first message AFTER the offset + // that was specified in the fetch request. + return currentOffset.equals(EventHubSystemConsumer.END_OF_STREAM) ? currentOffset : + String.valueOf(Long.parseLong(currentOffset) + 1); + } + + @Override + public Map getOffsetsAfter(Map offsets) { + Map results = new HashMap<>(); + offsets.forEach((partition, offset) -> results.put(partition, getNextOffset(offset))); + return results; + } + + @Override + public Map getSystemStreamMetadata(Set streamNames) { + Map requestedMetadata = new HashMap<>(); + Map> ehRuntimeInfos = new HashMap<>(); + streamNames.forEach((streamName) -> { + if (!_eventHubClients.containsKey(streamName)) { + addEventHubClient(streamName); + } + ehRuntimeInfos.put(streamName, + _eventHubClients.get(streamName).getEventHubClient().getRuntimeInformation()); + }); + ehRuntimeInfos.forEach((streamName, ehRuntimeInfo) -> { + try { + EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(); // TODO: timeout + Map sspMetadataMap = new HashMap<>(); + for (String partition : ehInfo.getPartitionIds()) { //TODO getPartitionRuntimeInformation + sspMetadataMap.put(new Partition(Integer.parseInt(partition)), + new SystemStreamMetadata.SystemStreamPartitionMetadata(PartitionReceiver.START_OF_STREAM, + EventHubSystemConsumer.END_OF_STREAM, EventHubSystemConsumer.END_OF_STREAM)); + } + requestedMetadata.put(streamName, new SystemStreamMetadata(streamName, sspMetadataMap)); + } catch (Exception e){ + String msg = String.format("Error while fetching EventHubRuntimeInfo for System:%s, Stream:%s", + _systemName, streamName); + LOG.error(msg); + throw new SamzaException(msg); + } + }); + return requestedMetadata; + } + + private void addEventHubClient(String streamName) { + String ehNamespace = _config.getStreamNamespace(streamName); + String ehEntityPath = _config.getStreamEntityPath(streamName); + _eventHubClients.put(streamName, new EventHubClientWrapper(null, 0, + ehNamespace, ehEntityPath, _config.getStreamSasKeyName(streamName), _config.getStreamSasToken(streamName))); + } + + @Override + public void createChangelogStream(String streamName, int numOfPartitions) { + throw new UnsupportedOperationException("Event Hubs does not support change log stream."); + } + + @Override + public void validateChangelogStream(String streamName, int numOfPartitions) { + throw new UnsupportedOperationException("Event Hubs does not support change log stream."); + } + + @Override + public void createCoordinatorStream(String streamName) { + throw new UnsupportedOperationException("Event Hubs does not support coordinator stream."); + } + + @Override + public Integer offsetComparator(String offset1, String offset2) { + try { + if (offset1.equals(EventHubSystemConsumer.END_OF_STREAM)) { + return offset2.equals(EventHubSystemConsumer.END_OF_STREAM) ? 0 : 1; + } + return offset2.equals(EventHubSystemConsumer.END_OF_STREAM) ? -1 : + Long.compare(Long.parseLong(offset1), Long.parseLong(offset2)); + } catch (NumberFormatException exception) { + return null; + } + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java new file mode 100644 index 0000000000..409efac75b --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java @@ -0,0 +1,107 @@ +package org.apache.samza.system.eventhub.consumer; + +import com.microsoft.azure.eventhubs.PartitionReceiveHandler; +import com.microsoft.azure.eventhubs.PartitionReceiver; +import com.microsoft.azure.servicebus.ServiceBusException; +import com.microsoft.azure.servicebus.StringUtil; +import org.apache.samza.SamzaException; +import org.apache.samza.system.eventhub.EventHubClientWrapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Instant; +import java.util.Map; +import java.util.TreeMap; + +public class EventHubEntityConnection { + private static final Logger LOG = LoggerFactory.getLogger(EventHubEntityConnection.class); + + private final String _namespace; + private final String _entityPath; + private final String _sasKeyName; + private final String _sasKey; + private final String _consumerName; + private final Map _receivers = new TreeMap<>(); + private EventHubClientWrapper _ehClientWrapper; + private boolean _isStarted = false; + + final Map _offsets = new TreeMap<>(); + final Map _handlers = new TreeMap<>(); + + EventHubEntityConnection(String namespace, String entityPath, String sasKeyName, String sasKey, String consumerName) { + _namespace = namespace; + _entityPath = entityPath; + _sasKeyName = sasKeyName; + _sasKey = sasKey; + _consumerName = consumerName; + } + + // add partitions and handlers for this connection. This can be called multiple times + // for multiple partitions, but needs to be called before connectAndStart() + synchronized void addPartition(int partitionId, String offset, PartitionReceiveHandler handler) { + if (_isStarted) { + LOG.warn("Trying to add partition when the connection has already started."); + return; + } + _offsets.put(partitionId, offset); + _handlers.put(partitionId, handler); + } + + // establish the connection and start consuming events + synchronized void connectAndStart() { + _isStarted = true; + try { + LOG.info(String.format("Starting connection for namespace=%s, entity=%s ", _namespace, _entityPath)); + // upon the instantiation of the client, the connection will be established + _ehClientWrapper = + new EventHubClientWrapper(null, 0, _namespace, _entityPath, _sasKeyName, _sasKey); + for (Map.Entry entry : _offsets.entrySet()) { + Integer id = entry.getKey(); + String offset = entry.getValue(); + try { + PartitionReceiver receiver; + if (StringUtil.isNullOrWhiteSpace(offset)) { + throw new SamzaException( + String.format("Invalid offset %s namespace=%s, entity=%s", offset, _namespace, _entityPath)); + } + if (offset.equals(EventHubSystemConsumer.END_OF_STREAM)) { + receiver = _ehClientWrapper.getEventHubClient() + .createReceiverSync(_consumerName, id.toString(), Instant.now()); + } else { + receiver = _ehClientWrapper.getEventHubClient() + .createReceiverSync(_consumerName, id.toString(), offset, + !offset.equals(PartitionReceiver.START_OF_STREAM)); + } + receiver.setReceiveHandler(_handlers.get(id)); + _receivers.put(id, receiver); + } catch (Exception e) { + throw new SamzaException( + String.format("Failed to create receiver for EventHubs: namespace=%s, entity=%s, partitionId=%d", + _namespace, _entityPath, id), e); + } + } + } catch (Exception e) { + throw new SamzaException( + String.format("Failed to create connection to EventHubs: namespace=%s, entity=%s", _namespace, _entityPath), + e); + } + LOG.info(String.format("Connection successfully started for namespace=%s, entity=%s ", _namespace, _entityPath)); + } + + synchronized void stop() { + LOG.info(String.format("Stopping connection for namespace=%s, entity=%s ", _namespace, _entityPath)); + try { + for (PartitionReceiver receiver : _receivers.values()) { + receiver.closeSync(); + } + _ehClientWrapper.closeSync(); + } catch (ServiceBusException e) { + throw new SamzaException( + String.format("Failed to stop connection for namespace=%s, entity=%s ", _namespace, _entityPath), e); + } + _isStarted = false; + _offsets.clear(); + _handlers.clear(); + LOG.info(String.format("Connection for namespace=%s, entity=%s stopped", _namespace, _entityPath)); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java new file mode 100644 index 0000000000..e0dc41b10c --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java @@ -0,0 +1,8 @@ +package org.apache.samza.system.eventhub.consumer; + +public class EventHubEntityConnectionFactory { + EventHubEntityConnection createConnection(String namespace, String entityPath, String sasKeyName, String sasKey, + String consumerName) { + return new EventHubEntityConnection(namespace, entityPath, sasKeyName, sasKey, consumerName); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java new file mode 100644 index 0000000000..34b954448c --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -0,0 +1,230 @@ +package org.apache.samza.system.eventhub.consumer; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.PartitionReceiveHandler; +import com.microsoft.azure.eventhubs.PartitionReceiver; +import com.microsoft.azure.servicebus.StringUtil; +import org.apache.samza.SamzaException; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.serializers.Serde; +import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.system.eventhub.EventDataWrapper; +import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.admin.EventHubSystemAdmin; +import org.apache.samza.system.eventhub.metrics.SamzaHistogram; +import org.apache.samza.util.BlockingEnvelopeMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.time.Instant; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + + +/** + * Implementation of a system consumer for EventHubs. For each system stream + * partition, it registers a handler with the EventHubsClient which constantly + * push data into a block queue. This class extends the BlockingEnvelopeMap + * provided by samza-api to to simplify the logic around those blocking queues. + * + * A high level architecture: + * + * ┌───────────────────────────────────────────────┐ + * │ EventHubsClient │ + * │ │ + * │ ┌───────────────────────────────────────┐ │ ┌─────────────────────┐ + * │ │ │ │ │ │ + * │ │ PartitionReceiveHandler_1 │───┼───────▶│ SSP1-BlockingQueue ├──────┐ + * │ │ │ │ │ │ │ + * │ └───────────────────────────────────────┘ │ └─────────────────────┘ │ + * │ │ │ + * │ ┌───────────────────────────────────────┐ │ ┌─────────────────────┐ │ + * │ │ │ │ │ │ │ + * │ │ PartitionReceiveHandler_2 │───┼───────▶│ SSP2-BlockingQueue ├──────┤ ┌──────────────────────────┐ + * │ │ │ │ │ │ ├───────▶│ │ + * │ └───────────────────────────────────────┘ │ └─────────────────────┘ └───────▶│ SystemConsumer.poll() │ + * │ │ ┌───────▶│ │ + * │ │ │ └──────────────────────────┘ + * │ ... │ ... │ + * │ │ │ + * │ │ │ + * │ ┌───────────────────────────────────────┐ │ ┌─────────────────────┐ │ + * │ │ │ │ │ │ │ + * │ │ PartitionReceiveHandler_N │───┼───────▶│ SSPN-BlockingQueue ├──────┘ + * │ │ │ │ │ │ + * │ └───────────────────────────────────────┘ │ └─────────────────────┘ + * │ │ + * │ │ + * └───────────────────────────────────────────────┘ + * + */ +public class EventHubSystemConsumer extends BlockingEnvelopeMap { + + private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemConsumer.class); + + public static final String START_OF_STREAM = PartitionReceiver.START_OF_STREAM; // -1 + public static final String END_OF_STREAM = "-2"; + private static final int MAX_EVENT_COUNT_PER_PARTITION_POLL = 50; + private static final int BLOCKING_QUEUE_SIZE = 100; + + private final Map _connections = new HashMap<>(); + private final Map> _serdes = new HashMap<>(); + private final EventHubConfig _config; + + private static Counter _aggEventReadRate = null; + private static Counter _aggEventByteReadRate = null; + private static SamzaHistogram _aggReadLatency = null; + private static Counter _aggReadErrors = null; + + private Map _eventReadRates; + private Map _eventByteReadRates; + private Map _readLatencies; + private Map _readErrors; + + public static final String AGGREGATE = "aggregate"; + + public static final String EVENT_READ_RATE = "eventReadRate"; + public static final String EVENT_BYTE_READ_RATE = "eventByteReadRate"; + public static final String READ_LATENCY = "readLatency"; + public static final String READ_ERRORS = "readErrors"; + + public EventHubSystemConsumer(EventHubConfig config, EventHubEntityConnectionFactory connectionFactory, + MetricsRegistry registry) { + super(registry, System::currentTimeMillis); + + _config = config; + List streamList = config.getStreamList(); + streamList.forEach(stream -> { + _connections.put(stream, connectionFactory.createConnection( + config.getStreamNamespace(stream), config.getStreamEntityPath(stream), + config.getStreamSasKeyName(stream), config.getStreamSasToken(stream), + config.getStreamConsumerGroup(stream))); + _serdes.put(stream, config.getSerde(stream).orElse(null)); + }); + _eventReadRates = streamList.stream() + .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_READ_RATE))); + _eventByteReadRates = streamList.stream() + .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_BYTE_READ_RATE))); + _readLatencies = streamList.stream() + .collect(Collectors.toMap(Function.identity(), x -> new SamzaHistogram(registry, x, READ_LATENCY))); + _readErrors = + streamList.stream().collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, READ_ERRORS))); + + // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. + synchronized (AGGREGATE) { + if (_aggEventReadRate == null) { + _aggEventReadRate = registry.newCounter(AGGREGATE, EVENT_READ_RATE); + _aggEventByteReadRate = registry.newCounter(AGGREGATE, EVENT_BYTE_READ_RATE); + _aggReadLatency = new SamzaHistogram(registry, AGGREGATE, READ_LATENCY); + _aggReadErrors = registry.newCounter(AGGREGATE, READ_ERRORS); + } + } + } + + @Override + public void register(SystemStreamPartition systemStreamPartition, String offset) { + super.register(systemStreamPartition, offset); + String stream = systemStreamPartition.getStream(); + EventHubEntityConnection connection = _connections.get(stream); + if (connection == null) { + throw new SamzaException("No EventHub connection for " + stream); + } + + if (StringUtil.isNullOrWhiteSpace(offset)) { + switch (_config.getStartPosition(systemStreamPartition.getStream())) { + case EARLIEST: + offset = START_OF_STREAM; + break; + case LATEST: + offset = END_OF_STREAM; + break; + default: + throw new SamzaException( + "Unknown starting position config " + _config.getStartPosition(systemStreamPartition.getStream())); + } + } + connection.addPartition(systemStreamPartition.getPartition().getPartitionId(), offset, + new PartitionReceiverHandlerImpl(systemStreamPartition, _eventReadRates.get(stream), + _eventByteReadRates.get(stream), _readLatencies.get(stream), _readErrors.get(stream), + _serdes.get(stream))); + } + + @Override + public void start() { + _connections.values().forEach(EventHubEntityConnection::connectAndStart); + } + + @Override + public void stop() { + _connections.values().forEach(EventHubEntityConnection::stop); + } + + private class PartitionReceiverHandlerImpl extends PartitionReceiveHandler { + + private final Counter _eventReadRate; + private final Counter _eventByteReadRate; + private final SamzaHistogram _readLatency; + private final Counter _errors; + private final Serde _serde; + SystemStreamPartition _ssp; + + PartitionReceiverHandlerImpl(SystemStreamPartition ssp, Counter eventReadRate, Counter eventByteReadRate, + SamzaHistogram readLatency, Counter readErrors, Serde serde) { + super(MAX_EVENT_COUNT_PER_PARTITION_POLL); + _ssp = ssp; + _eventReadRate = eventReadRate; + _eventByteReadRate = eventByteReadRate; + _readLatency = readLatency; + _errors = readErrors; + _serde = serde; + } + + @Override + public void onReceive(Iterable events) { + if (events != null) { + + events.forEach(event -> { + byte[] decryptedBody = event.getBody(); + if (_serde != null) { + decryptedBody = _serde.fromBytes(decryptedBody); + } + EventDataWrapper wrappedEvent = new EventDataWrapper(event, decryptedBody); + try { + updateMetrics(event); + // note that the partition key can be null + put(_ssp, new IncomingMessageEnvelope(_ssp, event.getSystemProperties().getOffset(), + event.getSystemProperties().getPartitionKey(), wrappedEvent)); + } catch (Exception e) { + String msg = String.format("Exception while adding the event from ssp %s to dispatch queue.", _ssp); + LOG.error(msg, e); + throw new SamzaException(msg, e); + } + }); + } + } + + private void updateMetrics(EventData event) { + _eventReadRate.inc(); + _aggEventReadRate.inc(); + _eventByteReadRate.inc(event.getBodyLength()); + _aggEventByteReadRate.inc(event.getBodyLength()); + long latencyMs = Duration.between(Instant.now(), event.getSystemProperties().getEnqueuedTime()).toMillis(); + _readLatency.update(latencyMs); + _aggReadLatency.update(latencyMs); + } + + @Override + public void onError(Throwable throwable) { + // TODO error handling + _errors.inc(); + _aggReadErrors.inc(); + LOG.error(String.format("Received error from event hub connection (ssp=%s): ", _ssp), throwable); + } + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java new file mode 100644 index 0000000000..c5b658ab3e --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java @@ -0,0 +1,43 @@ +package org.apache.samza.system.eventhub.metrics; + +import com.codahale.metrics.ExponentiallyDecayingReservoir; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.MetricsRegistry; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + + + +public class SamzaHistogram { + private final MetricsRegistry _registry; + private final Histogram _histogram; + private final List _percentiles; + private final Map> _gauges; + private static final List DEFAULT_HISTOGRAM_PERCENTILES = Arrays.asList(50D, 99D); + + public SamzaHistogram(MetricsRegistry registry, String group, String name) { + this(registry, group, name, DEFAULT_HISTOGRAM_PERCENTILES); + } + + public SamzaHistogram(MetricsRegistry registry, String group, String name, List percentiles) { + _registry = registry; + _histogram = new Histogram(new ExponentiallyDecayingReservoir()); + _percentiles = percentiles; + _gauges = _percentiles.stream() + .filter(x -> x > 0 && x <= 100) + .collect( + Collectors.toMap(Function.identity(), x -> _registry.newGauge(group, name + "_" + String.valueOf(0), 0D))); + } + + public void update(long value) { + _histogram.update(value); + Snapshot values = _histogram.getSnapshot(); + _percentiles.stream().forEach(x -> _gauges.get(x).set(values.getValue(x / 100))); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java new file mode 100644 index 0000000000..10f1e12f49 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -0,0 +1,266 @@ +package org.apache.samza.system.eventhub.producer; + +import com.microsoft.azure.eventhubs.EventData; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.serializers.Serde; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemProducer; +import org.apache.samza.system.eventhub.EventHubClientWrapper; +import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.metrics.SamzaHistogram; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.time.Instant; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; + +public class EventHubSystemProducer implements SystemProducer { + public static final String EVENT_SOURCE_TIMESTAMP = "event-source-timestamp"; + public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; + private final int _destinationPartitions; + private final boolean _sendKeyInEventProperties; + private final EventHubConfig _eventHubsConfig; + private final Config _config; + private Throwable _sendExceptionOnCallback; + private boolean _isStarted; + + private static Counter _aggEventWriteRate = null; + private static Counter _aggEventByteWriteRate = null; + private static SamzaHistogram _aggSendLatency = null; + private static SamzaHistogram _aggSendCallbackLatency = null; + private static Counter _aggSendErrors = null; + public static final String AGGREGATE = "aggregate"; + + private HashMap _eventWriteRate = new HashMap<>(); + private HashMap _eventByteWriteRate = new HashMap<>(); + private HashMap _sendLatency = new HashMap<>(); + private HashMap _sendCallbackLatency = new HashMap<>(); + private HashMap _sendErrors = new HashMap<>(); + + private static final String EVENT_WRITE_RATE = "eventWriteRate"; + private static final String EVENT_BYTE_WRITE_RATE = "eventByteWriteRate"; + private static final String SEND_ERRORS = "sendErrors"; + private static final String SEND_LATENCY = "sendLatency"; + private static final String SEND_CALLBACK_LATENCY = "sendCallbackLatency"; + private static final Duration SHUTDOWN_WAIT_TIME = Duration.ofMinutes(1L); + + public final static String CONFIG_PARTITIONING_METHOD = "partitioningMethod"; + public final static String DEFAULT_PARTITIONING_METHOD = EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING.toString(); + public static final String CONFIG_DESTINATION_NUM_PARTITION = "destinationPartitions"; + public static final String CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = "sendKeyInEventProperties"; + + private final EventHubClientWrapper.PartitioningMethod _partitioningMethod; + + private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemProducer.class.getName()); + + private final String _systemName; + private final MetricsRegistry _registry; + // Map of the system name to the event hub client. + private Map _eventHubClients = new HashMap<>(); + + private long _messageId; + private Map> _serdes = new HashMap<>(); + + private Map> _pendingFutures = new ConcurrentHashMap<>(); + + public EventHubSystemProducer(String systemName, Config config, MetricsRegistry registry) { + _messageId = 0; + _systemName = systemName; + _registry = registry; + _config = config; + _partitioningMethod = + EventHubClientWrapper.PartitioningMethod.valueOf(getConfigValue(config, CONFIG_PARTITIONING_METHOD, DEFAULT_PARTITIONING_METHOD)); + + _eventHubsConfig = new EventHubConfig(config, systemName); + _sendKeyInEventProperties = + Boolean.parseBoolean(getConfigValue(config, CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, "false")); + + // TODO this should be removed when we are able to find the number of partitions. + _destinationPartitions = Integer.parseInt(getConfigValue(config, CONFIG_DESTINATION_NUM_PARTITION, "-1")); + } + + private String getConfigValue(Config config, String configKey, String defaultValue) { + String configValue = config.get(configKey, defaultValue); + + if (configValue == null) { + throw new SamzaException(configKey + " is not configured."); + } + + return configValue; + } + + @Override + public synchronized void start() { + LOG.info("Starting system producer."); + for (String eventHub : _eventHubClients.keySet()) { + _eventWriteRate.put(eventHub, _registry.newCounter(eventHub, EVENT_WRITE_RATE)); + _eventByteWriteRate.put(eventHub, _registry.newCounter(eventHub, EVENT_BYTE_WRITE_RATE)); + _sendLatency.put(eventHub, new SamzaHistogram(_registry, eventHub, SEND_LATENCY)); + _sendCallbackLatency.put(eventHub, new SamzaHistogram(_registry, eventHub, SEND_CALLBACK_LATENCY)); + _sendErrors.put(eventHub, _registry.newCounter(eventHub, SEND_ERRORS)); + } + + // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. + synchronized (AGGREGATE) { + if (_aggEventWriteRate == null) { + _aggEventWriteRate = _registry.newCounter(AGGREGATE, EVENT_WRITE_RATE); + _aggEventByteWriteRate = _registry.newCounter(AGGREGATE, EVENT_BYTE_WRITE_RATE); + _aggSendLatency = new SamzaHistogram(_registry, AGGREGATE, SEND_LATENCY); + _aggSendCallbackLatency = new SamzaHistogram(_registry, AGGREGATE, SEND_CALLBACK_LATENCY); + _aggSendErrors = _registry.newCounter(AGGREGATE, SEND_ERRORS); + } + } + + _isStarted = true; + } + + @Override + public synchronized void stop() { + LOG.info("Stopping system producer."); + _eventHubClients.values().forEach(ehClient -> ehClient.closeSync(SHUTDOWN_WAIT_TIME.toMillis())); + _eventHubClients.clear(); + } + + @Override + public synchronized void register(String streamName) { + LOG.info("Trying to register {}.", streamName); + if (_isStarted) { + String msg = "Cannot register once the producer is started."; + LOG.error(msg); + throw new SamzaException(msg); + } + + String ehNamespace = _eventHubsConfig.getStreamNamespace(streamName); + String ehName = _eventHubsConfig.getStreamEntityPath(streamName); + + EventHubClientWrapper ehClient = + new EventHubClientWrapper(_partitioningMethod, _destinationPartitions, ehNamespace, ehName, + _eventHubsConfig.getStreamSasKeyName(streamName), _eventHubsConfig.getStreamSasToken(streamName)); + + _eventHubClients.put(streamName, ehClient); + _eventHubsConfig.getSerde(streamName).ifPresent(x -> _serdes.put(streamName, x)); + } + + @Override + public synchronized void send(String destination, OutgoingMessageEnvelope envelope) { + if (!_isStarted) { + throw new SamzaException("Trying to call send before the producer is started."); + } + + if (!_eventHubClients.containsKey(destination)) { + String msg = String.format("Trying to send event to a destination {%s} that is not registered.", destination); + LOG.error(msg); + throw new SamzaException(msg); + } + + if (_sendExceptionOnCallback != null) { + SamzaException e = new SamzaException(_sendExceptionOnCallback); + _sendExceptionOnCallback = null; + _pendingFutures.clear(); + LOG.error("One of the previous sends failed."); + throw e; + } + + EventData eventData = createEventData(destination, envelope); + + _eventWriteRate.get(destination).inc(); + _aggEventWriteRate.inc(); + _eventByteWriteRate.get(destination).inc(eventData.getBodyLength()); + _aggEventByteWriteRate.inc(eventData.getBodyLength()); + EventHubClientWrapper ehClient = _eventHubClients.get(destination); + + Instant startTime = Instant.now(); + + CompletableFuture sendResult; + sendResult = ehClient.send(eventData, envelope.getPartitionKey()); + + Instant endTime = Instant.now(); + long latencyMs = Duration.between(startTime, endTime).toMillis(); + _sendLatency.get(destination).update(latencyMs); + _aggSendLatency.update(latencyMs); + + long messageId = ++_messageId; + + // Rotate the messageIds + if (messageId == Long.MAX_VALUE) { + _messageId = 0; + } + + _pendingFutures.put(messageId, sendResult); + + // Auto remove the future from the list when they are complete. + sendResult.handle(((aVoid, throwable) -> { + long callbackLatencyMs = Duration.between(endTime, Instant.now()).toMillis(); + _sendCallbackLatency.get(destination).update(callbackLatencyMs); + _aggSendCallbackLatency.update(callbackLatencyMs); + if (throwable != null) { + _sendErrors.get(destination).inc(); + _aggSendErrors.inc(); + LOG.error("Send message to event hub: {} failed with exception: ", destination, throwable); + _sendExceptionOnCallback = throwable; + } + _pendingFutures.remove(messageId); + return aVoid; + })); + } + + private EventData createEventData(String streamName, OutgoingMessageEnvelope envelope) { + Optional> serde = Optional.ofNullable(_serdes.getOrDefault(streamName, null)); + byte[] eventValue = (byte[]) envelope.getMessage(); + if (serde.isPresent()) { + eventValue = serde.get().toBytes(eventValue); + } + + EventData eventData = new EventData(eventValue); + + eventData.getProperties().put(PRODUCE_TIMESTAMP, Long.toString(System.currentTimeMillis())); + + if (_sendKeyInEventProperties) { + String keyValue = ""; + if (envelope.getKey() != null) { + keyValue = (envelope.getKey() instanceof byte[]) ? new String((byte[]) envelope.getKey()) + : envelope.getKey().toString(); + } + eventData.getProperties().put("key", keyValue); + } + return eventData; + } + + @Override + public void flush(String source) { + LOG.info("Trying to flush pending {} sends messages: {}", _pendingFutures.size(), _pendingFutures.keySet()); + // Wait till all the pending sends are complete. + while (!_pendingFutures.isEmpty()) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + String msg = "Flush failed with error"; + LOG.error(msg, e); + throw new SamzaException(msg, e); + } + } + + if (_sendExceptionOnCallback != null) { + String msg = "Sending one of the message failed during flush"; + Throwable throwable = _sendExceptionOnCallback; + _sendExceptionOnCallback = null; + LOG.error(msg, throwable); + throw new SamzaException(msg, throwable); + } + + LOG.info("Flush succeeded."); + } + + Collection> getPendingFutures() { + return _pendingFutures.values(); + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java new file mode 100644 index 0000000000..711f85358a --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java @@ -0,0 +1,38 @@ +package org.apache.samza.system.eventhub; + +import com.microsoft.azure.eventhubs.EventData; + +import java.nio.charset.Charset; +import java.util.*; + +public class MockEventData extends EventData { + + EventData.SystemProperties _overridedSystemProperties; + + public MockEventData(byte[] data, String partitionKey, String offset) { + super(data); + HashMap properties = new HashMap<>(); + properties.put("x-opt-offset", offset); + properties.put("x-opt-partition-key", partitionKey); + properties.put("x-opt-enqueued-time", new Date(System.currentTimeMillis())); + _overridedSystemProperties = new SystemProperties(properties); + } + + @Override + public EventData.SystemProperties getSystemProperties() { + return _overridedSystemProperties; + } + + public static List generateEventData(int numEvents) { + Random rand = new Random(System.currentTimeMillis()); + List result = new ArrayList<>(); + for (int i = 0; i < numEvents; i++) { + String key = "key_" + rand.nextInt(); + String message = "message:" + rand.nextInt(); + String offset = "offset_" + i; + EventData eventData = new MockEventData(message.getBytes(Charset.defaultCharset()), key, offset); + result.add(eventData); + } + return result; + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java new file mode 100644 index 0000000000..4245307d5e --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java @@ -0,0 +1,66 @@ +package org.apache.samza.system.eventhub; + +import org.apache.commons.collections4.map.HashedMap; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.metrics.Timer; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class TestMetricsRegistry implements MetricsRegistry{ + + private Map> _counters = new HashedMap<>(); + private Map>> _gauges = new HashedMap<>(); + + public List getCounters(String groupName) { + return _counters.get(groupName); + } + + public List> getGauges(String groupName) { + return _gauges.get(groupName); + } + + @Override + public Counter newCounter(String group, String name) { + if (!_counters.containsKey(group)) { + _counters.put(group, new ArrayList<>()); + } + Counter c = new Counter(name); + _counters.get(group).add(c); + return c; + } + + @Override + public Gauge newGauge(String group, String name, T value) { + if (!_gauges.containsKey(group)) { + _gauges.put(group, new ArrayList<>()); + } + + Gauge g = new Gauge(name, value); + _gauges.get(group).add(g); + return g; + } + + @Override + public Counter newCounter(String group, Counter counter) { + return null; + } + + @Override + public Gauge newGauge(String group, Gauge value) { + return null; + } + + @Override + public Timer newTimer(String group, String name) { + return null; + } + + @Override + public Timer newTimer(String group, Timer timer) { + return null; + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java new file mode 100644 index 0000000000..939dd5a64c --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java @@ -0,0 +1,65 @@ +package org.apache.samza.system.eventhub.admin; + +import com.microsoft.azure.eventhubs.PartitionReceiver; +import junit.framework.Assert; +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.apache.samza.system.eventhub.EventHubClientWrapper; +import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.EventHubSystemFactory; +import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; +import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; +import org.junit.Test; + +import java.util.HashMap; + +public class TestEventHubSystemAdmin { + + public static final String SYSTEM_NAME = "eventhub-s1"; + public static final String STREAM_NAME1 = "test_stream1"; + + public static final String EVENTHUB_NAMESPACE = ""; + public static final String EVENTHUB_ENTITY1 = ""; + public static final String EVENTHUB_KEY_NAME = ""; + public static final String EVENTHUB_KEY = ""; + + private Config createEventHubConfig(EventHubClientWrapper.PartitioningMethod partitioningMethod) { + HashMap mapConfig = new HashMap<>(); + mapConfig.put(EventHubSystemProducer.CONFIG_PARTITIONING_METHOD, partitioningMethod.toString()); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_NAMESPACE); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY_NAME); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME1), "earliest"); + return new MapConfig(mapConfig); + } + + @Test + public void testOffsetComparison() { + EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); + EventHubSystemAdmin eventHubSystemAdmin = (EventHubSystemAdmin) eventHubSystemFactory.getAdmin(SYSTEM_NAME, + createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + Assert.assertEquals(-1, eventHubSystemAdmin.offsetComparator("100", "200").intValue()); + Assert.assertEquals(0, eventHubSystemAdmin.offsetComparator("150", "150").intValue()); + Assert.assertEquals(1, eventHubSystemAdmin.offsetComparator("200", "100").intValue()); + Assert.assertNull(eventHubSystemAdmin.offsetComparator("1", "a")); + Assert.assertEquals(-1, eventHubSystemAdmin + .offsetComparator("100", EventHubSystemConsumer.END_OF_STREAM).intValue()); + Assert.assertEquals(0, eventHubSystemAdmin.offsetComparator(EventHubSystemConsumer.END_OF_STREAM, + EventHubSystemConsumer.END_OF_STREAM).intValue()); + Assert.assertEquals(1, eventHubSystemAdmin + .offsetComparator( EventHubSystemConsumer.END_OF_STREAM, "100").intValue()); + Assert.assertEquals(-1, eventHubSystemAdmin + .offsetComparator(PartitionReceiver.START_OF_STREAM, "10").intValue()); + } + + @Test + public void testGetNextOffset() { + EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); + EventHubSystemAdmin eventHubSystemAdmin = (EventHubSystemAdmin) eventHubSystemFactory.getAdmin(SYSTEM_NAME, + createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + + } + +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java new file mode 100644 index 0000000000..f25fb89512 --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java @@ -0,0 +1,49 @@ +package org.apache.samza.system.eventhub.consumer; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.PartitionReceiveHandler; +import org.apache.samza.system.eventhub.consumer.EventHubEntityConnectionFactory; + +import java.util.List; +import java.util.Map; + +class MockEventHubEntityConnectionFactory extends EventHubEntityConnectionFactory { + + @Override + EventHubEntityConnection createConnection(String namespace, String entityPath, String sasKeyName, String sasKey, + String consumerName) { + return new MockEventHubEntityConnection(entityPath, _eventData.get(entityPath)); + } + + private final Map>> _eventData; + + MockEventHubEntityConnectionFactory(Map>> eventData) { + _eventData = eventData; + } + + private class MockEventHubEntityConnection extends EventHubEntityConnection { + private final Map> _eventData; + + MockEventHubEntityConnection(String entity, Map> eventData) { + super(null, entity, null, null, null); + assert eventData != null : "No event data found for entity:" + entity; + _eventData = eventData; + } + + @Override + void connectAndStart() { + _offsets.keySet().forEach(partitionId -> { + List events = _eventData.get(partitionId); + PartitionReceiveHandler partitionHandler = _handlers.get(partitionId); + assert events != null : String.format("partition %d not found", partitionId); + assert partitionHandler != null : String.format("handler %d not registered", partitionId); + partitionHandler.onReceive(events); + }); + } + + @Override + void stop() { + // do nothing + } + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java new file mode 100644 index 0000000000..540622caca --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -0,0 +1,175 @@ +package org.apache.samza.system.eventhub.consumer; + + +import com.microsoft.azure.eventhubs.EventData; +import org.apache.samza.Partition; +import org.apache.samza.metrics.Counter; +import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.system.eventhub.EventDataWrapper; +import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.MockEventData; +import org.apache.samza.system.eventhub.TestMetricsRegistry; +import org.junit.Assert; +import org.junit.Test; + +import java.util.*; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class TestEventHubSystemConsumer { + private final String MOCK_ENTITY_1 = "mocktopic1"; + private final String MOCK_ENTITY_2 = "mocktopic2"; + + private void verifyEvents(List messages, List eventDataList) { + Assert.assertEquals(messages.size(), eventDataList.size()); + for (int i = 0; i < messages.size(); i++) { + IncomingMessageEnvelope message = messages.get(i); + EventData eventData = eventDataList.get(i); + Assert.assertEquals(message.getKey(), eventData.getSystemProperties().getPartitionKey()); + Assert.assertEquals(((EventDataWrapper) message.getMessage()).getDecryptedBody(), eventData.getBody()); + Assert.assertEquals(message.getOffset(), eventData.getSystemProperties().getOffset()); + } + } + + @Test + public void testSinglePartitionConsumptionHappyPath() throws Exception { + String systemName = "eventhubs"; + String streamName = "testStream"; + int numEvents = 10; // needs to be less than BLOCKING_QUEUE_SIZE + int partitionId = 0; + + TestMetricsRegistry testMetrics = new TestMetricsRegistry(); + Map>> eventData = new HashMap<>(); + Map> singleTopicEventData = new HashMap<>(); + + List singlePartitionEventData = MockEventData.generateEventData(numEvents); + singleTopicEventData.put(partitionId, singlePartitionEventData); + eventData.put(MOCK_ENTITY_1, singleTopicEventData); + EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); + + Map configMap = new HashMap<>(); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), streamName); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "latest"); + EventHubSystemConsumer consumer = + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); + + SystemStreamPartition ssp = new SystemStreamPartition(systemName, streamName, new Partition(partitionId)); + consumer.register(ssp, null); + consumer.start(); + List result = consumer.poll(Collections.singleton(ssp), 1000).get(ssp); + verifyEvents(result, singlePartitionEventData); + Assert.assertEquals(testMetrics.getCounters(streamName).size(), 3); + Assert.assertEquals(testMetrics.getGauges(streamName).size(), 2); + Map counters = + testMetrics.getCounters(streamName).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); + + Assert.assertEquals(counters.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents); + Assert.assertEquals(counters.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); + } + + @Test + public void testMultiPartitionConsumptionHappyPath() throws Exception { + String systemName = "eventhubs"; + String streamName = "testStream"; + int numEvents = 10; // needs to be less than BLOCKING_QUEUE_SIZE + int partitionId1 = 0; + int partitionId2 = 1; + TestMetricsRegistry testMetrics = new TestMetricsRegistry(); + + Map>> eventData = new HashMap<>(); + Map> singleTopicEventData = new HashMap<>(); + List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); + List singlePartitionEventData2 = MockEventData.generateEventData(numEvents); + singleTopicEventData.put(partitionId1, singlePartitionEventData1); + singleTopicEventData.put(partitionId2, singlePartitionEventData2); + eventData.put(MOCK_ENTITY_1, singleTopicEventData); + EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); + + Map configMap = new HashMap<>(); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), streamName); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); + EventHubSystemConsumer consumer = + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); + + SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId1)); + consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); + SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId2)); + consumer.register(ssp2, EventHubSystemConsumer.START_OF_STREAM); + consumer.start(); + Set ssps = new HashSet<>(); + ssps.add(ssp1); + ssps.add(ssp2); + Map> results = consumer.poll(ssps, 1000); + verifyEvents(results.get(ssp1), singlePartitionEventData1); + verifyEvents(results.get(ssp2), singlePartitionEventData2); + + Assert.assertEquals(testMetrics.getCounters(streamName).size(), 3); + Assert.assertEquals(testMetrics.getGauges(streamName).size(), 2); + Map counters = + testMetrics.getCounters(streamName).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); + + Assert.assertEquals(counters.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents * 2); + Assert.assertEquals(counters.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); + } + + @Test + public void testMultiStreamsConsumptionHappyPath() throws Exception { + String systemName = "eventhubs"; + String streamName1 = "testStream1"; + String streamName2 = "testStream2"; + int numEvents = 10; // needs to be less than BLOCKING_QUEUE_SIZE + int partitionId = 0; + TestMetricsRegistry testMetrics = new TestMetricsRegistry(); + + Map>> eventData = new HashMap<>(); + Map> singleTopicEventData1 = new HashMap<>(); + List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); + singleTopicEventData1.put(partitionId, singlePartitionEventData1); + eventData.put(MOCK_ENTITY_1, singleTopicEventData1); + Map> singleTopicEventData2 = new HashMap<>(); + List singlePartitionEventData2 = MockEventData.generateEventData(numEvents); + singleTopicEventData2.put(partitionId, singlePartitionEventData2); + eventData.put(MOCK_ENTITY_2, singleTopicEventData2); + EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); + + Map configMap = new HashMap<>(); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), + String.format("%s,%s", streamName1, streamName2)); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName1), MOCK_ENTITY_1); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName2), MOCK_ENTITY_2); + EventHubSystemConsumer consumer = + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); + + SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName1, new Partition(partitionId)); + consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); + SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName2, new Partition(partitionId)); + consumer.register(ssp2, EventHubSystemConsumer.START_OF_STREAM); + consumer.start(); + Set ssps = new HashSet<>(); + ssps.add(ssp1); + ssps.add(ssp2); + Map> results = consumer.poll(ssps, 1000); + verifyEvents(results.get(ssp1), singlePartitionEventData1); + verifyEvents(results.get(ssp2), singlePartitionEventData2); + + Assert.assertEquals(testMetrics.getCounters(streamName1).size(), 3); + Assert.assertEquals(testMetrics.getGauges(streamName1).size(), 2); + + Assert.assertEquals(testMetrics.getCounters(streamName2).size(), 3); + Assert.assertEquals(testMetrics.getGauges(streamName2).size(), 2); + + Map counters1 = + testMetrics.getCounters(streamName1).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); + + Assert.assertEquals(counters1.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents); + Assert.assertEquals(counters1.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); + + Map counters2 = + testMetrics.getCounters(streamName2).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); + + Assert.assertEquals(counters2.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents); + Assert.assertEquals(counters2.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java new file mode 100644 index 0000000000..fb5333aa1f --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -0,0 +1,180 @@ +package org.apache.samza.system.eventhub.producer; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.eventhubs.PartitionReceiver; +import com.microsoft.azure.servicebus.ServiceBusException; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemProducer; +import org.apache.samza.system.SystemStream; +import org.apache.samza.system.eventhub.EventHubClientWrapper; +import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.EventHubSystemFactory; +import org.apache.samza.util.NoOpMetricsRegistry; +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; + +public class TestEventHubSystemProducer { + private static final Logger LOG = LoggerFactory.getLogger(TestEventHubSystemProducer.class.getName()); + + public static final String EVENTHUB_NAMESPACE = ""; + public static final String EVENTHUB_ENTITY1 = ""; + public static final String EVENTHUB_KEY_NAME = ""; + public static final String EVENTHUB_KEY = ""; + + public static final String SYSTEM_NAME = "system1"; + public static final String STREAM_NAME1 = "test_stream1"; + public static final String STREAM_NAME2 = "test_stream2"; + + @Test + public void testSystemFactoryCreateAndStartProducer() { + Config eventHubConfig = createEventHubConfig(); + EventHubSystemFactory systemFactory = new EventHubSystemFactory(); + SystemProducer systemProducer = systemFactory.getProducer(SYSTEM_NAME, eventHubConfig, new NoOpMetricsRegistry()); + Assert.assertNotNull(systemProducer); + + systemProducer.register(STREAM_NAME1); + systemProducer.register(STREAM_NAME2); + systemProducer.start(); + systemProducer.stop(); + } + + private Config createEventHubConfig() { + return createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING); + } + + private Config createEventHubConfig(EventHubClientWrapper.PartitioningMethod partitioningMethod) { + HashMap mapConfig = new HashMap<>(); + mapConfig.put(EventHubSystemProducer.CONFIG_PARTITIONING_METHOD, partitioningMethod.toString()); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_NAMESPACE); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY_NAME); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME1), "earliest"); + + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME2); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_NAMESPACE); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_ENTITY1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY_NAME); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME2), "earliest"); + + return new MapConfig(mapConfig); + } + + @Test + public void testSend() { + Config eventHubConfig = createEventHubConfig(); + EventHubSystemFactory systemFactory = new EventHubSystemFactory(); + SystemProducer systemProducer = systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); + + systemProducer.register(STREAM_NAME1); + + try { + systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1)); + Assert.fail("Sending event before starting producer should throw exception"); + } catch (SamzaException e) { + } + + systemProducer.start(); + systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1)); + + try { + systemProducer.send(STREAM_NAME2, createMessageEnvelope(STREAM_NAME1)); + Assert.fail("Sending event to destination that is not registered should throw exception"); + } catch (SamzaException e) { + } + + try { + systemProducer.register(STREAM_NAME2); + Assert.fail("Trying to register after starting producer should throw exception"); + } catch (SamzaException e) { + } + + systemProducer.flush(STREAM_NAME1); + systemProducer.stop(); + } + + @Test + public void testReceive() throws ServiceBusException { + EventHubClientWrapper wrapper = + new EventHubClientWrapper(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING, 8, EVENTHUB_NAMESPACE, + EVENTHUB_ENTITY1, EVENTHUB_KEY_NAME, EVENTHUB_KEY); + EventHubClient client = wrapper.getEventHubClient(); + PartitionReceiver receiver = + client.createReceiverSync(EventHubClient.DEFAULT_CONSUMER_GROUP_NAME, "0", + "163456", true); + receiveMessages(receiver, 300); + } + + private void receiveMessages(PartitionReceiver receiver, int numMessages) throws ServiceBusException { + int count = 0; + while (count < numMessages) { + + Iterable messages = receiver.receiveSync(100); + if (messages == null) { + System.out.println("End of stream"); + break; + } + for (EventData data : messages) { + count++; + LOG.info("Data" + new String(data.getBody())); + System.out.println("\nDATA: " + new String(data.getBody())); + System.out.println("BYTES_SIZE: " + data.getBytes().length); + System.out.println("OFFSET: " + data.getSystemProperties().getOffset()); + } + } + } + + @Test + public void testSendToSpecificPartition() { + Config eventHubConfig = createEventHubConfig(EventHubClientWrapper.PartitioningMethod.PARTITION_KEY_AS_PARTITION); + EventHubSystemFactory systemFactory = new EventHubSystemFactory(); + SystemProducer systemProducer = systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); + + systemProducer.register(STREAM_NAME1); + systemProducer.start(); + for (int i = 0; i < 100; i++) { + systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1, 0)); + } + systemProducer.flush(STREAM_NAME1); + systemProducer.stop(); + } + + private OutgoingMessageEnvelope createMessageEnvelope(String streamName, int partition) { + return new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, streamName), partition, "key1".getBytes(), + "value0".getBytes()); + } + + private OutgoingMessageEnvelope createMessageEnvelope(String streamName) { + return new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, streamName), "key1".getBytes(), + "value".getBytes()); + } + + @Test + public void testFlush() { + Config eventHubConfig = createEventHubConfig(); + EventHubSystemFactory systemFactory = new EventHubSystemFactory(); + EventHubSystemProducer systemProducer = + (EventHubSystemProducer) systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); + systemProducer.register(STREAM_NAME1); + systemProducer.register(STREAM_NAME2); + systemProducer.start(); + int numEvents = 100; + for (int i = 0; i < numEvents; i++) { + systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1)); + systemProducer.send(STREAM_NAME2, createMessageEnvelope(STREAM_NAME2)); + } + systemProducer.flush(EVENTHUB_ENTITY1); + Assert.assertEquals(systemProducer.getPendingFutures().size(), 0); + systemProducer.stop(); + } +} From cc6eda038249db42cc1ddfc6396884854e7ad484 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 25 Sep 2017 11:20:23 -0700 Subject: [PATCH 03/34] fixed indents, reformated some code --- .../system/eventhub/EventDataWrapper.java | 40 +- .../eventhub/EventHubClientWrapper.java | 241 +++++----- .../samza/system/eventhub/EventHubConfig.java | 234 +++++----- .../eventhub/EventHubSystemFactory.java | 50 +- .../eventhub/admin/EventHubSystemAdmin.java | 159 ++++--- .../consumer/EventHubEntityConnection.java | 160 +++---- .../EventHubEntityConnectionFactory.java | 8 +- .../consumer/EventHubSystemConsumer.java | 356 +++++++-------- .../eventhub/metrics/SamzaHistogram.java | 51 +-- .../producer/EventHubSystemProducer.java | 429 +++++++++--------- .../samza/system/eventhub/MockEventData.java | 48 +- .../system/eventhub/TestMetricsRegistry.java | 84 ++-- .../admin/TestEventHubSystemAdmin.java | 83 ++-- .../MockEventHubEntityConnectionFactory.java | 65 ++- .../consumer/TestEventHubSystemConsumer.java | 306 ++++++------- .../producer/TestEventHubSystemProducer.java | 289 ++++++------ 16 files changed, 1294 insertions(+), 1309 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java index eb17c28e74..5fc95441d6 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java @@ -6,29 +6,29 @@ * Simpler wrapper of {@link EventData} events with the decrypted payload */ public class EventDataWrapper { - private final EventData _eventData; - private final byte[] _body; + private final EventData _eventData; + private final byte[] _body; - public EventDataWrapper(EventData eventData, byte[] body) { - _eventData = eventData; - _body = body; - } + public EventDataWrapper(EventData eventData, byte[] body) { + _eventData = eventData; + _body = body; + } - public EventData getEventData() { - return _eventData; - } + public EventData getEventData() { + return _eventData; + } - /** - * @return the body of decrypted body of the message. In case not encryption is setup for this topic - * just returns the body of the message. - */ - public byte[] getDecryptedBody() { - return _body; - } + /** + * @return the body of decrypted body of the message. In case not encryption is setup for this topic + * just returns the body of the message. + */ + public byte[] getDecryptedBody() { + return _body; + } - @Override - public String toString() { - return "EventDataWrapper: body: " + (new String(_body)) + ", EventData " + _eventData; - } + @Override + public String toString() { + return "EventDataWrapper: body: " + (new String(_body)) + ", EventData " + _eventData; + } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java index 555a442dfe..1f8cc87e6c 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java @@ -16,135 +16,136 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.*; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public class EventHubClientWrapper { - private static final Logger LOG = LoggerFactory.getLogger(EventHubClientWrapper.class.getName()); - - public enum PartitioningMethod { - EVENT_HUB_HASHING, - - PARTITION_KEY_AS_PARTITION, + private static final Logger LOG = LoggerFactory.getLogger(EventHubClientWrapper.class.getName()); + private final PartitioningMethod _partitioningMethod; + private final int _numPartitions; + private EventHubClient _eventHubClient; + private Map _partitionSenders = new HashMap<>(); + + public EventHubClientWrapper(PartitioningMethod partitioningMethod, int numPartitions, + String eventHubNamespace, String entityPath, String sasKeyName, String sasToken) { + String remoteHost = String.format("%s.servicebus.windows.net", eventHubNamespace); + _partitioningMethod = partitioningMethod; + _numPartitions = numPartitions; + try { + // Create a event hub connection string pointing to localhost + ConnectionStringBuilder connectionStringBuilder = + new ConnectionStringBuilder(eventHubNamespace, entityPath, sasKeyName, sasToken); + + _eventHubClient = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString()); + } catch (IOException ioe) { + throw new IllegalStateException( + "Failed to connect to remote host " + remoteHost + ":" + ClientConstants.AMQPS_PORT, ioe); + } catch (ServiceBusException e) { + String msg = String.format("Creation of event hub client failed for eventHub %s %s %s %s %s %s with exception", + entityPath, partitioningMethod, numPartitions, eventHubNamespace, sasKeyName, sasToken); + LOG.error(msg, e); + throw new SamzaException(msg, e); } - - private final PartitioningMethod _partitioningMethod; - private final int _numPartitions; - - public EventHubClient getEventHubClient() { - return _eventHubClient; + } + + public EventHubClient getEventHubClient() { + return _eventHubClient; + } + + public void closeSync() { + _partitionSenders.entrySet().forEach(x -> { + try { + x.getValue().closeSync(); + } catch (ServiceBusException e) { + LOG.warn("Closing the partition sender failed for partition " + x.getKey(), e); + } + }); + + try { + _eventHubClient.closeSync(); + } catch (ServiceBusException e) { + LOG.warn("Closing the event hub client failed ", e); } - - private EventHubClient _eventHubClient; - private Map _partitionSenders = new HashMap<>(); - - public EventHubClientWrapper(PartitioningMethod partitioningMethod, int numPartitions, - String eventHubNamespace, String entityPath, String sasKeyName, String sasToken) { - String remoteHost = String.format("%s.servicebus.windows.net", eventHubNamespace); - _partitioningMethod = partitioningMethod; - _numPartitions = numPartitions; - try { - // Create a event hub connection string pointing to localhost - ConnectionStringBuilder connectionStringBuilder = - new ConnectionStringBuilder(eventHubNamespace, entityPath, sasKeyName, sasToken); - - _eventHubClient = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString()); - } catch (IOException ioe) { - throw new IllegalStateException( - "Failed to connect to remote host " + remoteHost + ":" + ClientConstants.AMQPS_PORT, ioe); - } catch (ServiceBusException e) { - String msg = String.format("Creation of event hub client failed for eventHub %s %s %s %s %s %s with exception", entityPath, partitioningMethod, numPartitions, eventHubNamespace, sasKeyName, sasToken); - LOG.error(msg, e); - throw new SamzaException(msg, e); - } + } + + /** + * Timed connection close. + * + * @param timeoutMS Time in Miliseconds to wait for individual components (partition senders, event hub client and + * tunnel in order) to shutdown before moving to the next stage. + * For example a timeoutMS of 30000, will result in a wait of max 30secs for a successful close of + * all partition senders followed by a max 30secs wait for a successful close of eventhub client and + * then another max 30secs wait for successful tunnel close. Tunnel close failure/timeout will result + * in RuntimeException. + */ + + public void closeSync(long timeoutMS) { + List> futures = new ArrayList<>(); + _partitionSenders.entrySet().forEach(x -> futures.add(x.getValue().close())); + CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); + try { + future.get(timeoutMS, TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + LOG.warn("Closing the partition sender failed ", e); } - public void closeSync() { - _partitionSenders.entrySet().forEach(x -> { - try { - x.getValue().closeSync(); - } catch (ServiceBusException e) { - LOG.warn("Closing the partition sender failed for partition " + x.getKey(), e); - } - }); - - try { - _eventHubClient.closeSync(); - } catch (ServiceBusException e) { - LOG.warn("Closing the event hub client failed ", e); - } + future = _eventHubClient.close(); + try { + future.get(timeoutMS, TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + LOG.warn("Closing the event hub client failed ", e); } - - /** - * Timed connection close. - * @param timeoutMS Time in Miliseconds to wait for individual components (partition senders, event hub client and - * tunnel in order) to shutdown before moving to the next stage. - * For example a timeoutMS of 30000, will result in a wait of max 30secs for a successful close of - * all partition senders followed by a max 30secs wait for a successful close of eventhub client and - * then another max 30secs wait for successful tunnel close. Tunnel close failure/timeout will result - * in RuntimeException. - */ - - public void closeSync(long timeoutMS) { - List> futures = new ArrayList<>(); - _partitionSenders.entrySet().forEach(x -> futures.add(x.getValue().close())); - CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); - try { - future.get(timeoutMS, TimeUnit.MILLISECONDS); - } catch (ExecutionException | InterruptedException | TimeoutException e) { - LOG.warn("Closing the partition sender failed ", e); - } - - future = _eventHubClient.close(); - try { - future.get(timeoutMS, TimeUnit.MILLISECONDS); - } catch (ExecutionException | InterruptedException | TimeoutException e) { - LOG.warn("Closing the event hub client failed ", e); - } + } + + public CompletableFuture send(EventData eventData, Object partitionKey) { + if (_partitioningMethod == PartitioningMethod.EVENT_HUB_HASHING) { + return _eventHubClient.send(eventData, convertPartitionKeyToString(partitionKey)); + } else if (_partitioningMethod == PartitioningMethod.PARTITION_KEY_AS_PARTITION) { + if (!(partitionKey instanceof Integer)) { + String msg = "Partition key should be of type Integer"; + LOG.error(msg); + throw new SamzaException(msg); + } + + PartitionSender sender = getPartitionSender((int) partitionKey); + return sender.send(eventData); + } else { + throw new SamzaException("Unknown partitioning method " + _partitioningMethod); } - - - public CompletableFuture send(EventData eventData, Object partitionKey) { - if (_partitioningMethod == PartitioningMethod.EVENT_HUB_HASHING) { - return _eventHubClient.send(eventData, convertPartitionKeyToString(partitionKey)); - } else if (_partitioningMethod == PartitioningMethod.PARTITION_KEY_AS_PARTITION) { - if (!(partitionKey instanceof Integer)) { - String msg = "Partition key should be of type Integer"; - LOG.error(msg); - throw new SamzaException(msg); - } - - PartitionSender sender = getPartitionSender((int) partitionKey); - return sender.send(eventData); - } else { - throw new SamzaException("Unknown partitioning method " + _partitioningMethod); - } + } + + private String convertPartitionKeyToString(Object partitionKey) { + if (partitionKey instanceof String) { + return (String) partitionKey; + } else if (partitionKey instanceof Integer) { + return String.valueOf(partitionKey); + } else if (partitionKey instanceof byte[]) { + return new String((byte[]) partitionKey, Charset.defaultCharset()); + } else { + throw new SamzaException("Unsupported key type: " + partitionKey.getClass().toString()); } - - private String convertPartitionKeyToString(Object partitionKey) { - if (partitionKey instanceof String) { - return (String) partitionKey; - } else if (partitionKey instanceof Integer) { - return String.valueOf(partitionKey); - } else if (partitionKey instanceof byte[]) { - return new String((byte[]) partitionKey, Charset.defaultCharset()); - } else { - throw new SamzaException("Unsupported key type: " + partitionKey.getClass().toString()); - } + } + + private PartitionSender getPartitionSender(int partition) { + if (!_partitionSenders.containsKey(partition)) { + try { + PartitionSender partitionSender = + _eventHubClient.createPartitionSenderSync(String.valueOf(partition % _numPartitions)); + _partitionSenders.put(partition, partitionSender); + } catch (ServiceBusException e) { + String msg = "Creation of partition sender failed with exception"; + LOG.error(msg, e); + throw new SamzaException(msg, e); + } } - private PartitionSender getPartitionSender(int partition) { - if (!_partitionSenders.containsKey(partition)) { - try { - PartitionSender partitionSender = - _eventHubClient.createPartitionSenderSync(String.valueOf(partition % _numPartitions)); - _partitionSenders.put(partition, partitionSender); - } catch (ServiceBusException e) { - String msg = "Creation of partition sender failed with exception"; - LOG.error(msg, e); - throw new SamzaException(msg, e); - } - } - - return _partitionSenders.get(partition); - } + return _partitionSenders.get(partition); + } + + public enum PartitioningMethod { + EVENT_HUB_HASHING, + PARTITION_KEY_AS_PARTITION, + } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java index 6944a17292..40f93fa227 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -12,120 +12,126 @@ import java.util.Optional; public class EventHubConfig extends MapConfig { - public static final String CONFIG_STREAM_LIST = "systems.%s.stream.list"; - - public static final String CONFIG_STREAM_NAMESPACE = "systems.%s.streams.%s.eventhubs.namespace"; - - public static final String CONFIG_STREAM_ENTITYPATH = "systems.%s.streams.%s.eventhubs.entitypath"; - - public static final String CONFIG_STREAM_SAS_KEY_NAME = "systems.%s.streams.%s.eventhubs.sas.keyname"; - - public static final String CONFIG_STREAM_SAS_TOKEN = "systems.%s.streams.%s.eventhubs.sas.token"; - - public static final String CONFIG_STREAM_SERDE_FACTORY = "systems.%s.streams.%s.eventhubs.serdeFactory"; - public static final String CONFIG_STREAM_SERDE_PREFIX = "systems.%s.streams.%s.eventhubs.serde."; - - public static final String CONFIG_STREAM_CONSUMER_GROUP = "systems.%s.streams.%s.eventhubs.consumer.group"; - public static final String DEFAULT_CONFIG_STREAM_CONSUMER_GROUP = EventHubClient.DEFAULT_CONSUMER_GROUP_NAME; - - public static final String CONFIG_STREAM_CONSUMER_START_POSITION = "systems.%s.streams.%s.eventhubs.start.position"; - public static final String DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION = StartPosition.LATEST.name(); - - public enum StartPosition { - EARLIEST, - LATEST - } - - private final String _system; - - public EventHubConfig(Map config, String systemName) { - super(config); - _system = systemName; - } - - /** - * Get the list of streams that are defined. Each stream has enough - * information for connecting to a certain EventHub entity. - * @return list of stream names - */ - public List getStreamList() { - return getList(String.format(CONFIG_STREAM_LIST, _system)); - } - - /** - * Get the EventHubs namespace for the stream - * @param streamName name of stream - * @return EventHubs namespace - */ - public String getStreamNamespace(String streamName) { - return get(String.format(CONFIG_STREAM_NAMESPACE, _system, streamName)); - } - - /** - * Get the EventHubs entity path (topic name) for the stream - * @param streamName name of stream - * @return EventHubs entity path - */ - public String getStreamEntityPath(String streamName) { - return get(String.format(CONFIG_STREAM_ENTITYPATH, _system, streamName)); - } - - /** - * Get the EventHubs SAS (Shared Access Signature) key name for the stream - * @param streamName name of stream - * @return EventHubs SAS key name - */ - public String getStreamSasKeyName(String streamName) { - return get(String.format(CONFIG_STREAM_SAS_KEY_NAME, _system, streamName)); - } - - /** - * Get the EventHubs SAS (Shared Access Signature) token for the stream - * @param streamName name of stream - * @return EventHubs SAS token - */ - public String getStreamSasToken(String streamName) { - return get(String.format(CONFIG_STREAM_SAS_TOKEN, _system, streamName)); - } - - public Optional> getSerde(String streamName) { - Serde serde = null; - String serdeFactoryClassName = this.get(String.format(CONFIG_STREAM_SERDE_FACTORY, _system, streamName)); - if (!StringUtils.isEmpty(serdeFactoryClassName)) { - SerdeFactory factory = EventHubSystemFactory.getSerdeFactory(serdeFactoryClassName); - serde = factory.getSerde(streamName, this.subset(String.format(CONFIG_STREAM_SERDE_PREFIX, _system, streamName))); - } - return Optional.ofNullable(serde); - } - - /** - * Get the EventHubs consumer group used for consumption for the stream - * @param streamName name of stream - * @return EventHubs consumer group - */ - public String getStreamConsumerGroup(String streamName) { - return get(String.format(CONFIG_STREAM_CONSUMER_GROUP, _system, streamName), DEFAULT_CONFIG_STREAM_CONSUMER_GROUP); - } - - /** - * Get the start position when there is no checkpoints. By default the consumer starts from latest (end of stream) - * @param streamName name of the stream - * @return Starting position when no checkpoints - */ - public StartPosition getStartPosition(String streamName) { - String startPositionStr = get(String.format(CONFIG_STREAM_CONSUMER_START_POSITION, _system, streamName), - DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION); - return StartPosition.valueOf(startPositionStr.toUpperCase()); - } - - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), _system); - } - - @Override - public boolean equals(Object obj) { - return super.equals(obj) && _system.equals(((EventHubConfig) obj)._system); + public static final String CONFIG_STREAM_LIST = "systems.%s.stream.list"; + + public static final String CONFIG_STREAM_NAMESPACE = "systems.%s.streams.%s.eventhubs.namespace"; + + public static final String CONFIG_STREAM_ENTITYPATH = "systems.%s.streams.%s.eventhubs.entitypath"; + + public static final String CONFIG_STREAM_SAS_KEY_NAME = "systems.%s.streams.%s.eventhubs.sas.keyname"; + + public static final String CONFIG_STREAM_SAS_TOKEN = "systems.%s.streams.%s.eventhubs.sas.token"; + + public static final String CONFIG_STREAM_SERDE_FACTORY = "systems.%s.streams.%s.eventhubs.serdeFactory"; + public static final String CONFIG_STREAM_SERDE_PREFIX = "systems.%s.streams.%s.eventhubs.serde."; + + public static final String CONFIG_STREAM_CONSUMER_GROUP = "systems.%s.streams.%s.eventhubs.consumer.group"; + public static final String DEFAULT_CONFIG_STREAM_CONSUMER_GROUP = EventHubClient.DEFAULT_CONSUMER_GROUP_NAME; + + public static final String CONFIG_STREAM_CONSUMER_START_POSITION = "systems.%s.streams.%s.eventhubs.start.position"; + public static final String DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION = StartPosition.LATEST.name(); + private final String _system; + + public EventHubConfig(Map config, String systemName) { + super(config); + _system = systemName; + } + + /** + * Get the list of streams that are defined. Each stream has enough + * information for connecting to a certain EventHub entity. + * + * @return list of stream names + */ + public List getStreamList() { + return getList(String.format(CONFIG_STREAM_LIST, _system)); + } + + /** + * Get the EventHubs namespace for the stream + * + * @param streamName name of stream + * @return EventHubs namespace + */ + public String getStreamNamespace(String streamName) { + return get(String.format(CONFIG_STREAM_NAMESPACE, _system, streamName)); + } + + /** + * Get the EventHubs entity path (topic name) for the stream + * + * @param streamName name of stream + * @return EventHubs entity path + */ + public String getStreamEntityPath(String streamName) { + return get(String.format(CONFIG_STREAM_ENTITYPATH, _system, streamName)); + } + + /** + * Get the EventHubs SAS (Shared Access Signature) key name for the stream + * + * @param streamName name of stream + * @return EventHubs SAS key name + */ + public String getStreamSasKeyName(String streamName) { + return get(String.format(CONFIG_STREAM_SAS_KEY_NAME, _system, streamName)); + } + + /** + * Get the EventHubs SAS (Shared Access Signature) token for the stream + * + * @param streamName name of stream + * @return EventHubs SAS token + */ + public String getStreamSasToken(String streamName) { + return get(String.format(CONFIG_STREAM_SAS_TOKEN, _system, streamName)); + } + + public Optional> getSerde(String streamName) { + Serde serde = null; + String serdeFactoryClassName = this.get(String.format(CONFIG_STREAM_SERDE_FACTORY, _system, streamName)); + if (!StringUtils.isEmpty(serdeFactoryClassName)) { + SerdeFactory factory = EventHubSystemFactory.getSerdeFactory(serdeFactoryClassName); + serde = factory.getSerde(streamName, this.subset(String.format(CONFIG_STREAM_SERDE_PREFIX, _system, streamName))); } + return Optional.ofNullable(serde); + } + + /** + * Get the EventHubs consumer group used for consumption for the stream + * + * @param streamName name of stream + * @return EventHubs consumer group + */ + public String getStreamConsumerGroup(String streamName) { + return get(String.format(CONFIG_STREAM_CONSUMER_GROUP, _system, streamName), DEFAULT_CONFIG_STREAM_CONSUMER_GROUP); + } + + /** + * Get the start position when there is no checkpoints. By default the consumer starts from latest (end of stream) + * + * @param streamName name of the stream + * @return Starting position when no checkpoints + */ + public StartPosition getStartPosition(String streamName) { + String startPositionStr = get(String.format(CONFIG_STREAM_CONSUMER_START_POSITION, _system, streamName), + DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION); + return StartPosition.valueOf(startPositionStr.toUpperCase()); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), _system); + } + + @Override + public boolean equals(Object obj) { + return super.equals(obj) && _system.equals(((EventHubConfig) obj)._system); + } + + public enum StartPosition { + EARLIEST, + LATEST + } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java index 839ecf06fd..dd12916733 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java @@ -14,37 +14,35 @@ import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; import java.lang.reflect.Constructor; -import java.util.HashMap; -import java.util.Map; public class EventHubSystemFactory implements SystemFactory { - @Override - public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { - return new EventHubSystemConsumer(new EventHubConfig(config, systemName), - new EventHubEntityConnectionFactory(), registry); + @SuppressWarnings("unchecked") + public static SerdeFactory getSerdeFactory(String serdeFactoryClassName) { + SerdeFactory factory; + try { + Class> classObj = (Class>) Class.forName(serdeFactoryClassName); + Constructor> ctor = classObj.getDeclaredConstructor(); + factory = ctor.newInstance(); + } catch (Exception e) { + throw new SamzaException("Failed to create Serde Factory for: " + serdeFactoryClassName, e); } + return factory; + } - @Override - public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { - return new EventHubSystemProducer(systemName, config, registry); - } + @Override + public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { + return new EventHubSystemConsumer(new EventHubConfig(config, systemName), + new EventHubEntityConnectionFactory(), registry); + } - @Override - public SystemAdmin getAdmin(String systemName, Config config) { - return new EventHubSystemAdmin(systemName, new EventHubConfig(config, systemName)); - } + @Override + public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { + return new EventHubSystemProducer(systemName, config, registry); + } - @SuppressWarnings("unchecked") - public static SerdeFactory getSerdeFactory(String serdeFactoryClassName) { - SerdeFactory factory; - try { - Class> classObj = (Class>) Class.forName(serdeFactoryClassName); - Constructor> ctor = classObj.getDeclaredConstructor(); - factory = ctor.newInstance(); - } catch (Exception e) { - throw new SamzaException("Failed to create Serde Factory for: " + serdeFactoryClassName, e); - } - return factory; - } + @Override + public SystemAdmin getAdmin(String systemName, Config config) { + return new EventHubSystemAdmin(systemName, new EventHubConfig(config, systemName)); + } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index 350e602ac8..1323e429d9 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -1,7 +1,6 @@ package org.apache.samza.system.eventhub.admin; import com.microsoft.azure.eventhubs.EventHubRuntimeInformation; -import com.microsoft.azure.eventhubs.PartitionReceiver; import org.apache.samza.Partition; import org.apache.samza.SamzaException; import org.apache.samza.system.SystemAdmin; @@ -19,94 +18,94 @@ import java.util.concurrent.CompletableFuture; public class EventHubSystemAdmin implements SystemAdmin { - private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemAdmin.class); + private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemAdmin.class); - private String _systemName; - private EventHubConfig _config; - private Map _eventHubClients = new HashMap<>(); + private String _systemName; + private EventHubConfig _config; + private Map _eventHubClients = new HashMap<>(); - public EventHubSystemAdmin(String systemName, EventHubConfig config) { - _systemName = systemName; - _config = config; - } + public EventHubSystemAdmin(String systemName, EventHubConfig config) { + _systemName = systemName; + _config = config; + } - private static String getNextOffset(String currentOffset) { - // EventHub will return the first message AFTER the offset - // that was specified in the fetch request. - return currentOffset.equals(EventHubSystemConsumer.END_OF_STREAM) ? currentOffset : - String.valueOf(Long.parseLong(currentOffset) + 1); - } + private static String getNextOffset(String currentOffset) { + // EventHub will return the first message AFTER the offset + // that was specified in the fetch request. + return currentOffset.equals(EventHubSystemConsumer.END_OF_STREAM) ? currentOffset : + String.valueOf(Long.parseLong(currentOffset) + 1); + } - @Override - public Map getOffsetsAfter(Map offsets) { - Map results = new HashMap<>(); - offsets.forEach((partition, offset) -> results.put(partition, getNextOffset(offset))); - return results; - } + @Override + public Map getOffsetsAfter(Map offsets) { + Map results = new HashMap<>(); + offsets.forEach((partition, offset) -> results.put(partition, getNextOffset(offset))); + return results; + } - @Override - public Map getSystemStreamMetadata(Set streamNames) { - Map requestedMetadata = new HashMap<>(); - Map> ehRuntimeInfos = new HashMap<>(); - streamNames.forEach((streamName) -> { - if (!_eventHubClients.containsKey(streamName)) { - addEventHubClient(streamName); - } - ehRuntimeInfos.put(streamName, - _eventHubClients.get(streamName).getEventHubClient().getRuntimeInformation()); - }); - ehRuntimeInfos.forEach((streamName, ehRuntimeInfo) -> { - try { - EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(); // TODO: timeout - Map sspMetadataMap = new HashMap<>(); - for (String partition : ehInfo.getPartitionIds()) { //TODO getPartitionRuntimeInformation - sspMetadataMap.put(new Partition(Integer.parseInt(partition)), - new SystemStreamMetadata.SystemStreamPartitionMetadata(PartitionReceiver.START_OF_STREAM, - EventHubSystemConsumer.END_OF_STREAM, EventHubSystemConsumer.END_OF_STREAM)); - } - requestedMetadata.put(streamName, new SystemStreamMetadata(streamName, sspMetadataMap)); - } catch (Exception e){ - String msg = String.format("Error while fetching EventHubRuntimeInfo for System:%s, Stream:%s", - _systemName, streamName); - LOG.error(msg); - throw new SamzaException(msg); - } - }); - return requestedMetadata; - } + @Override + public Map getSystemStreamMetadata(Set streamNames) { + Map requestedMetadata = new HashMap<>(); + Map> ehRuntimeInfos = new HashMap<>(); + streamNames.forEach((streamName) -> { + if (!_eventHubClients.containsKey(streamName)) { + addEventHubClient(streamName); + } + ehRuntimeInfos.put(streamName, + _eventHubClients.get(streamName).getEventHubClient().getRuntimeInformation()); + }); + ehRuntimeInfos.forEach((streamName, ehRuntimeInfo) -> { + try { + EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(); // TODO: timeout + Map sspMetadataMap = new HashMap<>(); + for (String partition : ehInfo.getPartitionIds()) { //TODO getPartitionRuntimeInformation + sspMetadataMap.put(new Partition(Integer.parseInt(partition)), + new SystemStreamMetadata.SystemStreamPartitionMetadata(EventHubSystemConsumer.START_OF_STREAM, + EventHubSystemConsumer.END_OF_STREAM, EventHubSystemConsumer.END_OF_STREAM)); + } + requestedMetadata.put(streamName, new SystemStreamMetadata(streamName, sspMetadataMap)); + } catch (Exception e) { + String msg = String.format("Error while fetching EventHubRuntimeInfo for System:%s, Stream:%s", + _systemName, streamName); + LOG.error(msg); + throw new SamzaException(msg); + } + }); + return requestedMetadata; + } - private void addEventHubClient(String streamName) { - String ehNamespace = _config.getStreamNamespace(streamName); - String ehEntityPath = _config.getStreamEntityPath(streamName); - _eventHubClients.put(streamName, new EventHubClientWrapper(null, 0, - ehNamespace, ehEntityPath, _config.getStreamSasKeyName(streamName), _config.getStreamSasToken(streamName))); - } + private void addEventHubClient(String streamName) { + String ehNamespace = _config.getStreamNamespace(streamName); + String ehEntityPath = _config.getStreamEntityPath(streamName); + _eventHubClients.put(streamName, new EventHubClientWrapper(null, 0, + ehNamespace, ehEntityPath, _config.getStreamSasKeyName(streamName), _config.getStreamSasToken(streamName))); + } - @Override - public void createChangelogStream(String streamName, int numOfPartitions) { - throw new UnsupportedOperationException("Event Hubs does not support change log stream."); - } + @Override + public void createChangelogStream(String streamName, int numOfPartitions) { + throw new UnsupportedOperationException("Event Hubs does not support change log stream."); + } - @Override - public void validateChangelogStream(String streamName, int numOfPartitions) { - throw new UnsupportedOperationException("Event Hubs does not support change log stream."); - } + @Override + public void validateChangelogStream(String streamName, int numOfPartitions) { + throw new UnsupportedOperationException("Event Hubs does not support change log stream."); + } - @Override - public void createCoordinatorStream(String streamName) { - throw new UnsupportedOperationException("Event Hubs does not support coordinator stream."); - } + @Override + public void createCoordinatorStream(String streamName) { + throw new UnsupportedOperationException("Event Hubs does not support coordinator stream."); + } - @Override - public Integer offsetComparator(String offset1, String offset2) { - try { - if (offset1.equals(EventHubSystemConsumer.END_OF_STREAM)) { - return offset2.equals(EventHubSystemConsumer.END_OF_STREAM) ? 0 : 1; - } - return offset2.equals(EventHubSystemConsumer.END_OF_STREAM) ? -1 : - Long.compare(Long.parseLong(offset1), Long.parseLong(offset2)); - } catch (NumberFormatException exception) { - return null; - } + @Override + public Integer offsetComparator(String offset1, String offset2) { + try { + if (offset1.equals(EventHubSystemConsumer.END_OF_STREAM)) { + return offset2.equals(EventHubSystemConsumer.END_OF_STREAM) ? 0 : 1; + } + return offset2.equals(EventHubSystemConsumer.END_OF_STREAM) ? -1 : + Long.compare(Long.parseLong(offset1), Long.parseLong(offset2)); + } catch (NumberFormatException exception) { + return null; } + } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java index 409efac75b..6a4114a045 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java @@ -14,94 +14,94 @@ import java.util.TreeMap; public class EventHubEntityConnection { - private static final Logger LOG = LoggerFactory.getLogger(EventHubEntityConnection.class); + private static final Logger LOG = LoggerFactory.getLogger(EventHubEntityConnection.class); + final Map _offsets = new TreeMap<>(); + final Map _handlers = new TreeMap<>(); + private final String _namespace; + private final String _entityPath; + private final String _sasKeyName; + private final String _sasKey; + private final String _consumerName; + private final Map _receivers = new TreeMap<>(); + private EventHubClientWrapper _ehClientWrapper; + private boolean _isStarted = false; - private final String _namespace; - private final String _entityPath; - private final String _sasKeyName; - private final String _sasKey; - private final String _consumerName; - private final Map _receivers = new TreeMap<>(); - private EventHubClientWrapper _ehClientWrapper; - private boolean _isStarted = false; + EventHubEntityConnection(String namespace, String entityPath, String sasKeyName, String sasKey, String consumerName) { + _namespace = namespace; + _entityPath = entityPath; + _sasKeyName = sasKeyName; + _sasKey = sasKey; + _consumerName = consumerName; + } - final Map _offsets = new TreeMap<>(); - final Map _handlers = new TreeMap<>(); - - EventHubEntityConnection(String namespace, String entityPath, String sasKeyName, String sasKey, String consumerName) { - _namespace = namespace; - _entityPath = entityPath; - _sasKeyName = sasKeyName; - _sasKey = sasKey; - _consumerName = consumerName; + // add partitions and handlers for this connection. This can be called multiple times + // for multiple partitions, but needs to be called before connectAndStart() + synchronized void addPartition(int partitionId, String offset, PartitionReceiveHandler handler) { + if (_isStarted) { + LOG.warn("Trying to add partition when the connection has already started."); + return; } + _offsets.put(partitionId, offset); + _handlers.put(partitionId, handler); + } - // add partitions and handlers for this connection. This can be called multiple times - // for multiple partitions, but needs to be called before connectAndStart() - synchronized void addPartition(int partitionId, String offset, PartitionReceiveHandler handler) { - if (_isStarted) { - LOG.warn("Trying to add partition when the connection has already started."); - return; - } - _offsets.put(partitionId, offset); - _handlers.put(partitionId, handler); - } - - // establish the connection and start consuming events - synchronized void connectAndStart() { - _isStarted = true; + // establish the connection and start consuming events + synchronized void connectAndStart() { + _isStarted = true; + try { + LOG.info(String.format("Starting connection for namespace=%s, entity=%s ", _namespace, _entityPath)); + // upon the instantiation of the client, the connection will be established + _ehClientWrapper = + new EventHubClientWrapper(null, 0, + _namespace, _entityPath, _sasKeyName, _sasKey); + for (Map.Entry entry : _offsets.entrySet()) { + Integer id = entry.getKey(); + String offset = entry.getValue(); try { - LOG.info(String.format("Starting connection for namespace=%s, entity=%s ", _namespace, _entityPath)); - // upon the instantiation of the client, the connection will be established - _ehClientWrapper = - new EventHubClientWrapper(null, 0, _namespace, _entityPath, _sasKeyName, _sasKey); - for (Map.Entry entry : _offsets.entrySet()) { - Integer id = entry.getKey(); - String offset = entry.getValue(); - try { - PartitionReceiver receiver; - if (StringUtil.isNullOrWhiteSpace(offset)) { - throw new SamzaException( - String.format("Invalid offset %s namespace=%s, entity=%s", offset, _namespace, _entityPath)); - } - if (offset.equals(EventHubSystemConsumer.END_OF_STREAM)) { - receiver = _ehClientWrapper.getEventHubClient() - .createReceiverSync(_consumerName, id.toString(), Instant.now()); - } else { - receiver = _ehClientWrapper.getEventHubClient() - .createReceiverSync(_consumerName, id.toString(), offset, - !offset.equals(PartitionReceiver.START_OF_STREAM)); - } - receiver.setReceiveHandler(_handlers.get(id)); - _receivers.put(id, receiver); - } catch (Exception e) { - throw new SamzaException( - String.format("Failed to create receiver for EventHubs: namespace=%s, entity=%s, partitionId=%d", - _namespace, _entityPath, id), e); - } - } - } catch (Exception e) { + PartitionReceiver receiver; + if (StringUtil.isNullOrWhiteSpace(offset)) { throw new SamzaException( - String.format("Failed to create connection to EventHubs: namespace=%s, entity=%s", _namespace, _entityPath), - e); + String.format("Invalid offset %s namespace=%s, entity=%s", offset, _namespace, _entityPath)); + } + if (offset.equals(EventHubSystemConsumer.END_OF_STREAM)) { + receiver = _ehClientWrapper.getEventHubClient() + .createReceiverSync(_consumerName, id.toString(), Instant.now()); + } else { + receiver = _ehClientWrapper.getEventHubClient() + .createReceiverSync(_consumerName, id.toString(), offset, + !offset.equals(EventHubSystemConsumer.START_OF_STREAM)); + } + receiver.setReceiveHandler(_handlers.get(id)); + _receivers.put(id, receiver); + } catch (Exception e) { + throw new SamzaException( + String.format("Failed to create receiver for EventHubs: namespace=%s, entity=%s, partitionId=%d", + _namespace, _entityPath, id), e); } - LOG.info(String.format("Connection successfully started for namespace=%s, entity=%s ", _namespace, _entityPath)); + } + } catch (Exception e) { + throw new SamzaException( + String.format("Failed to create connection to EventHubs: namespace=%s, entity=%s", + _namespace, _entityPath), + e); } + LOG.info(String.format("Connection successfully started for namespace=%s, entity=%s ", _namespace, _entityPath)); + } - synchronized void stop() { - LOG.info(String.format("Stopping connection for namespace=%s, entity=%s ", _namespace, _entityPath)); - try { - for (PartitionReceiver receiver : _receivers.values()) { - receiver.closeSync(); - } - _ehClientWrapper.closeSync(); - } catch (ServiceBusException e) { - throw new SamzaException( - String.format("Failed to stop connection for namespace=%s, entity=%s ", _namespace, _entityPath), e); - } - _isStarted = false; - _offsets.clear(); - _handlers.clear(); - LOG.info(String.format("Connection for namespace=%s, entity=%s stopped", _namespace, _entityPath)); + synchronized void stop() { + LOG.info(String.format("Stopping connection for namespace=%s, entity=%s ", _namespace, _entityPath)); + try { + for (PartitionReceiver receiver : _receivers.values()) { + receiver.closeSync(); + } + _ehClientWrapper.closeSync(); + } catch (ServiceBusException e) { + throw new SamzaException( + String.format("Failed to stop connection for namespace=%s, entity=%s ", _namespace, _entityPath), e); } + _isStarted = false; + _offsets.clear(); + _handlers.clear(); + LOG.info(String.format("Connection for namespace=%s, entity=%s stopped", _namespace, _entityPath)); + } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java index e0dc41b10c..c3bf637c15 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java @@ -1,8 +1,8 @@ package org.apache.samza.system.eventhub.consumer; public class EventHubEntityConnectionFactory { - EventHubEntityConnection createConnection(String namespace, String entityPath, String sasKeyName, String sasKey, - String consumerName) { - return new EventHubEntityConnection(namespace, entityPath, sasKeyName, sasKey, consumerName); - } + EventHubEntityConnection createConnection(String namespace, String entityPath, String sasKeyName, String sasKey, + String consumerName) { + return new EventHubEntityConnection(namespace, entityPath, sasKeyName, sasKey, consumerName); + } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index 34b954448c..09741c16f9 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -12,7 +12,6 @@ import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.eventhub.EventDataWrapper; import org.apache.samza.system.eventhub.EventHubConfig; -import org.apache.samza.system.eventhub.admin.EventHubSystemAdmin; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; import org.apache.samza.util.BlockingEnvelopeMap; import org.slf4j.Logger; @@ -32,199 +31,192 @@ * partition, it registers a handler with the EventHubsClient which constantly * push data into a block queue. This class extends the BlockingEnvelopeMap * provided by samza-api to to simplify the logic around those blocking queues. - * + *

* A high level architecture: - * - * ┌───────────────────────────────────────────────┐ - * │ EventHubsClient │ - * │ │ - * │ ┌───────────────────────────────────────┐ │ ┌─────────────────────┐ - * │ │ │ │ │ │ - * │ │ PartitionReceiveHandler_1 │───┼───────▶│ SSP1-BlockingQueue ├──────┐ - * │ │ │ │ │ │ │ - * │ └───────────────────────────────────────┘ │ └─────────────────────┘ │ - * │ │ │ - * │ ┌───────────────────────────────────────┐ │ ┌─────────────────────┐ │ - * │ │ │ │ │ │ │ - * │ │ PartitionReceiveHandler_2 │───┼───────▶│ SSP2-BlockingQueue ├──────┤ ┌──────────────────────────┐ - * │ │ │ │ │ │ ├───────▶│ │ - * │ └───────────────────────────────────────┘ │ └─────────────────────┘ └───────▶│ SystemConsumer.poll() │ - * │ │ ┌───────▶│ │ - * │ │ │ └──────────────────────────┘ - * │ ... │ ... │ - * │ │ │ - * │ │ │ - * │ ┌───────────────────────────────────────┐ │ ┌─────────────────────┐ │ - * │ │ │ │ │ │ │ - * │ │ PartitionReceiveHandler_N │───┼───────▶│ SSPN-BlockingQueue ├──────┘ - * │ │ │ │ │ │ - * │ └───────────────────────────────────────┘ │ └─────────────────────┘ - * │ │ - * │ │ - * └───────────────────────────────────────────────┘ - * + *

+ * ┌───────────────────────────────────────────────┐ + * │ EventHubsClient │ + * │ │ + * │ ┌───────────────────────────────────────┐ │ ┌─────────────────────┐ + * │ │ │ │ │ │ + * │ │ PartitionReceiveHandler_1 │───┼───────▶│ SSP1-BlockingQueue ├──────┐ + * │ │ │ │ │ │ │ + * │ └───────────────────────────────────────┘ │ └─────────────────────┘ │ + * │ │ │ + * │ ┌───────────────────────────────────────┐ │ ┌─────────────────────┐ │ + * │ │ │ │ │ │ │ + * │ │ PartitionReceiveHandler_2 │───┼───────▶│ SSP2-BlockingQueue ├──────┤ ┌──────────────────────────┐ + * │ │ │ │ │ │ ├───────▶│ │ + * │ └───────────────────────────────────────┘ │ └─────────────────────┘ └───────▶│ SystemConsumer.poll() │ + * │ │ ┌───────▶│ │ + * │ │ │ └──────────────────────────┘ + * │ ... │ ... │ + * │ │ │ + * │ │ │ + * │ ┌───────────────────────────────────────┐ │ ┌─────────────────────┐ │ + * │ │ │ │ │ │ │ + * │ │ PartitionReceiveHandler_N │───┼───────▶│ SSPN-BlockingQueue ├──────┘ + * │ │ │ │ │ │ + * │ └───────────────────────────────────────┘ │ └─────────────────────┘ + * │ │ + * │ │ + * └───────────────────────────────────────────────┘ */ public class EventHubSystemConsumer extends BlockingEnvelopeMap { - private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemConsumer.class); - - public static final String START_OF_STREAM = PartitionReceiver.START_OF_STREAM; // -1 - public static final String END_OF_STREAM = "-2"; - private static final int MAX_EVENT_COUNT_PER_PARTITION_POLL = 50; - private static final int BLOCKING_QUEUE_SIZE = 100; - - private final Map _connections = new HashMap<>(); - private final Map> _serdes = new HashMap<>(); - private final EventHubConfig _config; - - private static Counter _aggEventReadRate = null; - private static Counter _aggEventByteReadRate = null; - private static SamzaHistogram _aggReadLatency = null; - private static Counter _aggReadErrors = null; - - private Map _eventReadRates; - private Map _eventByteReadRates; - private Map _readLatencies; - private Map _readErrors; - - public static final String AGGREGATE = "aggregate"; - - public static final String EVENT_READ_RATE = "eventReadRate"; - public static final String EVENT_BYTE_READ_RATE = "eventByteReadRate"; - public static final String READ_LATENCY = "readLatency"; - public static final String READ_ERRORS = "readErrors"; - - public EventHubSystemConsumer(EventHubConfig config, EventHubEntityConnectionFactory connectionFactory, - MetricsRegistry registry) { - super(registry, System::currentTimeMillis); - - _config = config; - List streamList = config.getStreamList(); - streamList.forEach(stream -> { - _connections.put(stream, connectionFactory.createConnection( - config.getStreamNamespace(stream), config.getStreamEntityPath(stream), - config.getStreamSasKeyName(stream), config.getStreamSasToken(stream), - config.getStreamConsumerGroup(stream))); - _serdes.put(stream, config.getSerde(stream).orElse(null)); - }); - _eventReadRates = streamList.stream() - .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_READ_RATE))); - _eventByteReadRates = streamList.stream() - .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_BYTE_READ_RATE))); - _readLatencies = streamList.stream() - .collect(Collectors.toMap(Function.identity(), x -> new SamzaHistogram(registry, x, READ_LATENCY))); - _readErrors = - streamList.stream().collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, READ_ERRORS))); - - // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. - synchronized (AGGREGATE) { - if (_aggEventReadRate == null) { - _aggEventReadRate = registry.newCounter(AGGREGATE, EVENT_READ_RATE); - _aggEventByteReadRate = registry.newCounter(AGGREGATE, EVENT_BYTE_READ_RATE); - _aggReadLatency = new SamzaHistogram(registry, AGGREGATE, READ_LATENCY); - _aggReadErrors = registry.newCounter(AGGREGATE, READ_ERRORS); - } - } + public static final String START_OF_STREAM = PartitionReceiver.START_OF_STREAM; // -1 + public static final String END_OF_STREAM = "-2"; + public static final String AGGREGATE = "aggregate"; + public static final String EVENT_READ_RATE = "eventReadRate"; + public static final String EVENT_BYTE_READ_RATE = "eventByteReadRate"; + public static final String READ_LATENCY = "readLatency"; + public static final String READ_ERRORS = "readErrors"; + private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemConsumer.class); + private static final int MAX_EVENT_COUNT_PER_PARTITION_POLL = 50; + private static final int BLOCKING_QUEUE_SIZE = 100; + private static Counter _aggEventReadRate = null; + private static Counter _aggEventByteReadRate = null; + private static SamzaHistogram _aggReadLatency = null; + private static Counter _aggReadErrors = null; + private final Map _connections = new HashMap<>(); + private final Map> _serdes = new HashMap<>(); + private final EventHubConfig _config; + private Map _eventReadRates; + private Map _eventByteReadRates; + private Map _readLatencies; + private Map _readErrors; + + public EventHubSystemConsumer(EventHubConfig config, EventHubEntityConnectionFactory connectionFactory, + MetricsRegistry registry) { + super(registry, System::currentTimeMillis); + + _config = config; + List streamList = config.getStreamList(); + streamList.forEach(stream -> { + _connections.put(stream, connectionFactory.createConnection( + config.getStreamNamespace(stream), config.getStreamEntityPath(stream), + config.getStreamSasKeyName(stream), config.getStreamSasToken(stream), + config.getStreamConsumerGroup(stream))); + _serdes.put(stream, config.getSerde(stream).orElse(null)); + }); + _eventReadRates = streamList.stream() + .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_READ_RATE))); + _eventByteReadRates = streamList.stream() + .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_BYTE_READ_RATE))); + _readLatencies = streamList.stream() + .collect(Collectors.toMap(Function.identity(), x -> new SamzaHistogram(registry, x, READ_LATENCY))); + _readErrors = + streamList.stream().collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, READ_ERRORS))); + + // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. + synchronized (AGGREGATE) { + if (_aggEventReadRate == null) { + _aggEventReadRate = registry.newCounter(AGGREGATE, EVENT_READ_RATE); + _aggEventByteReadRate = registry.newCounter(AGGREGATE, EVENT_BYTE_READ_RATE); + _aggReadLatency = new SamzaHistogram(registry, AGGREGATE, READ_LATENCY); + _aggReadErrors = registry.newCounter(AGGREGATE, READ_ERRORS); + } + } + } + + @Override + public void register(SystemStreamPartition systemStreamPartition, String offset) { + super.register(systemStreamPartition, offset); + String stream = systemStreamPartition.getStream(); + EventHubEntityConnection connection = _connections.get(stream); + if (connection == null) { + throw new SamzaException("No EventHub connection for " + stream); } - @Override - public void register(SystemStreamPartition systemStreamPartition, String offset) { - super.register(systemStreamPartition, offset); - String stream = systemStreamPartition.getStream(); - EventHubEntityConnection connection = _connections.get(stream); - if (connection == null) { - throw new SamzaException("No EventHub connection for " + stream); - } - - if (StringUtil.isNullOrWhiteSpace(offset)) { - switch (_config.getStartPosition(systemStreamPartition.getStream())) { - case EARLIEST: - offset = START_OF_STREAM; - break; - case LATEST: - offset = END_OF_STREAM; - break; - default: - throw new SamzaException( - "Unknown starting position config " + _config.getStartPosition(systemStreamPartition.getStream())); - } - } - connection.addPartition(systemStreamPartition.getPartition().getPartitionId(), offset, - new PartitionReceiverHandlerImpl(systemStreamPartition, _eventReadRates.get(stream), - _eventByteReadRates.get(stream), _readLatencies.get(stream), _readErrors.get(stream), - _serdes.get(stream))); + if (StringUtil.isNullOrWhiteSpace(offset)) { + switch (_config.getStartPosition(systemStreamPartition.getStream())) { + case EARLIEST: + offset = START_OF_STREAM; + break; + case LATEST: + offset = END_OF_STREAM; + break; + default: + throw new SamzaException( + "Unknown starting position config " + _config.getStartPosition(systemStreamPartition.getStream())); + } + } + connection.addPartition(systemStreamPartition.getPartition().getPartitionId(), offset, + new PartitionReceiverHandlerImpl(systemStreamPartition, _eventReadRates.get(stream), + _eventByteReadRates.get(stream), _readLatencies.get(stream), _readErrors.get(stream), + _serdes.get(stream))); + } + + @Override + public void start() { + _connections.values().forEach(EventHubEntityConnection::connectAndStart); + } + + @Override + public void stop() { + _connections.values().forEach(EventHubEntityConnection::stop); + } + + private class PartitionReceiverHandlerImpl extends PartitionReceiveHandler { + + private final Counter _eventReadRate; + private final Counter _eventByteReadRate; + private final SamzaHistogram _readLatency; + private final Counter _errors; + private final Serde _serde; + SystemStreamPartition _ssp; + + PartitionReceiverHandlerImpl(SystemStreamPartition ssp, Counter eventReadRate, Counter eventByteReadRate, + SamzaHistogram readLatency, Counter readErrors, Serde serde) { + super(MAX_EVENT_COUNT_PER_PARTITION_POLL); + _ssp = ssp; + _eventReadRate = eventReadRate; + _eventByteReadRate = eventByteReadRate; + _readLatency = readLatency; + _errors = readErrors; + _serde = serde; } @Override - public void start() { - _connections.values().forEach(EventHubEntityConnection::connectAndStart); + public void onReceive(Iterable events) { + if (events != null) { + + events.forEach(event -> { + byte[] decryptedBody = event.getBody(); + if (_serde != null) { + decryptedBody = _serde.fromBytes(decryptedBody); + } + EventDataWrapper wrappedEvent = new EventDataWrapper(event, decryptedBody); + try { + updateMetrics(event); + // note that the partition key can be null + put(_ssp, new IncomingMessageEnvelope(_ssp, event.getSystemProperties().getOffset(), + event.getSystemProperties().getPartitionKey(), wrappedEvent)); + } catch (Exception e) { + String msg = String.format("Exception while adding the event from ssp %s to dispatch queue.", _ssp); + LOG.error(msg, e); + throw new SamzaException(msg, e); + } + }); + } } - @Override - public void stop() { - _connections.values().forEach(EventHubEntityConnection::stop); + private void updateMetrics(EventData event) { + _eventReadRate.inc(); + _aggEventReadRate.inc(); + _eventByteReadRate.inc(event.getBodyLength()); + _aggEventByteReadRate.inc(event.getBodyLength()); + long latencyMs = Duration.between(Instant.now(), event.getSystemProperties().getEnqueuedTime()).toMillis(); + _readLatency.update(latencyMs); + _aggReadLatency.update(latencyMs); } - private class PartitionReceiverHandlerImpl extends PartitionReceiveHandler { - - private final Counter _eventReadRate; - private final Counter _eventByteReadRate; - private final SamzaHistogram _readLatency; - private final Counter _errors; - private final Serde _serde; - SystemStreamPartition _ssp; - - PartitionReceiverHandlerImpl(SystemStreamPartition ssp, Counter eventReadRate, Counter eventByteReadRate, - SamzaHistogram readLatency, Counter readErrors, Serde serde) { - super(MAX_EVENT_COUNT_PER_PARTITION_POLL); - _ssp = ssp; - _eventReadRate = eventReadRate; - _eventByteReadRate = eventByteReadRate; - _readLatency = readLatency; - _errors = readErrors; - _serde = serde; - } - - @Override - public void onReceive(Iterable events) { - if (events != null) { - - events.forEach(event -> { - byte[] decryptedBody = event.getBody(); - if (_serde != null) { - decryptedBody = _serde.fromBytes(decryptedBody); - } - EventDataWrapper wrappedEvent = new EventDataWrapper(event, decryptedBody); - try { - updateMetrics(event); - // note that the partition key can be null - put(_ssp, new IncomingMessageEnvelope(_ssp, event.getSystemProperties().getOffset(), - event.getSystemProperties().getPartitionKey(), wrappedEvent)); - } catch (Exception e) { - String msg = String.format("Exception while adding the event from ssp %s to dispatch queue.", _ssp); - LOG.error(msg, e); - throw new SamzaException(msg, e); - } - }); - } - } - - private void updateMetrics(EventData event) { - _eventReadRate.inc(); - _aggEventReadRate.inc(); - _eventByteReadRate.inc(event.getBodyLength()); - _aggEventByteReadRate.inc(event.getBodyLength()); - long latencyMs = Duration.between(Instant.now(), event.getSystemProperties().getEnqueuedTime()).toMillis(); - _readLatency.update(latencyMs); - _aggReadLatency.update(latencyMs); - } - - @Override - public void onError(Throwable throwable) { - // TODO error handling - _errors.inc(); - _aggReadErrors.inc(); - LOG.error(String.format("Received error from event hub connection (ssp=%s): ", _ssp), throwable); - } + @Override + public void onError(Throwable throwable) { + // TODO error handling + _errors.inc(); + _aggReadErrors.inc(); + LOG.error(String.format("Received error from event hub connection (ssp=%s): ", _ssp), throwable); } + } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java index c5b658ab3e..1f1bd8ce63 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java @@ -13,31 +13,30 @@ import java.util.stream.Collectors; - public class SamzaHistogram { - private final MetricsRegistry _registry; - private final Histogram _histogram; - private final List _percentiles; - private final Map> _gauges; - private static final List DEFAULT_HISTOGRAM_PERCENTILES = Arrays.asList(50D, 99D); - - public SamzaHistogram(MetricsRegistry registry, String group, String name) { - this(registry, group, name, DEFAULT_HISTOGRAM_PERCENTILES); - } - - public SamzaHistogram(MetricsRegistry registry, String group, String name, List percentiles) { - _registry = registry; - _histogram = new Histogram(new ExponentiallyDecayingReservoir()); - _percentiles = percentiles; - _gauges = _percentiles.stream() - .filter(x -> x > 0 && x <= 100) - .collect( - Collectors.toMap(Function.identity(), x -> _registry.newGauge(group, name + "_" + String.valueOf(0), 0D))); - } - - public void update(long value) { - _histogram.update(value); - Snapshot values = _histogram.getSnapshot(); - _percentiles.stream().forEach(x -> _gauges.get(x).set(values.getValue(x / 100))); - } + private static final List DEFAULT_HISTOGRAM_PERCENTILES = Arrays.asList(50D, 99D); + private final MetricsRegistry _registry; + private final Histogram _histogram; + private final List _percentiles; + private final Map> _gauges; + + public SamzaHistogram(MetricsRegistry registry, String group, String name) { + this(registry, group, name, DEFAULT_HISTOGRAM_PERCENTILES); + } + + public SamzaHistogram(MetricsRegistry registry, String group, String name, List percentiles) { + _registry = registry; + _histogram = new Histogram(new ExponentiallyDecayingReservoir()); + _percentiles = percentiles; + _gauges = _percentiles.stream() + .filter(x -> x > 0 && x <= 100) + .collect( + Collectors.toMap(Function.identity(), x -> _registry.newGauge(group, name + "_" + String.valueOf(0), 0D))); + } + + public void update(long value) { + _histogram.update(value); + Snapshot values = _histogram.getSnapshot(); + _percentiles.stream().forEach(x -> _gauges.get(x).set(values.getValue(x / 100))); + } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 10f1e12f49..85eea2f05f 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -24,243 +24,236 @@ import java.util.concurrent.ConcurrentHashMap; public class EventHubSystemProducer implements SystemProducer { - public static final String EVENT_SOURCE_TIMESTAMP = "event-source-timestamp"; - public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; - private final int _destinationPartitions; - private final boolean _sendKeyInEventProperties; - private final EventHubConfig _eventHubsConfig; - private final Config _config; - private Throwable _sendExceptionOnCallback; - private boolean _isStarted; - - private static Counter _aggEventWriteRate = null; - private static Counter _aggEventByteWriteRate = null; - private static SamzaHistogram _aggSendLatency = null; - private static SamzaHistogram _aggSendCallbackLatency = null; - private static Counter _aggSendErrors = null; - public static final String AGGREGATE = "aggregate"; - - private HashMap _eventWriteRate = new HashMap<>(); - private HashMap _eventByteWriteRate = new HashMap<>(); - private HashMap _sendLatency = new HashMap<>(); - private HashMap _sendCallbackLatency = new HashMap<>(); - private HashMap _sendErrors = new HashMap<>(); - - private static final String EVENT_WRITE_RATE = "eventWriteRate"; - private static final String EVENT_BYTE_WRITE_RATE = "eventByteWriteRate"; - private static final String SEND_ERRORS = "sendErrors"; - private static final String SEND_LATENCY = "sendLatency"; - private static final String SEND_CALLBACK_LATENCY = "sendCallbackLatency"; - private static final Duration SHUTDOWN_WAIT_TIME = Duration.ofMinutes(1L); - - public final static String CONFIG_PARTITIONING_METHOD = "partitioningMethod"; - public final static String DEFAULT_PARTITIONING_METHOD = EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING.toString(); - public static final String CONFIG_DESTINATION_NUM_PARTITION = "destinationPartitions"; - public static final String CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = "sendKeyInEventProperties"; - - private final EventHubClientWrapper.PartitioningMethod _partitioningMethod; - - private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemProducer.class.getName()); - - private final String _systemName; - private final MetricsRegistry _registry; - // Map of the system name to the event hub client. - private Map _eventHubClients = new HashMap<>(); - - private long _messageId; - private Map> _serdes = new HashMap<>(); - - private Map> _pendingFutures = new ConcurrentHashMap<>(); - - public EventHubSystemProducer(String systemName, Config config, MetricsRegistry registry) { - _messageId = 0; - _systemName = systemName; - _registry = registry; - _config = config; - _partitioningMethod = - EventHubClientWrapper.PartitioningMethod.valueOf(getConfigValue(config, CONFIG_PARTITIONING_METHOD, DEFAULT_PARTITIONING_METHOD)); - - _eventHubsConfig = new EventHubConfig(config, systemName); - _sendKeyInEventProperties = - Boolean.parseBoolean(getConfigValue(config, CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, "false")); - - // TODO this should be removed when we are able to find the number of partitions. - _destinationPartitions = Integer.parseInt(getConfigValue(config, CONFIG_DESTINATION_NUM_PARTITION, "-1")); + public static final String EVENT_SOURCE_TIMESTAMP = "event-source-timestamp"; + public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; + public static final String AGGREGATE = "aggregate"; + public final static String CONFIG_PARTITIONING_METHOD = "partitioningMethod"; + public final static String DEFAULT_PARTITIONING_METHOD = EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING.toString(); + public static final String CONFIG_DESTINATION_NUM_PARTITION = "destinationPartitions"; + public static final String CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = "sendKeyInEventProperties"; + private static final String EVENT_WRITE_RATE = "eventWriteRate"; + private static final String EVENT_BYTE_WRITE_RATE = "eventByteWriteRate"; + private static final String SEND_ERRORS = "sendErrors"; + private static final String SEND_LATENCY = "sendLatency"; + private static final String SEND_CALLBACK_LATENCY = "sendCallbackLatency"; + private static final Duration SHUTDOWN_WAIT_TIME = Duration.ofMinutes(1L); + private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemProducer.class.getName()); + private static Counter _aggEventWriteRate = null; + private static Counter _aggEventByteWriteRate = null; + private static SamzaHistogram _aggSendLatency = null; + private static SamzaHistogram _aggSendCallbackLatency = null; + private static Counter _aggSendErrors = null; + private final int _destinationPartitions; + private final boolean _sendKeyInEventProperties; + private final EventHubConfig _eventHubsConfig; + private final Config _config; + private final EventHubClientWrapper.PartitioningMethod _partitioningMethod; + private final String _systemName; + private final MetricsRegistry _registry; + private Throwable _sendExceptionOnCallback; + private boolean _isStarted; + private HashMap _eventWriteRate = new HashMap<>(); + private HashMap _eventByteWriteRate = new HashMap<>(); + private HashMap _sendLatency = new HashMap<>(); + private HashMap _sendCallbackLatency = new HashMap<>(); + private HashMap _sendErrors = new HashMap<>(); + // Map of the system name to the event hub client. + private Map _eventHubClients = new HashMap<>(); + + private long _messageId; + private Map> _serdes = new HashMap<>(); + + private Map> _pendingFutures = new ConcurrentHashMap<>(); + + public EventHubSystemProducer(String systemName, Config config, MetricsRegistry registry) { + _messageId = 0; + _systemName = systemName; + _registry = registry; + _config = config; + _partitioningMethod = + EventHubClientWrapper.PartitioningMethod.valueOf(getConfigValue(config, CONFIG_PARTITIONING_METHOD, DEFAULT_PARTITIONING_METHOD)); + + _eventHubsConfig = new EventHubConfig(config, systemName); + _sendKeyInEventProperties = + Boolean.parseBoolean(getConfigValue(config, CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, "false")); + + // TODO this should be removed when we are able to find the number of partitions. + _destinationPartitions = Integer.parseInt(getConfigValue(config, CONFIG_DESTINATION_NUM_PARTITION, "-1")); + } + + private String getConfigValue(Config config, String configKey, String defaultValue) { + String configValue = config.get(configKey, defaultValue); + + if (configValue == null) { + throw new SamzaException(configKey + " is not configured."); } - private String getConfigValue(Config config, String configKey, String defaultValue) { - String configValue = config.get(configKey, defaultValue); + return configValue; + } + + @Override + public synchronized void start() { + LOG.info("Starting system producer."); + for (String eventHub : _eventHubClients.keySet()) { + _eventWriteRate.put(eventHub, _registry.newCounter(eventHub, EVENT_WRITE_RATE)); + _eventByteWriteRate.put(eventHub, _registry.newCounter(eventHub, EVENT_BYTE_WRITE_RATE)); + _sendLatency.put(eventHub, new SamzaHistogram(_registry, eventHub, SEND_LATENCY)); + _sendCallbackLatency.put(eventHub, new SamzaHistogram(_registry, eventHub, SEND_CALLBACK_LATENCY)); + _sendErrors.put(eventHub, _registry.newCounter(eventHub, SEND_ERRORS)); + } + + // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. + synchronized (AGGREGATE) { + if (_aggEventWriteRate == null) { + _aggEventWriteRate = _registry.newCounter(AGGREGATE, EVENT_WRITE_RATE); + _aggEventByteWriteRate = _registry.newCounter(AGGREGATE, EVENT_BYTE_WRITE_RATE); + _aggSendLatency = new SamzaHistogram(_registry, AGGREGATE, SEND_LATENCY); + _aggSendCallbackLatency = new SamzaHistogram(_registry, AGGREGATE, SEND_CALLBACK_LATENCY); + _aggSendErrors = _registry.newCounter(AGGREGATE, SEND_ERRORS); + } + } + + _isStarted = true; + } + + @Override + public synchronized void stop() { + LOG.info("Stopping system producer."); + _eventHubClients.values().forEach(ehClient -> ehClient.closeSync(SHUTDOWN_WAIT_TIME.toMillis())); + _eventHubClients.clear(); + } + + @Override + public synchronized void register(String streamName) { + LOG.info("Trying to register {}.", streamName); + if (_isStarted) { + String msg = "Cannot register once the producer is started."; + LOG.error(msg); + throw new SamzaException(msg); + } + + String ehNamespace = _eventHubsConfig.getStreamNamespace(streamName); + String ehName = _eventHubsConfig.getStreamEntityPath(streamName); - if (configValue == null) { - throw new SamzaException(configKey + " is not configured."); - } + EventHubClientWrapper ehClient = + new EventHubClientWrapper(_partitioningMethod, _destinationPartitions, ehNamespace, ehName, + _eventHubsConfig.getStreamSasKeyName(streamName), _eventHubsConfig.getStreamSasToken(streamName)); - return configValue; + _eventHubClients.put(streamName, ehClient); + _eventHubsConfig.getSerde(streamName).ifPresent(x -> _serdes.put(streamName, x)); + } + + @Override + public synchronized void send(String destination, OutgoingMessageEnvelope envelope) { + if (!_isStarted) { + throw new SamzaException("Trying to call send before the producer is started."); } - @Override - public synchronized void start() { - LOG.info("Starting system producer."); - for (String eventHub : _eventHubClients.keySet()) { - _eventWriteRate.put(eventHub, _registry.newCounter(eventHub, EVENT_WRITE_RATE)); - _eventByteWriteRate.put(eventHub, _registry.newCounter(eventHub, EVENT_BYTE_WRITE_RATE)); - _sendLatency.put(eventHub, new SamzaHistogram(_registry, eventHub, SEND_LATENCY)); - _sendCallbackLatency.put(eventHub, new SamzaHistogram(_registry, eventHub, SEND_CALLBACK_LATENCY)); - _sendErrors.put(eventHub, _registry.newCounter(eventHub, SEND_ERRORS)); - } - - // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. - synchronized (AGGREGATE) { - if (_aggEventWriteRate == null) { - _aggEventWriteRate = _registry.newCounter(AGGREGATE, EVENT_WRITE_RATE); - _aggEventByteWriteRate = _registry.newCounter(AGGREGATE, EVENT_BYTE_WRITE_RATE); - _aggSendLatency = new SamzaHistogram(_registry, AGGREGATE, SEND_LATENCY); - _aggSendCallbackLatency = new SamzaHistogram(_registry, AGGREGATE, SEND_CALLBACK_LATENCY); - _aggSendErrors = _registry.newCounter(AGGREGATE, SEND_ERRORS); - } - } - - _isStarted = true; + if (!_eventHubClients.containsKey(destination)) { + String msg = String.format("Trying to send event to a destination {%s} that is not registered.", destination); + LOG.error(msg); + throw new SamzaException(msg); } - @Override - public synchronized void stop() { - LOG.info("Stopping system producer."); - _eventHubClients.values().forEach(ehClient -> ehClient.closeSync(SHUTDOWN_WAIT_TIME.toMillis())); - _eventHubClients.clear(); + if (_sendExceptionOnCallback != null) { + SamzaException e = new SamzaException(_sendExceptionOnCallback); + _sendExceptionOnCallback = null; + _pendingFutures.clear(); + LOG.error("One of the previous sends failed."); + throw e; } - @Override - public synchronized void register(String streamName) { - LOG.info("Trying to register {}.", streamName); - if (_isStarted) { - String msg = "Cannot register once the producer is started."; - LOG.error(msg); - throw new SamzaException(msg); - } + EventData eventData = createEventData(destination, envelope); - String ehNamespace = _eventHubsConfig.getStreamNamespace(streamName); - String ehName = _eventHubsConfig.getStreamEntityPath(streamName); + _eventWriteRate.get(destination).inc(); + _aggEventWriteRate.inc(); + _eventByteWriteRate.get(destination).inc(eventData.getBodyLength()); + _aggEventByteWriteRate.inc(eventData.getBodyLength()); + EventHubClientWrapper ehClient = _eventHubClients.get(destination); - EventHubClientWrapper ehClient = - new EventHubClientWrapper(_partitioningMethod, _destinationPartitions, ehNamespace, ehName, - _eventHubsConfig.getStreamSasKeyName(streamName), _eventHubsConfig.getStreamSasToken(streamName)); + Instant startTime = Instant.now(); - _eventHubClients.put(streamName, ehClient); - _eventHubsConfig.getSerde(streamName).ifPresent(x -> _serdes.put(streamName, x)); - } + CompletableFuture sendResult; + sendResult = ehClient.send(eventData, envelope.getPartitionKey()); - @Override - public synchronized void send(String destination, OutgoingMessageEnvelope envelope) { - if (!_isStarted) { - throw new SamzaException("Trying to call send before the producer is started."); - } - - if (!_eventHubClients.containsKey(destination)) { - String msg = String.format("Trying to send event to a destination {%s} that is not registered.", destination); - LOG.error(msg); - throw new SamzaException(msg); - } - - if (_sendExceptionOnCallback != null) { - SamzaException e = new SamzaException(_sendExceptionOnCallback); - _sendExceptionOnCallback = null; - _pendingFutures.clear(); - LOG.error("One of the previous sends failed."); - throw e; - } - - EventData eventData = createEventData(destination, envelope); - - _eventWriteRate.get(destination).inc(); - _aggEventWriteRate.inc(); - _eventByteWriteRate.get(destination).inc(eventData.getBodyLength()); - _aggEventByteWriteRate.inc(eventData.getBodyLength()); - EventHubClientWrapper ehClient = _eventHubClients.get(destination); - - Instant startTime = Instant.now(); - - CompletableFuture sendResult; - sendResult = ehClient.send(eventData, envelope.getPartitionKey()); - - Instant endTime = Instant.now(); - long latencyMs = Duration.between(startTime, endTime).toMillis(); - _sendLatency.get(destination).update(latencyMs); - _aggSendLatency.update(latencyMs); - - long messageId = ++_messageId; - - // Rotate the messageIds - if (messageId == Long.MAX_VALUE) { - _messageId = 0; - } - - _pendingFutures.put(messageId, sendResult); - - // Auto remove the future from the list when they are complete. - sendResult.handle(((aVoid, throwable) -> { - long callbackLatencyMs = Duration.between(endTime, Instant.now()).toMillis(); - _sendCallbackLatency.get(destination).update(callbackLatencyMs); - _aggSendCallbackLatency.update(callbackLatencyMs); - if (throwable != null) { - _sendErrors.get(destination).inc(); - _aggSendErrors.inc(); - LOG.error("Send message to event hub: {} failed with exception: ", destination, throwable); - _sendExceptionOnCallback = throwable; - } - _pendingFutures.remove(messageId); - return aVoid; - })); + Instant endTime = Instant.now(); + long latencyMs = Duration.between(startTime, endTime).toMillis(); + _sendLatency.get(destination).update(latencyMs); + _aggSendLatency.update(latencyMs); + + long messageId = ++_messageId; + + // Rotate the messageIds + if (messageId == Long.MAX_VALUE) { + _messageId = 0; } - private EventData createEventData(String streamName, OutgoingMessageEnvelope envelope) { - Optional> serde = Optional.ofNullable(_serdes.getOrDefault(streamName, null)); - byte[] eventValue = (byte[]) envelope.getMessage(); - if (serde.isPresent()) { - eventValue = serde.get().toBytes(eventValue); - } - - EventData eventData = new EventData(eventValue); - - eventData.getProperties().put(PRODUCE_TIMESTAMP, Long.toString(System.currentTimeMillis())); - - if (_sendKeyInEventProperties) { - String keyValue = ""; - if (envelope.getKey() != null) { - keyValue = (envelope.getKey() instanceof byte[]) ? new String((byte[]) envelope.getKey()) - : envelope.getKey().toString(); - } - eventData.getProperties().put("key", keyValue); - } - return eventData; + _pendingFutures.put(messageId, sendResult); + + // Auto remove the future from the list when they are complete. + sendResult.handle(((aVoid, throwable) -> { + long callbackLatencyMs = Duration.between(endTime, Instant.now()).toMillis(); + _sendCallbackLatency.get(destination).update(callbackLatencyMs); + _aggSendCallbackLatency.update(callbackLatencyMs); + if (throwable != null) { + _sendErrors.get(destination).inc(); + _aggSendErrors.inc(); + LOG.error("Send message to event hub: {} failed with exception: ", destination, throwable); + _sendExceptionOnCallback = throwable; + } + _pendingFutures.remove(messageId); + return aVoid; + })); + } + + private EventData createEventData(String streamName, OutgoingMessageEnvelope envelope) { + Optional> serde = Optional.ofNullable(_serdes.getOrDefault(streamName, null)); + byte[] eventValue = (byte[]) envelope.getMessage(); + if (serde.isPresent()) { + eventValue = serde.get().toBytes(eventValue); } - @Override - public void flush(String source) { - LOG.info("Trying to flush pending {} sends messages: {}", _pendingFutures.size(), _pendingFutures.keySet()); - // Wait till all the pending sends are complete. - while (!_pendingFutures.isEmpty()) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - String msg = "Flush failed with error"; - LOG.error(msg, e); - throw new SamzaException(msg, e); - } - } - - if (_sendExceptionOnCallback != null) { - String msg = "Sending one of the message failed during flush"; - Throwable throwable = _sendExceptionOnCallback; - _sendExceptionOnCallback = null; - LOG.error(msg, throwable); - throw new SamzaException(msg, throwable); - } - - LOG.info("Flush succeeded."); + EventData eventData = new EventData(eventValue); + + eventData.getProperties().put(PRODUCE_TIMESTAMP, Long.toString(System.currentTimeMillis())); + + if (_sendKeyInEventProperties) { + String keyValue = ""; + if (envelope.getKey() != null) { + keyValue = (envelope.getKey() instanceof byte[]) ? new String((byte[]) envelope.getKey()) + : envelope.getKey().toString(); + } + eventData.getProperties().put("key", keyValue); + } + return eventData; + } + + @Override + public void flush(String source) { + LOG.info("Trying to flush pending {} sends messages: {}", _pendingFutures.size(), _pendingFutures.keySet()); + // Wait till all the pending sends are complete. + while (!_pendingFutures.isEmpty()) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + String msg = "Flush failed with error"; + LOG.error(msg, e); + throw new SamzaException(msg, e); + } } - Collection> getPendingFutures() { - return _pendingFutures.values(); + if (_sendExceptionOnCallback != null) { + String msg = "Sending one of the message failed during flush"; + Throwable throwable = _sendExceptionOnCallback; + _sendExceptionOnCallback = null; + LOG.error(msg, throwable); + throw new SamzaException(msg, throwable); } + + LOG.info("Flush succeeded."); + } + + Collection> getPendingFutures() { + return _pendingFutures.values(); + } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java index 711f85358a..8b5b4e7db0 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java @@ -7,32 +7,32 @@ public class MockEventData extends EventData { - EventData.SystemProperties _overridedSystemProperties; + EventData.SystemProperties _overridedSystemProperties; - public MockEventData(byte[] data, String partitionKey, String offset) { - super(data); - HashMap properties = new HashMap<>(); - properties.put("x-opt-offset", offset); - properties.put("x-opt-partition-key", partitionKey); - properties.put("x-opt-enqueued-time", new Date(System.currentTimeMillis())); - _overridedSystemProperties = new SystemProperties(properties); - } + public MockEventData(byte[] data, String partitionKey, String offset) { + super(data); + HashMap properties = new HashMap<>(); + properties.put("x-opt-offset", offset); + properties.put("x-opt-partition-key", partitionKey); + properties.put("x-opt-enqueued-time", new Date(System.currentTimeMillis())); + _overridedSystemProperties = new SystemProperties(properties); + } - @Override - public EventData.SystemProperties getSystemProperties() { - return _overridedSystemProperties; + public static List generateEventData(int numEvents) { + Random rand = new Random(System.currentTimeMillis()); + List result = new ArrayList<>(); + for (int i = 0; i < numEvents; i++) { + String key = "key_" + rand.nextInt(); + String message = "message:" + rand.nextInt(); + String offset = "offset_" + i; + EventData eventData = new MockEventData(message.getBytes(Charset.defaultCharset()), key, offset); + result.add(eventData); } + return result; + } - public static List generateEventData(int numEvents) { - Random rand = new Random(System.currentTimeMillis()); - List result = new ArrayList<>(); - for (int i = 0; i < numEvents; i++) { - String key = "key_" + rand.nextInt(); - String message = "message:" + rand.nextInt(); - String offset = "offset_" + i; - EventData eventData = new MockEventData(message.getBytes(Charset.defaultCharset()), key, offset); - result.add(eventData); - } - return result; - } + @Override + public EventData.SystemProperties getSystemProperties() { + return _overridedSystemProperties; + } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java index 4245307d5e..5523398de4 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java @@ -10,57 +10,57 @@ import java.util.List; import java.util.Map; -public class TestMetricsRegistry implements MetricsRegistry{ +public class TestMetricsRegistry implements MetricsRegistry { - private Map> _counters = new HashedMap<>(); - private Map>> _gauges = new HashedMap<>(); + private Map> _counters = new HashedMap<>(); + private Map>> _gauges = new HashedMap<>(); - public List getCounters(String groupName) { - return _counters.get(groupName); - } + public List getCounters(String groupName) { + return _counters.get(groupName); + } - public List> getGauges(String groupName) { - return _gauges.get(groupName); - } + public List> getGauges(String groupName) { + return _gauges.get(groupName); + } - @Override - public Counter newCounter(String group, String name) { - if (!_counters.containsKey(group)) { - _counters.put(group, new ArrayList<>()); - } - Counter c = new Counter(name); - _counters.get(group).add(c); - return c; + @Override + public Counter newCounter(String group, String name) { + if (!_counters.containsKey(group)) { + _counters.put(group, new ArrayList<>()); } + Counter c = new Counter(name); + _counters.get(group).add(c); + return c; + } - @Override - public Gauge newGauge(String group, String name, T value) { - if (!_gauges.containsKey(group)) { - _gauges.put(group, new ArrayList<>()); - } - - Gauge g = new Gauge(name, value); - _gauges.get(group).add(g); - return g; + @Override + public Gauge newGauge(String group, String name, T value) { + if (!_gauges.containsKey(group)) { + _gauges.put(group, new ArrayList<>()); } - @Override - public Counter newCounter(String group, Counter counter) { - return null; - } + Gauge g = new Gauge(name, value); + _gauges.get(group).add(g); + return g; + } - @Override - public Gauge newGauge(String group, Gauge value) { - return null; - } + @Override + public Counter newCounter(String group, Counter counter) { + return null; + } - @Override - public Timer newTimer(String group, String name) { - return null; - } + @Override + public Gauge newGauge(String group, Gauge value) { + return null; + } - @Override - public Timer newTimer(String group, Timer timer) { - return null; - } + @Override + public Timer newTimer(String group, String name) { + return null; + } + + @Override + public Timer newTimer(String group, Timer timer) { + return null; + } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java index 939dd5a64c..dd59078142 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java @@ -1,6 +1,5 @@ package org.apache.samza.system.eventhub.admin; -import com.microsoft.azure.eventhubs.PartitionReceiver; import junit.framework.Assert; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; @@ -15,51 +14,51 @@ public class TestEventHubSystemAdmin { - public static final String SYSTEM_NAME = "eventhub-s1"; - public static final String STREAM_NAME1 = "test_stream1"; + public static final String SYSTEM_NAME = "eventhub-s1"; + public static final String STREAM_NAME1 = "test_stream1"; - public static final String EVENTHUB_NAMESPACE = ""; - public static final String EVENTHUB_ENTITY1 = ""; - public static final String EVENTHUB_KEY_NAME = ""; - public static final String EVENTHUB_KEY = ""; + public static final String EVENTHUB_NAMESPACE = ""; + public static final String EVENTHUB_ENTITY1 = ""; + public static final String EVENTHUB_KEY_NAME = ""; + public static final String EVENTHUB_KEY = ""; - private Config createEventHubConfig(EventHubClientWrapper.PartitioningMethod partitioningMethod) { - HashMap mapConfig = new HashMap<>(); - mapConfig.put(EventHubSystemProducer.CONFIG_PARTITIONING_METHOD, partitioningMethod.toString()); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_NAMESPACE); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY_NAME); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME1), "earliest"); - return new MapConfig(mapConfig); - } + private Config createEventHubConfig(EventHubClientWrapper.PartitioningMethod partitioningMethod) { + HashMap mapConfig = new HashMap<>(); + mapConfig.put(EventHubSystemProducer.CONFIG_PARTITIONING_METHOD, partitioningMethod.toString()); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_NAMESPACE); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY_NAME); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME1), "earliest"); + return new MapConfig(mapConfig); + } - @Test - public void testOffsetComparison() { - EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); - EventHubSystemAdmin eventHubSystemAdmin = (EventHubSystemAdmin) eventHubSystemFactory.getAdmin(SYSTEM_NAME, - createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); - Assert.assertEquals(-1, eventHubSystemAdmin.offsetComparator("100", "200").intValue()); - Assert.assertEquals(0, eventHubSystemAdmin.offsetComparator("150", "150").intValue()); - Assert.assertEquals(1, eventHubSystemAdmin.offsetComparator("200", "100").intValue()); - Assert.assertNull(eventHubSystemAdmin.offsetComparator("1", "a")); - Assert.assertEquals(-1, eventHubSystemAdmin - .offsetComparator("100", EventHubSystemConsumer.END_OF_STREAM).intValue()); - Assert.assertEquals(0, eventHubSystemAdmin.offsetComparator(EventHubSystemConsumer.END_OF_STREAM, - EventHubSystemConsumer.END_OF_STREAM).intValue()); - Assert.assertEquals(1, eventHubSystemAdmin - .offsetComparator( EventHubSystemConsumer.END_OF_STREAM, "100").intValue()); - Assert.assertEquals(-1, eventHubSystemAdmin - .offsetComparator(PartitionReceiver.START_OF_STREAM, "10").intValue()); - } + @Test + public void testOffsetComparison() { + EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); + EventHubSystemAdmin eventHubSystemAdmin = (EventHubSystemAdmin) eventHubSystemFactory.getAdmin(SYSTEM_NAME, + createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + Assert.assertEquals(-1, eventHubSystemAdmin.offsetComparator("100", "200").intValue()); + Assert.assertEquals(0, eventHubSystemAdmin.offsetComparator("150", "150").intValue()); + Assert.assertEquals(1, eventHubSystemAdmin.offsetComparator("200", "100").intValue()); + Assert.assertNull(eventHubSystemAdmin.offsetComparator("1", "a")); + Assert.assertEquals(-1, eventHubSystemAdmin + .offsetComparator("100", EventHubSystemConsumer.END_OF_STREAM).intValue()); + Assert.assertEquals(0, eventHubSystemAdmin.offsetComparator(EventHubSystemConsumer.END_OF_STREAM, + EventHubSystemConsumer.END_OF_STREAM).intValue()); + Assert.assertEquals(1, eventHubSystemAdmin + .offsetComparator(EventHubSystemConsumer.END_OF_STREAM, "100").intValue()); + Assert.assertEquals(-1, eventHubSystemAdmin + .offsetComparator(EventHubSystemConsumer.START_OF_STREAM, "10").intValue()); + } - @Test - public void testGetNextOffset() { - EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); - EventHubSystemAdmin eventHubSystemAdmin = (EventHubSystemAdmin) eventHubSystemFactory.getAdmin(SYSTEM_NAME, - createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + @Test + public void testGetNextOffset() { + EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); + EventHubSystemAdmin eventHubSystemAdmin = (EventHubSystemAdmin) eventHubSystemFactory.getAdmin(SYSTEM_NAME, + createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); - } + } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java index f25fb89512..de84c15041 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java @@ -2,48 +2,47 @@ import com.microsoft.azure.eventhubs.EventData; import com.microsoft.azure.eventhubs.PartitionReceiveHandler; -import org.apache.samza.system.eventhub.consumer.EventHubEntityConnectionFactory; import java.util.List; import java.util.Map; class MockEventHubEntityConnectionFactory extends EventHubEntityConnectionFactory { - @Override - EventHubEntityConnection createConnection(String namespace, String entityPath, String sasKeyName, String sasKey, - String consumerName) { - return new MockEventHubEntityConnection(entityPath, _eventData.get(entityPath)); - } + private final Map>> _eventData; + + MockEventHubEntityConnectionFactory(Map>> eventData) { + _eventData = eventData; + } + + @Override + EventHubEntityConnection createConnection(String namespace, String entityPath, String sasKeyName, String sasKey, + String consumerName) { + return new MockEventHubEntityConnection(entityPath, _eventData.get(entityPath)); + } - private final Map>> _eventData; + private class MockEventHubEntityConnection extends EventHubEntityConnection { + private final Map> _eventData; - MockEventHubEntityConnectionFactory(Map>> eventData) { - _eventData = eventData; + MockEventHubEntityConnection(String entity, Map> eventData) { + super(null, entity, null, null, null); + assert eventData != null : "No event data found for entity:" + entity; + _eventData = eventData; } - private class MockEventHubEntityConnection extends EventHubEntityConnection { - private final Map> _eventData; - - MockEventHubEntityConnection(String entity, Map> eventData) { - super(null, entity, null, null, null); - assert eventData != null : "No event data found for entity:" + entity; - _eventData = eventData; - } - - @Override - void connectAndStart() { - _offsets.keySet().forEach(partitionId -> { - List events = _eventData.get(partitionId); - PartitionReceiveHandler partitionHandler = _handlers.get(partitionId); - assert events != null : String.format("partition %d not found", partitionId); - assert partitionHandler != null : String.format("handler %d not registered", partitionId); - partitionHandler.onReceive(events); - }); - } - - @Override - void stop() { - // do nothing - } + @Override + void connectAndStart() { + _offsets.keySet().forEach(partitionId -> { + List events = _eventData.get(partitionId); + PartitionReceiveHandler partitionHandler = _handlers.get(partitionId); + assert events != null : String.format("partition %d not found", partitionId); + assert partitionHandler != null : String.format("handler %d not registered", partitionId); + partitionHandler.onReceive(events); + }); + } + + @Override + void stop() { + // do nothing } + } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index 540622caca..7ffbb19232 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -18,158 +18,158 @@ import java.util.stream.Collectors; public class TestEventHubSystemConsumer { - private final String MOCK_ENTITY_1 = "mocktopic1"; - private final String MOCK_ENTITY_2 = "mocktopic2"; - - private void verifyEvents(List messages, List eventDataList) { - Assert.assertEquals(messages.size(), eventDataList.size()); - for (int i = 0; i < messages.size(); i++) { - IncomingMessageEnvelope message = messages.get(i); - EventData eventData = eventDataList.get(i); - Assert.assertEquals(message.getKey(), eventData.getSystemProperties().getPartitionKey()); - Assert.assertEquals(((EventDataWrapper) message.getMessage()).getDecryptedBody(), eventData.getBody()); - Assert.assertEquals(message.getOffset(), eventData.getSystemProperties().getOffset()); - } - } - - @Test - public void testSinglePartitionConsumptionHappyPath() throws Exception { - String systemName = "eventhubs"; - String streamName = "testStream"; - int numEvents = 10; // needs to be less than BLOCKING_QUEUE_SIZE - int partitionId = 0; - - TestMetricsRegistry testMetrics = new TestMetricsRegistry(); - Map>> eventData = new HashMap<>(); - Map> singleTopicEventData = new HashMap<>(); - - List singlePartitionEventData = MockEventData.generateEventData(numEvents); - singleTopicEventData.put(partitionId, singlePartitionEventData); - eventData.put(MOCK_ENTITY_1, singleTopicEventData); - EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); - - Map configMap = new HashMap<>(); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), streamName); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "latest"); - EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); - - SystemStreamPartition ssp = new SystemStreamPartition(systemName, streamName, new Partition(partitionId)); - consumer.register(ssp, null); - consumer.start(); - List result = consumer.poll(Collections.singleton(ssp), 1000).get(ssp); - verifyEvents(result, singlePartitionEventData); - Assert.assertEquals(testMetrics.getCounters(streamName).size(), 3); - Assert.assertEquals(testMetrics.getGauges(streamName).size(), 2); - Map counters = - testMetrics.getCounters(streamName).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); - - Assert.assertEquals(counters.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents); - Assert.assertEquals(counters.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); - } - - @Test - public void testMultiPartitionConsumptionHappyPath() throws Exception { - String systemName = "eventhubs"; - String streamName = "testStream"; - int numEvents = 10; // needs to be less than BLOCKING_QUEUE_SIZE - int partitionId1 = 0; - int partitionId2 = 1; - TestMetricsRegistry testMetrics = new TestMetricsRegistry(); - - Map>> eventData = new HashMap<>(); - Map> singleTopicEventData = new HashMap<>(); - List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); - List singlePartitionEventData2 = MockEventData.generateEventData(numEvents); - singleTopicEventData.put(partitionId1, singlePartitionEventData1); - singleTopicEventData.put(partitionId2, singlePartitionEventData2); - eventData.put(MOCK_ENTITY_1, singleTopicEventData); - EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); - - Map configMap = new HashMap<>(); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), streamName); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); - EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); - - SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId1)); - consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); - SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId2)); - consumer.register(ssp2, EventHubSystemConsumer.START_OF_STREAM); - consumer.start(); - Set ssps = new HashSet<>(); - ssps.add(ssp1); - ssps.add(ssp2); - Map> results = consumer.poll(ssps, 1000); - verifyEvents(results.get(ssp1), singlePartitionEventData1); - verifyEvents(results.get(ssp2), singlePartitionEventData2); - - Assert.assertEquals(testMetrics.getCounters(streamName).size(), 3); - Assert.assertEquals(testMetrics.getGauges(streamName).size(), 2); - Map counters = - testMetrics.getCounters(streamName).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); - - Assert.assertEquals(counters.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents * 2); - Assert.assertEquals(counters.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); - } - - @Test - public void testMultiStreamsConsumptionHappyPath() throws Exception { - String systemName = "eventhubs"; - String streamName1 = "testStream1"; - String streamName2 = "testStream2"; - int numEvents = 10; // needs to be less than BLOCKING_QUEUE_SIZE - int partitionId = 0; - TestMetricsRegistry testMetrics = new TestMetricsRegistry(); - - Map>> eventData = new HashMap<>(); - Map> singleTopicEventData1 = new HashMap<>(); - List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); - singleTopicEventData1.put(partitionId, singlePartitionEventData1); - eventData.put(MOCK_ENTITY_1, singleTopicEventData1); - Map> singleTopicEventData2 = new HashMap<>(); - List singlePartitionEventData2 = MockEventData.generateEventData(numEvents); - singleTopicEventData2.put(partitionId, singlePartitionEventData2); - eventData.put(MOCK_ENTITY_2, singleTopicEventData2); - EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); - - Map configMap = new HashMap<>(); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), - String.format("%s,%s", streamName1, streamName2)); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName1), MOCK_ENTITY_1); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName2), MOCK_ENTITY_2); - EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); - - SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName1, new Partition(partitionId)); - consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); - SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName2, new Partition(partitionId)); - consumer.register(ssp2, EventHubSystemConsumer.START_OF_STREAM); - consumer.start(); - Set ssps = new HashSet<>(); - ssps.add(ssp1); - ssps.add(ssp2); - Map> results = consumer.poll(ssps, 1000); - verifyEvents(results.get(ssp1), singlePartitionEventData1); - verifyEvents(results.get(ssp2), singlePartitionEventData2); - - Assert.assertEquals(testMetrics.getCounters(streamName1).size(), 3); - Assert.assertEquals(testMetrics.getGauges(streamName1).size(), 2); - - Assert.assertEquals(testMetrics.getCounters(streamName2).size(), 3); - Assert.assertEquals(testMetrics.getGauges(streamName2).size(), 2); - - Map counters1 = - testMetrics.getCounters(streamName1).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); - - Assert.assertEquals(counters1.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents); - Assert.assertEquals(counters1.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); - - Map counters2 = - testMetrics.getCounters(streamName2).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); - - Assert.assertEquals(counters2.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents); - Assert.assertEquals(counters2.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); + private final String MOCK_ENTITY_1 = "mocktopic1"; + private final String MOCK_ENTITY_2 = "mocktopic2"; + + private void verifyEvents(List messages, List eventDataList) { + Assert.assertEquals(messages.size(), eventDataList.size()); + for (int i = 0; i < messages.size(); i++) { + IncomingMessageEnvelope message = messages.get(i); + EventData eventData = eventDataList.get(i); + Assert.assertEquals(message.getKey(), eventData.getSystemProperties().getPartitionKey()); + Assert.assertEquals(((EventDataWrapper) message.getMessage()).getDecryptedBody(), eventData.getBody()); + Assert.assertEquals(message.getOffset(), eventData.getSystemProperties().getOffset()); } + } + + @Test + public void testSinglePartitionConsumptionHappyPath() throws Exception { + String systemName = "eventhubs"; + String streamName = "testStream"; + int numEvents = 10; // needs to be less than BLOCKING_QUEUE_SIZE + int partitionId = 0; + + TestMetricsRegistry testMetrics = new TestMetricsRegistry(); + Map>> eventData = new HashMap<>(); + Map> singleTopicEventData = new HashMap<>(); + + List singlePartitionEventData = MockEventData.generateEventData(numEvents); + singleTopicEventData.put(partitionId, singlePartitionEventData); + eventData.put(MOCK_ENTITY_1, singleTopicEventData); + EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); + + Map configMap = new HashMap<>(); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), streamName); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "latest"); + EventHubSystemConsumer consumer = + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); + + SystemStreamPartition ssp = new SystemStreamPartition(systemName, streamName, new Partition(partitionId)); + consumer.register(ssp, null); + consumer.start(); + List result = consumer.poll(Collections.singleton(ssp), 1000).get(ssp); + verifyEvents(result, singlePartitionEventData); + Assert.assertEquals(testMetrics.getCounters(streamName).size(), 3); + Assert.assertEquals(testMetrics.getGauges(streamName).size(), 2); + Map counters = + testMetrics.getCounters(streamName).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); + + Assert.assertEquals(counters.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents); + Assert.assertEquals(counters.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); + } + + @Test + public void testMultiPartitionConsumptionHappyPath() throws Exception { + String systemName = "eventhubs"; + String streamName = "testStream"; + int numEvents = 10; // needs to be less than BLOCKING_QUEUE_SIZE + int partitionId1 = 0; + int partitionId2 = 1; + TestMetricsRegistry testMetrics = new TestMetricsRegistry(); + + Map>> eventData = new HashMap<>(); + Map> singleTopicEventData = new HashMap<>(); + List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); + List singlePartitionEventData2 = MockEventData.generateEventData(numEvents); + singleTopicEventData.put(partitionId1, singlePartitionEventData1); + singleTopicEventData.put(partitionId2, singlePartitionEventData2); + eventData.put(MOCK_ENTITY_1, singleTopicEventData); + EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); + + Map configMap = new HashMap<>(); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), streamName); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); + EventHubSystemConsumer consumer = + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); + + SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId1)); + consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); + SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId2)); + consumer.register(ssp2, EventHubSystemConsumer.START_OF_STREAM); + consumer.start(); + Set ssps = new HashSet<>(); + ssps.add(ssp1); + ssps.add(ssp2); + Map> results = consumer.poll(ssps, 1000); + verifyEvents(results.get(ssp1), singlePartitionEventData1); + verifyEvents(results.get(ssp2), singlePartitionEventData2); + + Assert.assertEquals(testMetrics.getCounters(streamName).size(), 3); + Assert.assertEquals(testMetrics.getGauges(streamName).size(), 2); + Map counters = + testMetrics.getCounters(streamName).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); + + Assert.assertEquals(counters.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents * 2); + Assert.assertEquals(counters.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); + } + + @Test + public void testMultiStreamsConsumptionHappyPath() throws Exception { + String systemName = "eventhubs"; + String streamName1 = "testStream1"; + String streamName2 = "testStream2"; + int numEvents = 10; // needs to be less than BLOCKING_QUEUE_SIZE + int partitionId = 0; + TestMetricsRegistry testMetrics = new TestMetricsRegistry(); + + Map>> eventData = new HashMap<>(); + Map> singleTopicEventData1 = new HashMap<>(); + List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); + singleTopicEventData1.put(partitionId, singlePartitionEventData1); + eventData.put(MOCK_ENTITY_1, singleTopicEventData1); + Map> singleTopicEventData2 = new HashMap<>(); + List singlePartitionEventData2 = MockEventData.generateEventData(numEvents); + singleTopicEventData2.put(partitionId, singlePartitionEventData2); + eventData.put(MOCK_ENTITY_2, singleTopicEventData2); + EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); + + Map configMap = new HashMap<>(); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), + String.format("%s,%s", streamName1, streamName2)); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName1), MOCK_ENTITY_1); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName2), MOCK_ENTITY_2); + EventHubSystemConsumer consumer = + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); + + SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName1, new Partition(partitionId)); + consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); + SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName2, new Partition(partitionId)); + consumer.register(ssp2, EventHubSystemConsumer.START_OF_STREAM); + consumer.start(); + Set ssps = new HashSet<>(); + ssps.add(ssp1); + ssps.add(ssp2); + Map> results = consumer.poll(ssps, 1000); + verifyEvents(results.get(ssp1), singlePartitionEventData1); + verifyEvents(results.get(ssp2), singlePartitionEventData2); + + Assert.assertEquals(testMetrics.getCounters(streamName1).size(), 3); + Assert.assertEquals(testMetrics.getGauges(streamName1).size(), 2); + + Assert.assertEquals(testMetrics.getCounters(streamName2).size(), 3); + Assert.assertEquals(testMetrics.getGauges(streamName2).size(), 2); + + Map counters1 = + testMetrics.getCounters(streamName1).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); + + Assert.assertEquals(counters1.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents); + Assert.assertEquals(counters1.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); + + Map counters2 = + testMetrics.getCounters(streamName2).stream().collect(Collectors.toMap(Counter::getName, Function.identity())); + + Assert.assertEquals(counters2.get(EventHubSystemConsumer.EVENT_READ_RATE).getCount(), numEvents); + Assert.assertEquals(counters2.get(EventHubSystemConsumer.READ_ERRORS).getCount(), 0); + } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java index fb5333aa1f..ee5f375cdb 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -22,159 +22,158 @@ import java.util.HashMap; public class TestEventHubSystemProducer { - private static final Logger LOG = LoggerFactory.getLogger(TestEventHubSystemProducer.class.getName()); - - public static final String EVENTHUB_NAMESPACE = ""; - public static final String EVENTHUB_ENTITY1 = ""; - public static final String EVENTHUB_KEY_NAME = ""; - public static final String EVENTHUB_KEY = ""; - - public static final String SYSTEM_NAME = "system1"; - public static final String STREAM_NAME1 = "test_stream1"; - public static final String STREAM_NAME2 = "test_stream2"; - - @Test - public void testSystemFactoryCreateAndStartProducer() { - Config eventHubConfig = createEventHubConfig(); - EventHubSystemFactory systemFactory = new EventHubSystemFactory(); - SystemProducer systemProducer = systemFactory.getProducer(SYSTEM_NAME, eventHubConfig, new NoOpMetricsRegistry()); - Assert.assertNotNull(systemProducer); - - systemProducer.register(STREAM_NAME1); - systemProducer.register(STREAM_NAME2); - systemProducer.start(); - systemProducer.stop(); + public static final String EVENTHUB_NAMESPACE = "" + + ""; + public static final String EVENTHUB_ENTITY1 = ""; + public static final String EVENTHUB_KEY_NAME = ""; + public static final String EVENTHUB_KEY = ""; + public static final String SYSTEM_NAME = "system1"; + public static final String STREAM_NAME1 = "test_stream1"; + public static final String STREAM_NAME2 = "test_stream2"; + private static final Logger LOG = LoggerFactory.getLogger(TestEventHubSystemProducer.class.getName()); + + @Test + public void testSystemFactoryCreateAndStartProducer() { + Config eventHubConfig = createEventHubConfig(); + EventHubSystemFactory systemFactory = new EventHubSystemFactory(); + SystemProducer systemProducer = systemFactory.getProducer(SYSTEM_NAME, eventHubConfig, new NoOpMetricsRegistry()); + Assert.assertNotNull(systemProducer); + + systemProducer.register(STREAM_NAME1); + systemProducer.register(STREAM_NAME2); + systemProducer.start(); + systemProducer.stop(); + } + + private Config createEventHubConfig() { + return createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING); + } + + private Config createEventHubConfig(EventHubClientWrapper.PartitioningMethod partitioningMethod) { + HashMap mapConfig = new HashMap<>(); + mapConfig.put(EventHubSystemProducer.CONFIG_PARTITIONING_METHOD, partitioningMethod.toString()); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_NAMESPACE); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY_NAME); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME1), "earliest"); + + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME2); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_NAMESPACE); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_ENTITY1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY_NAME); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME2), "earliest"); + + return new MapConfig(mapConfig); + } + + @Test + public void testSend() { + Config eventHubConfig = createEventHubConfig(); + EventHubSystemFactory systemFactory = new EventHubSystemFactory(); + SystemProducer systemProducer = systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); + + systemProducer.register(STREAM_NAME1); + + try { + systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1)); + Assert.fail("Sending event before starting producer should throw exception"); + } catch (SamzaException e) { } - private Config createEventHubConfig() { - return createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING); - } + systemProducer.start(); + systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1)); - private Config createEventHubConfig(EventHubClientWrapper.PartitioningMethod partitioningMethod) { - HashMap mapConfig = new HashMap<>(); - mapConfig.put(EventHubSystemProducer.CONFIG_PARTITIONING_METHOD, partitioningMethod.toString()); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_NAMESPACE); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY_NAME); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME1), "earliest"); - - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME2); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_NAMESPACE); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_ENTITY1); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY_NAME); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME2), "earliest"); - - return new MapConfig(mapConfig); + try { + systemProducer.send(STREAM_NAME2, createMessageEnvelope(STREAM_NAME1)); + Assert.fail("Sending event to destination that is not registered should throw exception"); + } catch (SamzaException e) { } - @Test - public void testSend() { - Config eventHubConfig = createEventHubConfig(); - EventHubSystemFactory systemFactory = new EventHubSystemFactory(); - SystemProducer systemProducer = systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); - - systemProducer.register(STREAM_NAME1); - - try { - systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1)); - Assert.fail("Sending event before starting producer should throw exception"); - } catch (SamzaException e) { - } - - systemProducer.start(); - systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1)); - - try { - systemProducer.send(STREAM_NAME2, createMessageEnvelope(STREAM_NAME1)); - Assert.fail("Sending event to destination that is not registered should throw exception"); - } catch (SamzaException e) { - } - - try { - systemProducer.register(STREAM_NAME2); - Assert.fail("Trying to register after starting producer should throw exception"); - } catch (SamzaException e) { - } - - systemProducer.flush(STREAM_NAME1); - systemProducer.stop(); + try { + systemProducer.register(STREAM_NAME2); + Assert.fail("Trying to register after starting producer should throw exception"); + } catch (SamzaException e) { } - @Test - public void testReceive() throws ServiceBusException { - EventHubClientWrapper wrapper = - new EventHubClientWrapper(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING, 8, EVENTHUB_NAMESPACE, - EVENTHUB_ENTITY1, EVENTHUB_KEY_NAME, EVENTHUB_KEY); - EventHubClient client = wrapper.getEventHubClient(); - PartitionReceiver receiver = - client.createReceiverSync(EventHubClient.DEFAULT_CONSUMER_GROUP_NAME, "0", - "163456", true); - receiveMessages(receiver, 300); + systemProducer.flush(STREAM_NAME1); + systemProducer.stop(); + } + + @Test + public void testReceive() throws ServiceBusException { + EventHubClientWrapper wrapper = + new EventHubClientWrapper(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING, 8, EVENTHUB_NAMESPACE, + EVENTHUB_ENTITY1, EVENTHUB_KEY_NAME, EVENTHUB_KEY); + EventHubClient client = wrapper.getEventHubClient(); + PartitionReceiver receiver = + client.createReceiverSync(EventHubClient.DEFAULT_CONSUMER_GROUP_NAME, "0", + "163456", true); + receiveMessages(receiver, 300); + } + + private void receiveMessages(PartitionReceiver receiver, int numMessages) throws ServiceBusException { + int count = 0; + while (count < numMessages) { + + Iterable messages = receiver.receiveSync(100); + if (messages == null) { + System.out.println("End of stream"); + break; + } + for (EventData data : messages) { + count++; + LOG.info("Data" + new String(data.getBody())); + System.out.println("\nDATA: " + new String(data.getBody())); + System.out.println("BYTES_SIZE: " + data.getBytes().length); + System.out.println("OFFSET: " + data.getSystemProperties().getOffset()); + } } - - private void receiveMessages(PartitionReceiver receiver, int numMessages) throws ServiceBusException { - int count = 0; - while (count < numMessages) { - - Iterable messages = receiver.receiveSync(100); - if (messages == null) { - System.out.println("End of stream"); - break; - } - for (EventData data : messages) { - count++; - LOG.info("Data" + new String(data.getBody())); - System.out.println("\nDATA: " + new String(data.getBody())); - System.out.println("BYTES_SIZE: " + data.getBytes().length); - System.out.println("OFFSET: " + data.getSystemProperties().getOffset()); - } - } + } + + @Test + public void testSendToSpecificPartition() { + Config eventHubConfig = createEventHubConfig(EventHubClientWrapper.PartitioningMethod.PARTITION_KEY_AS_PARTITION); + EventHubSystemFactory systemFactory = new EventHubSystemFactory(); + SystemProducer systemProducer = systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); + + systemProducer.register(STREAM_NAME1); + systemProducer.start(); + for (int i = 0; i < 100; i++) { + systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1, 0)); } - - @Test - public void testSendToSpecificPartition() { - Config eventHubConfig = createEventHubConfig(EventHubClientWrapper.PartitioningMethod.PARTITION_KEY_AS_PARTITION); - EventHubSystemFactory systemFactory = new EventHubSystemFactory(); - SystemProducer systemProducer = systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); - - systemProducer.register(STREAM_NAME1); - systemProducer.start(); - for (int i = 0; i < 100; i++) { - systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1, 0)); - } - systemProducer.flush(STREAM_NAME1); - systemProducer.stop(); - } - - private OutgoingMessageEnvelope createMessageEnvelope(String streamName, int partition) { - return new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, streamName), partition, "key1".getBytes(), - "value0".getBytes()); - } - - private OutgoingMessageEnvelope createMessageEnvelope(String streamName) { - return new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, streamName), "key1".getBytes(), - "value".getBytes()); - } - - @Test - public void testFlush() { - Config eventHubConfig = createEventHubConfig(); - EventHubSystemFactory systemFactory = new EventHubSystemFactory(); - EventHubSystemProducer systemProducer = - (EventHubSystemProducer) systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); - systemProducer.register(STREAM_NAME1); - systemProducer.register(STREAM_NAME2); - systemProducer.start(); - int numEvents = 100; - for (int i = 0; i < numEvents; i++) { - systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1)); - systemProducer.send(STREAM_NAME2, createMessageEnvelope(STREAM_NAME2)); - } - systemProducer.flush(EVENTHUB_ENTITY1); - Assert.assertEquals(systemProducer.getPendingFutures().size(), 0); - systemProducer.stop(); + systemProducer.flush(STREAM_NAME1); + systemProducer.stop(); + } + + private OutgoingMessageEnvelope createMessageEnvelope(String streamName, int partition) { + return new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, streamName), partition, "key1".getBytes(), + "value0".getBytes()); + } + + private OutgoingMessageEnvelope createMessageEnvelope(String streamName) { + return new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, streamName), "key1".getBytes(), + "value".getBytes()); + } + + @Test + public void testFlush() { + Config eventHubConfig = createEventHubConfig(); + EventHubSystemFactory systemFactory = new EventHubSystemFactory(); + EventHubSystemProducer systemProducer = + (EventHubSystemProducer) systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); + systemProducer.register(STREAM_NAME1); + systemProducer.register(STREAM_NAME2); + systemProducer.start(); + int numEvents = 100; + for (int i = 0; i < numEvents; i++) { + systemProducer.send(STREAM_NAME1, createMessageEnvelope(STREAM_NAME1)); + systemProducer.send(STREAM_NAME2, createMessageEnvelope(STREAM_NAME2)); } + systemProducer.flush(EVENTHUB_ENTITY1); + Assert.assertEquals(systemProducer.getPendingFutures().size(), 0); + systemProducer.stop(); + } } From 39a53928828e8d88022d0e6f57a1d155f6814a47 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 25 Sep 2017 15:58:29 -0700 Subject: [PATCH 04/34] added admin tests, factored config creation --- .../system/eventhub/MockConfigFactory.java | 43 ++++++++++ .../admin/TestEventHubSystemAdmin.java | 78 ++++++++++++------- .../producer/TestEventHubSystemProducer.java | 50 +++--------- 3 files changed, 103 insertions(+), 68 deletions(-) create mode 100644 samza-azure/src/test/java/org/apache/samza/system/eventhub/MockConfigFactory.java diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockConfigFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockConfigFactory.java new file mode 100644 index 0000000000..e8c951a41b --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockConfigFactory.java @@ -0,0 +1,43 @@ +package org.apache.samza.system.eventhub; + +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; + +import java.util.HashMap; + +public class MockConfigFactory { + + public static final String SYSTEM_NAME = "eventhub-s1"; + public static final String STREAM_NAME1 = "test_stream1"; + public static final String STREAM_NAME2 = "test_stream2"; + + // Add target Event Hub connection information here + public static final String EVENTHUB_NAMESPACE = ""; + public static final String EVENTHUB_KEY_NAME = ""; + public static final String EVENTHUB_KEY = ""; + public static final String EVENTHUB_ENTITY1 = ""; + public static final String EVENTHUB_ENTITY2 = ""; + + public static final int MIN_EVENTHUB_ENTITY_PARTITION = 2; + + public static Config getEventHubConfig(EventHubClientWrapper.PartitioningMethod partitioningMethod) { + HashMap mapConfig = new HashMap<>(); + mapConfig.put(EventHubSystemProducer.CONFIG_PARTITIONING_METHOD, partitioningMethod.toString()); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_NAMESPACE); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY_NAME); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME1), "earliest"); + + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME2); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_NAMESPACE); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_ENTITY2); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY_NAME); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME2), "earliest"); + + return new MapConfig(mapConfig); + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java index dd59078142..e68b876cfa 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java @@ -1,44 +1,30 @@ package org.apache.samza.system.eventhub.admin; import junit.framework.Assert; -import org.apache.samza.config.Config; -import org.apache.samza.config.MapConfig; +import org.apache.samza.Partition; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemStreamMetadata; +import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.eventhub.EventHubClientWrapper; -import org.apache.samza.system.eventhub.EventHubConfig; import org.apache.samza.system.eventhub.EventHubSystemFactory; +import org.apache.samza.system.eventhub.MockConfigFactory; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; -import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; import org.junit.Test; import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; -public class TestEventHubSystemAdmin { - - public static final String SYSTEM_NAME = "eventhub-s1"; - public static final String STREAM_NAME1 = "test_stream1"; - - public static final String EVENTHUB_NAMESPACE = ""; - public static final String EVENTHUB_ENTITY1 = ""; - public static final String EVENTHUB_KEY_NAME = ""; - public static final String EVENTHUB_KEY = ""; +import static org.apache.samza.system.eventhub.MockConfigFactory.*; - private Config createEventHubConfig(EventHubClientWrapper.PartitioningMethod partitioningMethod) { - HashMap mapConfig = new HashMap<>(); - mapConfig.put(EventHubSystemProducer.CONFIG_PARTITIONING_METHOD, partitioningMethod.toString()); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_NAMESPACE); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY_NAME); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME1), "earliest"); - return new MapConfig(mapConfig); - } +public class TestEventHubSystemAdmin { @Test public void testOffsetComparison() { EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); EventHubSystemAdmin eventHubSystemAdmin = (EventHubSystemAdmin) eventHubSystemFactory.getAdmin(SYSTEM_NAME, - createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + MockConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); Assert.assertEquals(-1, eventHubSystemAdmin.offsetComparator("100", "200").intValue()); Assert.assertEquals(0, eventHubSystemAdmin.offsetComparator("150", "150").intValue()); Assert.assertEquals(1, eventHubSystemAdmin.offsetComparator("200", "100").intValue()); @@ -56,9 +42,47 @@ public void testOffsetComparison() { @Test public void testGetNextOffset() { EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); - EventHubSystemAdmin eventHubSystemAdmin = (EventHubSystemAdmin) eventHubSystemFactory.getAdmin(SYSTEM_NAME, - createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + SystemAdmin eventHubSystemAdmin = eventHubSystemFactory.getAdmin(SYSTEM_NAME, + MockConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + Map offsets = new HashMap<>(); + SystemStreamPartition ssp0 = new SystemStreamPartition(SYSTEM_NAME, STREAM_NAME1, new Partition(0)); + SystemStreamPartition ssp1 = new SystemStreamPartition(SYSTEM_NAME, STREAM_NAME1, new Partition(1)); + SystemStreamPartition ssp2 = new SystemStreamPartition(SYSTEM_NAME, STREAM_NAME1, new Partition(2)); + offsets.put(ssp0, Integer.toString(0)); + offsets.put(ssp1, EventHubSystemConsumer.END_OF_STREAM); + offsets.put(ssp2, EventHubSystemConsumer.START_OF_STREAM); + + Map updatedOffsets = eventHubSystemAdmin.getOffsetsAfter(offsets); + Assert.assertEquals(offsets.size(), updatedOffsets.size()); + Assert.assertEquals("1", updatedOffsets.get(ssp0)); + Assert.assertEquals("-2", updatedOffsets.get(ssp1)); + Assert.assertEquals("0", updatedOffsets.get(ssp2)); + } + + @Test + public void testGetStreamMetadata() { + EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); + SystemAdmin eventHubSystemAdmin = eventHubSystemFactory.getAdmin(SYSTEM_NAME, + MockConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + Set streams = new HashSet<>(); + streams.add(STREAM_NAME1); + streams.add(STREAM_NAME2); + Map metadataMap = eventHubSystemAdmin.getSystemStreamMetadata(streams); + + for (String stream : streams) { + Assert.assertTrue(metadataMap.containsKey(stream)); + Assert.assertEquals(stream, metadataMap.get(stream).getStreamName()); + Assert.assertNotNull(metadataMap.get(stream).getSystemStreamPartitionMetadata()); + Map partitionMetadataMap = + metadataMap.get(stream).getSystemStreamPartitionMetadata(); + Assert.assertTrue(partitionMetadataMap.size() >= MIN_EVENTHUB_ENTITY_PARTITION); + partitionMetadataMap.forEach((partition, metadata) -> { + Assert.assertEquals(EventHubSystemConsumer.START_OF_STREAM, metadata.getOldestOffset()); + Assert.assertEquals(EventHubSystemConsumer.END_OF_STREAM, metadata.getNewestOffset()); + Assert.assertEquals(EventHubSystemConsumer.END_OF_STREAM, metadata.getUpcomingOffset()); + }); + } } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java index ee5f375cdb..ccfe9a7e8b 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -6,30 +6,22 @@ import com.microsoft.azure.servicebus.ServiceBusException; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; -import org.apache.samza.config.MapConfig; import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemProducer; import org.apache.samza.system.SystemStream; import org.apache.samza.system.eventhub.EventHubClientWrapper; -import org.apache.samza.system.eventhub.EventHubConfig; import org.apache.samza.system.eventhub.EventHubSystemFactory; +import org.apache.samza.system.eventhub.MockConfigFactory; +import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.apache.samza.util.NoOpMetricsRegistry; import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; +import static org.apache.samza.system.eventhub.MockConfigFactory.*; public class TestEventHubSystemProducer { - public static final String EVENTHUB_NAMESPACE = "" + - ""; - public static final String EVENTHUB_ENTITY1 = ""; - public static final String EVENTHUB_KEY_NAME = ""; - public static final String EVENTHUB_KEY = ""; - public static final String SYSTEM_NAME = "system1"; - public static final String STREAM_NAME1 = "test_stream1"; - public static final String STREAM_NAME2 = "test_stream2"; private static final Logger LOG = LoggerFactory.getLogger(TestEventHubSystemProducer.class.getName()); @Test @@ -46,34 +38,14 @@ public void testSystemFactoryCreateAndStartProducer() { } private Config createEventHubConfig() { - return createEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING); - } - - private Config createEventHubConfig(EventHubClientWrapper.PartitioningMethod partitioningMethod) { - HashMap mapConfig = new HashMap<>(); - mapConfig.put(EventHubSystemProducer.CONFIG_PARTITIONING_METHOD, partitioningMethod.toString()); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_NAMESPACE); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY_NAME); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME1), "earliest"); - - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME2); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_NAMESPACE); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_ENTITY1); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY_NAME); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME2), "earliest"); - - return new MapConfig(mapConfig); + return MockConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING); } @Test public void testSend() { Config eventHubConfig = createEventHubConfig(); EventHubSystemFactory systemFactory = new EventHubSystemFactory(); - SystemProducer systemProducer = systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); + SystemProducer systemProducer = systemFactory.getProducer(SYSTEM_NAME, eventHubConfig, new NoOpMetricsRegistry()); systemProducer.register(STREAM_NAME1); @@ -110,7 +82,7 @@ public void testReceive() throws ServiceBusException { EventHubClient client = wrapper.getEventHubClient(); PartitionReceiver receiver = client.createReceiverSync(EventHubClient.DEFAULT_CONSUMER_GROUP_NAME, "0", - "163456", true); + EventHubSystemConsumer.START_OF_STREAM, true); receiveMessages(receiver, 300); } @@ -120,24 +92,20 @@ private void receiveMessages(PartitionReceiver receiver, int numMessages) throws Iterable messages = receiver.receiveSync(100); if (messages == null) { - System.out.println("End of stream"); break; } for (EventData data : messages) { count++; LOG.info("Data" + new String(data.getBody())); - System.out.println("\nDATA: " + new String(data.getBody())); - System.out.println("BYTES_SIZE: " + data.getBytes().length); - System.out.println("OFFSET: " + data.getSystemProperties().getOffset()); } } } @Test public void testSendToSpecificPartition() { - Config eventHubConfig = createEventHubConfig(EventHubClientWrapper.PartitioningMethod.PARTITION_KEY_AS_PARTITION); + Config eventHubConfig = MockConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.PARTITION_KEY_AS_PARTITION); EventHubSystemFactory systemFactory = new EventHubSystemFactory(); - SystemProducer systemProducer = systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); + SystemProducer systemProducer = systemFactory.getProducer(SYSTEM_NAME, eventHubConfig, new NoOpMetricsRegistry()); systemProducer.register(STREAM_NAME1); systemProducer.start(); @@ -163,7 +131,7 @@ public void testFlush() { Config eventHubConfig = createEventHubConfig(); EventHubSystemFactory systemFactory = new EventHubSystemFactory(); EventHubSystemProducer systemProducer = - (EventHubSystemProducer) systemFactory.getProducer("system1", eventHubConfig, new NoOpMetricsRegistry()); + (EventHubSystemProducer) systemFactory.getProducer(SYSTEM_NAME, eventHubConfig, new NoOpMetricsRegistry()); systemProducer.register(STREAM_NAME1); systemProducer.register(STREAM_NAME2); systemProducer.start(); From 9a9012ce1c22bce3e7cbe33385c53a29bd99acff Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 25 Sep 2017 17:28:01 -0700 Subject: [PATCH 05/34] factored producer configs --- .../samza/system/eventhub/EventHubConfig.java | 31 +++++++++++++++++++ .../eventhub/EventHubSystemFactory.java | 2 +- .../producer/EventHubSystemProducer.java | 30 ++++++------------ .../producer/TestEventHubSystemProducer.java | 4 +-- 4 files changed, 44 insertions(+), 23 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java index 40f93fa227..a33681de7b 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -30,6 +30,14 @@ public class EventHubConfig extends MapConfig { public static final String CONFIG_STREAM_CONSUMER_START_POSITION = "systems.%s.streams.%s.eventhubs.start.position"; public static final String DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION = StartPosition.LATEST.name(); + + public static final String CONFIG_PRODUCER_PARTITION_METHOD = "systems.%s.eventhubs.partition.method"; + public static final String DEFAULT_CONFIG_PRODUCER_PARTITION_METHOD = EventHubClientWrapper + .PartitioningMethod.EVENT_HUB_HASHING.name(); + + public static final String CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = "systems.%s.eventhubs.send.key"; + public static final String DEFAULT_CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = Boolean.toString(false); + private final String _system; public EventHubConfig(Map config, String systemName) { @@ -119,6 +127,29 @@ public StartPosition getStartPosition(String streamName) { return StartPosition.valueOf(startPositionStr.toUpperCase()); } + /** + * Get the partition method of the system. By default partitioning is handed by EventHub. + * + * @return The method the producer should use to partition the outgoing data + */ + public EventHubClientWrapper.PartitioningMethod getPartitioningMethod() { + String partitioningMethod = get(String.format(CONFIG_PRODUCER_PARTITION_METHOD, _system), + DEFAULT_CONFIG_PRODUCER_PARTITION_METHOD); + return EventHubClientWrapper.PartitioningMethod.valueOf(partitioningMethod); + + } + + /** + * Returns true if the OutgoingMessageEnvelope key should be sent in the outgoing envelope, false otherwise + * + * @return Boolean, is send key included + */ + public Boolean getSendKeyInEventProperties() { + String isSendKeyIncluded = get(String.format(CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, _system), + DEFAULT_CONFIG_SEND_KEY_IN_EVENT_PROPERTIES); + return Boolean.valueOf(isSendKeyIncluded); + } + @Override public int hashCode() { return Objects.hash(super.hashCode(), _system); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java index dd12916733..a77237f2d0 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java @@ -38,7 +38,7 @@ public SystemConsumer getConsumer(String systemName, Config config, MetricsRegis @Override public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { - return new EventHubSystemProducer(systemName, config, registry); + return new EventHubSystemProducer(systemName, new EventHubConfig(config, systemName), registry); } @Override diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 85eea2f05f..884fd41f07 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -24,7 +24,6 @@ import java.util.concurrent.ConcurrentHashMap; public class EventHubSystemProducer implements SystemProducer { - public static final String EVENT_SOURCE_TIMESTAMP = "event-source-timestamp"; public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; public static final String AGGREGATE = "aggregate"; public final static String CONFIG_PARTITIONING_METHOD = "partitioningMethod"; @@ -44,10 +43,7 @@ public class EventHubSystemProducer implements SystemProducer { private static SamzaHistogram _aggSendCallbackLatency = null; private static Counter _aggSendErrors = null; private final int _destinationPartitions; - private final boolean _sendKeyInEventProperties; - private final EventHubConfig _eventHubsConfig; - private final Config _config; - private final EventHubClientWrapper.PartitioningMethod _partitioningMethod; + private final EventHubConfig _config; private final String _systemName; private final MetricsRegistry _registry; private Throwable _sendExceptionOnCallback; @@ -65,19 +61,13 @@ public class EventHubSystemProducer implements SystemProducer { private Map> _pendingFutures = new ConcurrentHashMap<>(); - public EventHubSystemProducer(String systemName, Config config, MetricsRegistry registry) { + public EventHubSystemProducer(String systemName, EventHubConfig config, MetricsRegistry registry) { _messageId = 0; _systemName = systemName; - _registry = registry; _config = config; - _partitioningMethod = - EventHubClientWrapper.PartitioningMethod.valueOf(getConfigValue(config, CONFIG_PARTITIONING_METHOD, DEFAULT_PARTITIONING_METHOD)); - - _eventHubsConfig = new EventHubConfig(config, systemName); - _sendKeyInEventProperties = - Boolean.parseBoolean(getConfigValue(config, CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, "false")); + _registry = registry; - // TODO this should be removed when we are able to find the number of partitions. + // TODO this should be removed when we are able to find the number of partitions. Remove _destinationPartitions = Integer.parseInt(getConfigValue(config, CONFIG_DESTINATION_NUM_PARTITION, "-1")); } @@ -132,15 +122,15 @@ public synchronized void register(String streamName) { throw new SamzaException(msg); } - String ehNamespace = _eventHubsConfig.getStreamNamespace(streamName); - String ehName = _eventHubsConfig.getStreamEntityPath(streamName); + String ehNamespace = _config.getStreamNamespace(streamName); + String ehName = _config.getStreamEntityPath(streamName); EventHubClientWrapper ehClient = - new EventHubClientWrapper(_partitioningMethod, _destinationPartitions, ehNamespace, ehName, - _eventHubsConfig.getStreamSasKeyName(streamName), _eventHubsConfig.getStreamSasToken(streamName)); + new EventHubClientWrapper(_config.getPartitioningMethod(), _destinationPartitions, ehNamespace, ehName, + _config.getStreamSasKeyName(streamName), _config.getStreamSasToken(streamName)); _eventHubClients.put(streamName, ehClient); - _eventHubsConfig.getSerde(streamName).ifPresent(x -> _serdes.put(streamName, x)); + _config.getSerde(streamName).ifPresent(x -> _serdes.put(streamName, x)); } @Override @@ -217,7 +207,7 @@ private EventData createEventData(String streamName, OutgoingMessageEnvelope env eventData.getProperties().put(PRODUCE_TIMESTAMP, Long.toString(System.currentTimeMillis())); - if (_sendKeyInEventProperties) { + if (_config.getSendKeyInEventProperties()) { String keyValue = ""; if (envelope.getKey() != null) { keyValue = (envelope.getKey() instanceof byte[]) ? new String((byte[]) envelope.getKey()) diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java index ccfe9a7e8b..880fc18792 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -77,8 +77,8 @@ public void testSend() { @Test public void testReceive() throws ServiceBusException { EventHubClientWrapper wrapper = - new EventHubClientWrapper(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING, 8, EVENTHUB_NAMESPACE, - EVENTHUB_ENTITY1, EVENTHUB_KEY_NAME, EVENTHUB_KEY); + new EventHubClientWrapper(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING, 8, + EVENTHUB_NAMESPACE, EVENTHUB_ENTITY1, EVENTHUB_KEY_NAME, EVENTHUB_KEY); EventHubClient client = wrapper.getEventHubClient(); PartitionReceiver receiver = client.createReceiverSync(EventHubClient.DEFAULT_CONSUMER_GROUP_NAME, "0", From c1e5bfd1f9d746cd2ff04d83562ff472ffc1bb54 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Tue, 26 Sep 2017 10:44:01 -0700 Subject: [PATCH 06/34] cleanup producer, added test --- .../producer/EventHubSystemProducer.java | 39 ++++++++----------- ...ry.java => MockEventHubConfigFactory.java} | 6 +-- .../admin/TestEventHubSystemAdmin.java | 11 +++--- .../producer/TestEventHubSystemProducer.java | 8 ++-- 4 files changed, 29 insertions(+), 35 deletions(-) rename samza-azure/src/test/java/org/apache/samza/system/eventhub/{MockConfigFactory.java => MockEventHubConfigFactory.java} (90%) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 884fd41f07..6543af9e03 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -24,35 +24,38 @@ import java.util.concurrent.ConcurrentHashMap; public class EventHubSystemProducer implements SystemProducer { + private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemProducer.class.getName()); + + public static final String CONFIG_DESTINATION_NUM_PARTITION = "destinationPartitions"; public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; + + // Metrics recording public static final String AGGREGATE = "aggregate"; - public final static String CONFIG_PARTITIONING_METHOD = "partitioningMethod"; - public final static String DEFAULT_PARTITIONING_METHOD = EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING.toString(); - public static final String CONFIG_DESTINATION_NUM_PARTITION = "destinationPartitions"; - public static final String CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = "sendKeyInEventProperties"; private static final String EVENT_WRITE_RATE = "eventWriteRate"; private static final String EVENT_BYTE_WRITE_RATE = "eventByteWriteRate"; private static final String SEND_ERRORS = "sendErrors"; private static final String SEND_LATENCY = "sendLatency"; private static final String SEND_CALLBACK_LATENCY = "sendCallbackLatency"; - private static final Duration SHUTDOWN_WAIT_TIME = Duration.ofMinutes(1L); - private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemProducer.class.getName()); private static Counter _aggEventWriteRate = null; private static Counter _aggEventByteWriteRate = null; + private static Counter _aggSendErrors = null; private static SamzaHistogram _aggSendLatency = null; private static SamzaHistogram _aggSendCallbackLatency = null; - private static Counter _aggSendErrors = null; + private HashMap _eventWriteRate = new HashMap<>(); + private HashMap _eventByteWriteRate = new HashMap<>(); + private HashMap _sendLatency = new HashMap<>(); + private HashMap _sendCallbackLatency = new HashMap<>(); + private HashMap _sendErrors = new HashMap<>(); + private static final Duration SHUTDOWN_WAIT_TIME = Duration.ofMinutes(1L); + private final int _destinationPartitions; private final EventHubConfig _config; private final String _systemName; private final MetricsRegistry _registry; + private Throwable _sendExceptionOnCallback; private boolean _isStarted; - private HashMap _eventWriteRate = new HashMap<>(); - private HashMap _eventByteWriteRate = new HashMap<>(); - private HashMap _sendLatency = new HashMap<>(); - private HashMap _sendCallbackLatency = new HashMap<>(); - private HashMap _sendErrors = new HashMap<>(); + // Map of the system name to the event hub client. private Map _eventHubClients = new HashMap<>(); @@ -68,17 +71,7 @@ public EventHubSystemProducer(String systemName, EventHubConfig config, MetricsR _registry = registry; // TODO this should be removed when we are able to find the number of partitions. Remove - _destinationPartitions = Integer.parseInt(getConfigValue(config, CONFIG_DESTINATION_NUM_PARTITION, "-1")); - } - - private String getConfigValue(Config config, String configKey, String defaultValue) { - String configValue = config.get(configKey, defaultValue); - - if (configValue == null) { - throw new SamzaException(configKey + " is not configured."); - } - - return configValue; + _destinationPartitions = Integer.parseInt(config.get(CONFIG_DESTINATION_NUM_PARTITION, "-1")); } @Override diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockConfigFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java similarity index 90% rename from samza-azure/src/test/java/org/apache/samza/system/eventhub/MockConfigFactory.java rename to samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java index e8c951a41b..17a11f36af 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockConfigFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java @@ -2,11 +2,10 @@ import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; -import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; import java.util.HashMap; -public class MockConfigFactory { +public class MockEventHubConfigFactory { public static final String SYSTEM_NAME = "eventhub-s1"; public static final String STREAM_NAME1 = "test_stream1"; @@ -20,10 +19,11 @@ public class MockConfigFactory { public static final String EVENTHUB_ENTITY2 = ""; public static final int MIN_EVENTHUB_ENTITY_PARTITION = 2; + public static final int MAX_EVENTHUB_ENTITY_PARTITION = 32; public static Config getEventHubConfig(EventHubClientWrapper.PartitioningMethod partitioningMethod) { HashMap mapConfig = new HashMap<>(); - mapConfig.put(EventHubSystemProducer.CONFIG_PARTITIONING_METHOD, partitioningMethod.toString()); + mapConfig.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, SYSTEM_NAME), partitioningMethod.toString()); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_NAMESPACE); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java index e68b876cfa..6e97b5a53d 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java @@ -7,7 +7,7 @@ import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.eventhub.EventHubClientWrapper; import org.apache.samza.system.eventhub.EventHubSystemFactory; -import org.apache.samza.system.eventhub.MockConfigFactory; +import org.apache.samza.system.eventhub.MockEventHubConfigFactory; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.junit.Test; @@ -16,7 +16,7 @@ import java.util.Map; import java.util.Set; -import static org.apache.samza.system.eventhub.MockConfigFactory.*; +import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.*; public class TestEventHubSystemAdmin { @@ -24,7 +24,7 @@ public class TestEventHubSystemAdmin { public void testOffsetComparison() { EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); EventHubSystemAdmin eventHubSystemAdmin = (EventHubSystemAdmin) eventHubSystemFactory.getAdmin(SYSTEM_NAME, - MockConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + MockEventHubConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); Assert.assertEquals(-1, eventHubSystemAdmin.offsetComparator("100", "200").intValue()); Assert.assertEquals(0, eventHubSystemAdmin.offsetComparator("150", "150").intValue()); Assert.assertEquals(1, eventHubSystemAdmin.offsetComparator("200", "100").intValue()); @@ -43,7 +43,7 @@ public void testOffsetComparison() { public void testGetNextOffset() { EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); SystemAdmin eventHubSystemAdmin = eventHubSystemFactory.getAdmin(SYSTEM_NAME, - MockConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + MockEventHubConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); Map offsets = new HashMap<>(); SystemStreamPartition ssp0 = new SystemStreamPartition(SYSTEM_NAME, STREAM_NAME1, new Partition(0)); SystemStreamPartition ssp1 = new SystemStreamPartition(SYSTEM_NAME, STREAM_NAME1, new Partition(1)); @@ -63,7 +63,7 @@ public void testGetNextOffset() { public void testGetStreamMetadata() { EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); SystemAdmin eventHubSystemAdmin = eventHubSystemFactory.getAdmin(SYSTEM_NAME, - MockConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + MockEventHubConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); Set streams = new HashSet<>(); streams.add(STREAM_NAME1); streams.add(STREAM_NAME2); @@ -77,6 +77,7 @@ public void testGetStreamMetadata() { Map partitionMetadataMap = metadataMap.get(stream).getSystemStreamPartitionMetadata(); Assert.assertTrue(partitionMetadataMap.size() >= MIN_EVENTHUB_ENTITY_PARTITION); + Assert.assertTrue(partitionMetadataMap.size() <= MAX_EVENTHUB_ENTITY_PARTITION); partitionMetadataMap.forEach((partition, metadata) -> { Assert.assertEquals(EventHubSystemConsumer.START_OF_STREAM, metadata.getOldestOffset()); Assert.assertEquals(EventHubSystemConsumer.END_OF_STREAM, metadata.getNewestOffset()); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java index 880fc18792..4997c9673f 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -11,7 +11,7 @@ import org.apache.samza.system.SystemStream; import org.apache.samza.system.eventhub.EventHubClientWrapper; import org.apache.samza.system.eventhub.EventHubSystemFactory; -import org.apache.samza.system.eventhub.MockConfigFactory; +import org.apache.samza.system.eventhub.MockEventHubConfigFactory; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.apache.samza.util.NoOpMetricsRegistry; import org.junit.Assert; @@ -19,7 +19,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.samza.system.eventhub.MockConfigFactory.*; +import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.*; public class TestEventHubSystemProducer { private static final Logger LOG = LoggerFactory.getLogger(TestEventHubSystemProducer.class.getName()); @@ -38,7 +38,7 @@ public void testSystemFactoryCreateAndStartProducer() { } private Config createEventHubConfig() { - return MockConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING); + return MockEventHubConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING); } @Test @@ -103,7 +103,7 @@ private void receiveMessages(PartitionReceiver receiver, int numMessages) throws @Test public void testSendToSpecificPartition() { - Config eventHubConfig = MockConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.PARTITION_KEY_AS_PARTITION); + Config eventHubConfig = MockEventHubConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.PARTITION_KEY_AS_PARTITION); EventHubSystemFactory systemFactory = new EventHubSystemFactory(); SystemProducer systemProducer = systemFactory.getProducer(SYSTEM_NAME, eventHubConfig, new NoOpMetricsRegistry()); From 3aad90b243677f69dacadf8600693caf60a058a6 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Thu, 28 Sep 2017 17:35:10 -0700 Subject: [PATCH 07/34] abstract eventhub client wrapper --- .../eventhub/EventHubClientFactory.java | 8 + .../eventhub/EventHubClientWrapper.java | 159 ++---------------- .../samza/system/eventhub/EventHubConfig.java | 10 +- .../eventhub/SamzaEventHubClientWrapper.java | 76 +++++++++ .../eventhub/admin/EventHubSystemAdmin.java | 15 +- .../consumer/EventHubEntityConnection.java | 17 +- .../EventHubEntityConnectionFactory.java | 6 +- .../consumer/EventHubSystemConsumer.java | 2 +- .../producer/EventHubSystemProducer.java | 107 +++++++++--- .../eventhub/MockEventHubConfigFactory.java | 3 +- .../admin/TestEventHubSystemAdmin.java | 8 +- .../MockEventHubEntityConnectionFactory.java | 6 +- .../producer/TestEventHubSystemProducer.java | 16 +- 13 files changed, 235 insertions(+), 198 deletions(-) create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientFactory.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientWrapper.java diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientFactory.java new file mode 100644 index 0000000000..50cd54a14d --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientFactory.java @@ -0,0 +1,8 @@ +package org.apache.samza.system.eventhub; + +public class EventHubClientFactory { + public EventHubClientWrapper getEventHubClient(String eventHubNamespace, String entityPath, String sasKeyName, + String sasToken, EventHubConfig config) { + return new SamzaEventHubClientWrapper(eventHubNamespace, entityPath, sasKeyName, sasToken); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java index 1f8cc87e6c..903abc101a 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java @@ -1,151 +1,26 @@ package org.apache.samza.system.eventhub; -import com.microsoft.azure.eventhubs.EventData; import com.microsoft.azure.eventhubs.EventHubClient; -import com.microsoft.azure.eventhubs.PartitionSender; -import com.microsoft.azure.servicebus.ClientConstants; -import com.microsoft.azure.servicebus.ConnectionStringBuilder; -import com.microsoft.azure.servicebus.ServiceBusException; -import org.apache.samza.SamzaException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -public class EventHubClientWrapper { - private static final Logger LOG = LoggerFactory.getLogger(EventHubClientWrapper.class.getName()); - private final PartitioningMethod _partitioningMethod; - private final int _numPartitions; - private EventHubClient _eventHubClient; - private Map _partitionSenders = new HashMap<>(); - - public EventHubClientWrapper(PartitioningMethod partitioningMethod, int numPartitions, - String eventHubNamespace, String entityPath, String sasKeyName, String sasToken) { - String remoteHost = String.format("%s.servicebus.windows.net", eventHubNamespace); - _partitioningMethod = partitioningMethod; - _numPartitions = numPartitions; - try { - // Create a event hub connection string pointing to localhost - ConnectionStringBuilder connectionStringBuilder = - new ConnectionStringBuilder(eventHubNamespace, entityPath, sasKeyName, sasToken); - - _eventHubClient = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString()); - } catch (IOException ioe) { - throw new IllegalStateException( - "Failed to connect to remote host " + remoteHost + ":" + ClientConstants.AMQPS_PORT, ioe); - } catch (ServiceBusException e) { - String msg = String.format("Creation of event hub client failed for eventHub %s %s %s %s %s %s with exception", - entityPath, partitioningMethod, numPartitions, eventHubNamespace, sasKeyName, sasToken); - LOG.error(msg, e); - throw new SamzaException(msg, e); - } - } - - public EventHubClient getEventHubClient() { - return _eventHubClient; - } - - public void closeSync() { - _partitionSenders.entrySet().forEach(x -> { - try { - x.getValue().closeSync(); - } catch (ServiceBusException e) { - LOG.warn("Closing the partition sender failed for partition " + x.getKey(), e); - } - }); - - try { - _eventHubClient.closeSync(); - } catch (ServiceBusException e) { - LOG.warn("Closing the event hub client failed ", e); - } - } +public interface EventHubClientWrapper { + /** + * Initiate the connection to EventHub + */ + void init(); /** - * Timed connection close. + * Returns the EventHubClient instance of the wrapper so its methods can be invoked directly * - * @param timeoutMS Time in Miliseconds to wait for individual components (partition senders, event hub client and - * tunnel in order) to shutdown before moving to the next stage. - * For example a timeoutMS of 30000, will result in a wait of max 30secs for a successful close of - * all partition senders followed by a max 30secs wait for a successful close of eventhub client and - * then another max 30secs wait for successful tunnel close. Tunnel close failure/timeout will result - * in RuntimeException. + * @return EventHub client instance of the wrapper */ + EventHubClient getEventHubClient(); - public void closeSync(long timeoutMS) { - List> futures = new ArrayList<>(); - _partitionSenders.entrySet().forEach(x -> futures.add(x.getValue().close())); - CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); - try { - future.get(timeoutMS, TimeUnit.MILLISECONDS); - } catch (ExecutionException | InterruptedException | TimeoutException e) { - LOG.warn("Closing the partition sender failed ", e); - } - - future = _eventHubClient.close(); - try { - future.get(timeoutMS, TimeUnit.MILLISECONDS); - } catch (ExecutionException | InterruptedException | TimeoutException e) { - LOG.warn("Closing the event hub client failed ", e); - } - } - - public CompletableFuture send(EventData eventData, Object partitionKey) { - if (_partitioningMethod == PartitioningMethod.EVENT_HUB_HASHING) { - return _eventHubClient.send(eventData, convertPartitionKeyToString(partitionKey)); - } else if (_partitioningMethod == PartitioningMethod.PARTITION_KEY_AS_PARTITION) { - if (!(partitionKey instanceof Integer)) { - String msg = "Partition key should be of type Integer"; - LOG.error(msg); - throw new SamzaException(msg); - } - - PartitionSender sender = getPartitionSender((int) partitionKey); - return sender.send(eventData); - } else { - throw new SamzaException("Unknown partitioning method " + _partitioningMethod); - } - } - - private String convertPartitionKeyToString(Object partitionKey) { - if (partitionKey instanceof String) { - return (String) partitionKey; - } else if (partitionKey instanceof Integer) { - return String.valueOf(partitionKey); - } else if (partitionKey instanceof byte[]) { - return new String((byte[]) partitionKey, Charset.defaultCharset()); - } else { - throw new SamzaException("Unsupported key type: " + partitionKey.getClass().toString()); - } - } - - private PartitionSender getPartitionSender(int partition) { - if (!_partitionSenders.containsKey(partition)) { - try { - PartitionSender partitionSender = - _eventHubClient.createPartitionSenderSync(String.valueOf(partition % _numPartitions)); - _partitionSenders.put(partition, partitionSender); - } catch (ServiceBusException e) { - String msg = "Creation of partition sender failed with exception"; - LOG.error(msg, e); - throw new SamzaException(msg, e); - } - } - - return _partitionSenders.get(partition); - } - - public enum PartitioningMethod { - EVENT_HUB_HASHING, - PARTITION_KEY_AS_PARTITION, - } -} + /** + * Timed synchronous connection close to the EventHub. + * + * @param timeoutMS + * Time in Milliseconds to wait for individual components to + * shutdown before moving to the next stage. + */ + void close(long timeoutMS); +} \ No newline at end of file diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java index a33681de7b..0f76ad888d 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -5,6 +5,7 @@ import org.apache.samza.config.MapConfig; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.SerdeFactory; +import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; import java.util.List; import java.util.Map; @@ -32,12 +33,15 @@ public class EventHubConfig extends MapConfig { public static final String DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION = StartPosition.LATEST.name(); public static final String CONFIG_PRODUCER_PARTITION_METHOD = "systems.%s.eventhubs.partition.method"; - public static final String DEFAULT_CONFIG_PRODUCER_PARTITION_METHOD = EventHubClientWrapper + public static final String DEFAULT_CONFIG_PRODUCER_PARTITION_METHOD = EventHubSystemProducer .PartitioningMethod.EVENT_HUB_HASHING.name(); public static final String CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = "systems.%s.eventhubs.send.key"; public static final String DEFAULT_CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = Boolean.toString(false); + public static final String CONFIG_GET_RUNTIME_INFO_TIMEOUT_MILLIS = "systems.%s.eventhubs.getruntime.timeout"; + public static final String DEFAULT_CONFIG_GET_RUNTIME_INFO_TIMEOUT_MILLIS = Integer.toString(1000); + private final String _system; public EventHubConfig(Map config, String systemName) { @@ -132,10 +136,10 @@ public StartPosition getStartPosition(String streamName) { * * @return The method the producer should use to partition the outgoing data */ - public EventHubClientWrapper.PartitioningMethod getPartitioningMethod() { + public EventHubSystemProducer.PartitioningMethod getPartitioningMethod() { String partitioningMethod = get(String.format(CONFIG_PRODUCER_PARTITION_METHOD, _system), DEFAULT_CONFIG_PRODUCER_PARTITION_METHOD); - return EventHubClientWrapper.PartitioningMethod.valueOf(partitioningMethod); + return EventHubSystemProducer.PartitioningMethod.valueOf(partitioningMethod); } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientWrapper.java new file mode 100644 index 0000000000..6ed4c15d6e --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientWrapper.java @@ -0,0 +1,76 @@ +package org.apache.samza.system.eventhub; + +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.servicebus.ClientConstants; +import com.microsoft.azure.servicebus.ConnectionStringBuilder; +import com.microsoft.azure.servicebus.ServiceBusException; +import org.apache.samza.SamzaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class SamzaEventHubClientWrapper implements EventHubClientWrapper { + private static final Logger LOG = LoggerFactory.getLogger(SamzaEventHubClientWrapper.class.getName()); + private static final String EVENTHUB_REMOTE_HOST_FORMAT = "%s.servicebus.windows.net"; + + private EventHubClient _eventHubClient; + + private final String _eventHubNamespace; + private final String _entityPath; + private final String _sasKeyName; + private final String _sasKey; + + public SamzaEventHubClientWrapper(String eventHubNamespace, String entityPath, String sasKeyName, String sasKey) { + _eventHubNamespace = eventHubNamespace; + _entityPath = entityPath; + _sasKeyName = sasKeyName; + _sasKey = sasKey; + } + + public void init() { + String remoteHost = String.format(EVENTHUB_REMOTE_HOST_FORMAT, _eventHubNamespace); + try { + ConnectionStringBuilder connectionStringBuilder = + new ConnectionStringBuilder(_eventHubNamespace, _entityPath, _sasKeyName, _sasKey); + + _eventHubClient = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString()); + } catch (IOException ioe) { + throw new IllegalStateException( + "Failed to connect to remote host " + remoteHost + ":" + ClientConstants.AMQPS_PORT, ioe); + } catch (ServiceBusException e) { + String msg = String.format("Creation of event hub client failed for eventHub %s %s %s %s with exception", + _entityPath, _eventHubNamespace, _sasKeyName, _sasKey); + LOG.error(msg, e); + throw new SamzaException(msg, e); + } + } + + public EventHubClient getEventHubClient() { + return _eventHubClient; + } + + + public void close(long timeoutMS) { + if (timeoutMS <= 0) { + try { + _eventHubClient.closeSync(); + } catch (ServiceBusException e) { + LOG.warn("Closing the event hub client failed ", e); + } + } else { + CompletableFuture future = _eventHubClient.close(); + try { + future.get(timeoutMS, TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + LOG.warn("Closing the event hub client failed ", e); + } + } + + } + +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index 1323e429d9..370018a4ae 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -6,6 +6,7 @@ import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.system.eventhub.EventHubClientFactory; import org.apache.samza.system.eventhub.EventHubClientWrapper; import org.apache.samza.system.eventhub.EventHubConfig; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; @@ -19,7 +20,7 @@ public class EventHubSystemAdmin implements SystemAdmin { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemAdmin.class); - + private final EventHubClientFactory _eventHubClientFactory = new EventHubClientFactory(); private String _systemName; private EventHubConfig _config; private Map _eventHubClients = new HashMap<>(); @@ -49,7 +50,10 @@ public Map getSystemStreamMetadata(Set str Map> ehRuntimeInfos = new HashMap<>(); streamNames.forEach((streamName) -> { if (!_eventHubClients.containsKey(streamName)) { - addEventHubClient(streamName); + _eventHubClients.put(streamName, _eventHubClientFactory + .getEventHubClient(_config.getStreamNamespace(streamName), _config.getStreamEntityPath(streamName), + _config.getStreamSasKeyName(streamName), _config.getStreamSasToken(streamName), _config)); + _eventHubClients.get(streamName).init(); } ehRuntimeInfos.put(streamName, _eventHubClients.get(streamName).getEventHubClient().getRuntimeInformation()); @@ -74,13 +78,6 @@ public Map getSystemStreamMetadata(Set str return requestedMetadata; } - private void addEventHubClient(String streamName) { - String ehNamespace = _config.getStreamNamespace(streamName); - String ehEntityPath = _config.getStreamEntityPath(streamName); - _eventHubClients.put(streamName, new EventHubClientWrapper(null, 0, - ehNamespace, ehEntityPath, _config.getStreamSasKeyName(streamName), _config.getStreamSasToken(streamName))); - } - @Override public void createChangelogStream(String streamName, int numOfPartitions) { throw new UnsupportedOperationException("Event Hubs does not support change log stream."); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java index 6a4114a045..1ab1504449 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java @@ -5,7 +5,10 @@ import com.microsoft.azure.servicebus.ServiceBusException; import com.microsoft.azure.servicebus.StringUtil; import org.apache.samza.SamzaException; +import org.apache.samza.system.eventhub.EventHubClientFactory; import org.apache.samza.system.eventhub.EventHubClientWrapper; +import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.SamzaEventHubClientWrapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -22,16 +25,20 @@ public class EventHubEntityConnection { private final String _sasKeyName; private final String _sasKey; private final String _consumerName; + private final EventHubConfig _config; private final Map _receivers = new TreeMap<>(); private EventHubClientWrapper _ehClientWrapper; + private final EventHubClientFactory _eventHubClientFactory = new EventHubClientFactory(); private boolean _isStarted = false; - EventHubEntityConnection(String namespace, String entityPath, String sasKeyName, String sasKey, String consumerName) { + EventHubEntityConnection(String namespace, String entityPath, String sasKeyName, String sasKey, + String consumerName, EventHubConfig config) { _namespace = namespace; _entityPath = entityPath; _sasKeyName = sasKeyName; _sasKey = sasKey; _consumerName = consumerName; + _config = config; } // add partitions and handlers for this connection. This can be called multiple times @@ -51,9 +58,9 @@ synchronized void connectAndStart() { try { LOG.info(String.format("Starting connection for namespace=%s, entity=%s ", _namespace, _entityPath)); // upon the instantiation of the client, the connection will be established - _ehClientWrapper = - new EventHubClientWrapper(null, 0, - _namespace, _entityPath, _sasKeyName, _sasKey); + _ehClientWrapper = _eventHubClientFactory + .getEventHubClient(_namespace, _entityPath, _sasKeyName, _sasKey, _config); + _ehClientWrapper.init(); for (Map.Entry entry : _offsets.entrySet()) { Integer id = entry.getKey(); String offset = entry.getValue(); @@ -94,7 +101,7 @@ synchronized void stop() { for (PartitionReceiver receiver : _receivers.values()) { receiver.closeSync(); } - _ehClientWrapper.closeSync(); + _ehClientWrapper.close(0); } catch (ServiceBusException e) { throw new SamzaException( String.format("Failed to stop connection for namespace=%s, entity=%s ", _namespace, _entityPath), e); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java index c3bf637c15..80bd699153 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java @@ -1,8 +1,10 @@ package org.apache.samza.system.eventhub.consumer; +import org.apache.samza.system.eventhub.EventHubConfig; + public class EventHubEntityConnectionFactory { EventHubEntityConnection createConnection(String namespace, String entityPath, String sasKeyName, String sasKey, - String consumerName) { - return new EventHubEntityConnection(namespace, entityPath, sasKeyName, sasKey, consumerName); + String consumerName, EventHubConfig config) { + return new EventHubEntityConnection(namespace, entityPath, sasKeyName, sasKey, consumerName, config); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index 09741c16f9..cc22b4f9cd 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -96,7 +96,7 @@ public EventHubSystemConsumer(EventHubConfig config, EventHubEntityConnectionFac _connections.put(stream, connectionFactory.createConnection( config.getStreamNamespace(stream), config.getStreamEntityPath(stream), config.getStreamSasKeyName(stream), config.getStreamSasToken(stream), - config.getStreamConsumerGroup(stream))); + config.getStreamConsumerGroup(stream), config)); _serdes.put(stream, config.getSerde(stream).orElse(null)); }); _eventReadRates = streamList.stream() diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 6543af9e03..70072ff20d 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -1,32 +1,31 @@ package org.apache.samza.system.eventhub.producer; import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.eventhubs.PartitionSender; +import com.microsoft.azure.servicebus.ServiceBusException; import org.apache.samza.SamzaException; -import org.apache.samza.config.Config; import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.serializers.Serde; import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemProducer; +import org.apache.samza.system.eventhub.EventHubClientFactory; import org.apache.samza.system.eventhub.EventHubClientWrapper; import org.apache.samza.system.eventhub.EventHubConfig; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.nio.charset.Charset; import java.time.Duration; import java.time.Instant; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; +import java.util.*; +import java.util.concurrent.*; public class EventHubSystemProducer implements SystemProducer { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemProducer.class.getName()); - public static final String CONFIG_DESTINATION_NUM_PARTITION = "destinationPartitions"; public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; // Metrics recording @@ -46,18 +45,20 @@ public class EventHubSystemProducer implements SystemProducer { private HashMap _sendLatency = new HashMap<>(); private HashMap _sendCallbackLatency = new HashMap<>(); private HashMap _sendErrors = new HashMap<>(); - private static final Duration SHUTDOWN_WAIT_TIME = Duration.ofMinutes(1L); + public static final Duration SHUTDOWN_WAIT_TIME = Duration.ofMinutes(1L); - private final int _destinationPartitions; + private final EventHubClientFactory _eventHubClientFactory = new EventHubClientFactory(); private final EventHubConfig _config; private final String _systemName; private final MetricsRegistry _registry; + private final PartitioningMethod _partitioningMethod; private Throwable _sendExceptionOnCallback; private boolean _isStarted; // Map of the system name to the event hub client. private Map _eventHubClients = new HashMap<>(); + private Map> _streamPartitionSenders = new HashMap<>(); private long _messageId; private Map> _serdes = new HashMap<>(); @@ -69,9 +70,7 @@ public EventHubSystemProducer(String systemName, EventHubConfig config, MetricsR _systemName = systemName; _config = config; _registry = registry; - - // TODO this should be removed when we are able to find the number of partitions. Remove - _destinationPartitions = Integer.parseInt(config.get(CONFIG_DESTINATION_NUM_PARTITION, "-1")); + _partitioningMethod = _config.getPartitioningMethod(); } @Override @@ -102,7 +101,17 @@ public synchronized void start() { @Override public synchronized void stop() { LOG.info("Stopping system producer."); - _eventHubClients.values().forEach(ehClient -> ehClient.closeSync(SHUTDOWN_WAIT_TIME.toMillis())); + _streamPartitionSenders.values().forEach((streamPartitionSender) -> { + List> futures = new ArrayList<>(); + streamPartitionSender.forEach((key, value) -> futures.add(value.close())); + CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); + try { + future.get(SHUTDOWN_WAIT_TIME.toMillis(), TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + LOG.warn("Closing the partition sender failed ", e); + } + }); + _eventHubClients.values().forEach(ehClient -> ehClient.close(SHUTDOWN_WAIT_TIME.toMillis())); _eventHubClients.clear(); } @@ -115,14 +124,13 @@ public synchronized void register(String streamName) { throw new SamzaException(msg); } - String ehNamespace = _config.getStreamNamespace(streamName); - String ehName = _config.getStreamEntityPath(streamName); - - EventHubClientWrapper ehClient = - new EventHubClientWrapper(_config.getPartitioningMethod(), _destinationPartitions, ehNamespace, ehName, - _config.getStreamSasKeyName(streamName), _config.getStreamSasToken(streamName)); + EventHubClientWrapper ehClient = _eventHubClientFactory + .getEventHubClient(_config.getStreamNamespace(streamName), _config.getStreamEntityPath(streamName), + _config.getStreamSasKeyName(streamName), _config.getStreamSasToken(streamName), _config); + ehClient.init(); _eventHubClients.put(streamName, ehClient); + _streamPartitionSenders.put(streamName, new HashMap<>()); _config.getSerde(streamName).ifPresent(x -> _serdes.put(streamName, x)); } @@ -157,7 +165,7 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo Instant startTime = Instant.now(); CompletableFuture sendResult; - sendResult = ehClient.send(eventData, envelope.getPartitionKey()); + sendResult = sendToEventHub(destination, eventData, envelope.getPartitionKey(), ehClient.getEventHubClient()); Instant endTime = Instant.now(); long latencyMs = Duration.between(startTime, endTime).toMillis(); @@ -189,6 +197,58 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo })); } + private CompletableFuture sendToEventHub(String streamName, EventData eventData, Object partitionKey, + EventHubClient eventHubClient) { + if (_partitioningMethod == PartitioningMethod.EVENT_HUB_HASHING) { + return eventHubClient.send(eventData, convertPartitionKeyToString(partitionKey)); + } else if (_partitioningMethod == PartitioningMethod.PARTITION_KEY_AS_PARTITION) { + if (!(partitionKey instanceof Integer)) { + String msg = "Partition key should be of type Integer"; + LOG.error(msg); + throw new SamzaException(msg); + } + + PartitionSender sender = getPartitionSender(streamName, (int) partitionKey, eventHubClient); + return sender.send(eventData); + } else { + throw new SamzaException("Unknown partitioning method " + _partitioningMethod); + } + } + + private String convertPartitionKeyToString(Object partitionKey) { + if (partitionKey instanceof String) { + return (String) partitionKey; + } else if (partitionKey instanceof Integer) { + return String.valueOf(partitionKey); + } else if (partitionKey instanceof byte[]) { + return new String((byte[]) partitionKey, Charset.defaultCharset()); + } else { + throw new SamzaException("Unsupported key type: " + partitionKey.getClass().toString()); + } + } + + private PartitionSender getPartitionSender(String streamName, int partition, EventHubClient eventHubClient) { + Map partitionSenders = _streamPartitionSenders.get(streamName); + if (!partitionSenders.containsKey(partition)) { + try { + int numPartitions = eventHubClient.getRuntimeInformation().get().getPartitionCount(); + PartitionSender partitionSender = + eventHubClient.createPartitionSenderSync(String.valueOf(partition % numPartitions)); + partitionSenders.put(partition, partitionSender); + } catch (ServiceBusException e) { + String msg = "Creation of partition sender failed with exception"; + LOG.error(msg, e); + throw new SamzaException(msg, e); + } catch (InterruptedException | ExecutionException e) { + String msg = "Failed to fetch number of Event Hub partitions for partition sender creation"; + LOG.error(msg, e); + throw new SamzaException(msg, e); + } + } + + return partitionSenders.get(partition); + } + private EventData createEventData(String streamName, OutgoingMessageEnvelope envelope) { Optional> serde = Optional.ofNullable(_serdes.getOrDefault(streamName, null)); byte[] eventValue = (byte[]) envelope.getMessage(); @@ -239,4 +299,9 @@ public void flush(String source) { Collection> getPendingFutures() { return _pendingFutures.values(); } + + public enum PartitioningMethod { + EVENT_HUB_HASHING, + PARTITION_KEY_AS_PARTITION, + } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java index 17a11f36af..efa96aa287 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java @@ -2,6 +2,7 @@ import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; +import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; import java.util.HashMap; @@ -21,7 +22,7 @@ public class MockEventHubConfigFactory { public static final int MIN_EVENTHUB_ENTITY_PARTITION = 2; public static final int MAX_EVENTHUB_ENTITY_PARTITION = 32; - public static Config getEventHubConfig(EventHubClientWrapper.PartitioningMethod partitioningMethod) { + public static Config getEventHubConfig(EventHubSystemProducer.PartitioningMethod partitioningMethod) { HashMap mapConfig = new HashMap<>(); mapConfig.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, SYSTEM_NAME), partitioningMethod.toString()); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java index 6e97b5a53d..37744b5396 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java @@ -5,10 +5,10 @@ import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.system.eventhub.EventHubClientWrapper; import org.apache.samza.system.eventhub.EventHubSystemFactory; import org.apache.samza.system.eventhub.MockEventHubConfigFactory; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; +import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; import org.junit.Test; import java.util.HashMap; @@ -24,7 +24,7 @@ public class TestEventHubSystemAdmin { public void testOffsetComparison() { EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); EventHubSystemAdmin eventHubSystemAdmin = (EventHubSystemAdmin) eventHubSystemFactory.getAdmin(SYSTEM_NAME, - MockEventHubConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + MockEventHubConfigFactory.getEventHubConfig(EventHubSystemProducer.PartitioningMethod.EVENT_HUB_HASHING)); Assert.assertEquals(-1, eventHubSystemAdmin.offsetComparator("100", "200").intValue()); Assert.assertEquals(0, eventHubSystemAdmin.offsetComparator("150", "150").intValue()); Assert.assertEquals(1, eventHubSystemAdmin.offsetComparator("200", "100").intValue()); @@ -43,7 +43,7 @@ public void testOffsetComparison() { public void testGetNextOffset() { EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); SystemAdmin eventHubSystemAdmin = eventHubSystemFactory.getAdmin(SYSTEM_NAME, - MockEventHubConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + MockEventHubConfigFactory.getEventHubConfig(EventHubSystemProducer.PartitioningMethod.EVENT_HUB_HASHING)); Map offsets = new HashMap<>(); SystemStreamPartition ssp0 = new SystemStreamPartition(SYSTEM_NAME, STREAM_NAME1, new Partition(0)); SystemStreamPartition ssp1 = new SystemStreamPartition(SYSTEM_NAME, STREAM_NAME1, new Partition(1)); @@ -63,7 +63,7 @@ public void testGetNextOffset() { public void testGetStreamMetadata() { EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); SystemAdmin eventHubSystemAdmin = eventHubSystemFactory.getAdmin(SYSTEM_NAME, - MockEventHubConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING)); + MockEventHubConfigFactory.getEventHubConfig(EventHubSystemProducer.PartitioningMethod.EVENT_HUB_HASHING)); Set streams = new HashSet<>(); streams.add(STREAM_NAME1); streams.add(STREAM_NAME2); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java index de84c15041..49763f2512 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java @@ -2,6 +2,8 @@ import com.microsoft.azure.eventhubs.EventData; import com.microsoft.azure.eventhubs.PartitionReceiveHandler; +import org.apache.samza.system.eventhub.EventHubClientFactory; +import org.apache.samza.system.eventhub.EventHubConfig; import java.util.List; import java.util.Map; @@ -16,7 +18,7 @@ class MockEventHubEntityConnectionFactory extends EventHubEntityConnectionFactor @Override EventHubEntityConnection createConnection(String namespace, String entityPath, String sasKeyName, String sasKey, - String consumerName) { + String consumerName, EventHubConfig eventHubConfig) { return new MockEventHubEntityConnection(entityPath, _eventData.get(entityPath)); } @@ -24,7 +26,7 @@ private class MockEventHubEntityConnection extends EventHubEntityConnection { private final Map> _eventData; MockEventHubEntityConnection(String entity, Map> eventData) { - super(null, entity, null, null, null); + super(null, entity, null, null, null, null); assert eventData != null : "No event data found for entity:" + entity; _eventData = eventData; } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java index 4997c9673f..7cbef24712 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -9,9 +9,7 @@ import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemProducer; import org.apache.samza.system.SystemStream; -import org.apache.samza.system.eventhub.EventHubClientWrapper; -import org.apache.samza.system.eventhub.EventHubSystemFactory; -import org.apache.samza.system.eventhub.MockEventHubConfigFactory; +import org.apache.samza.system.eventhub.*; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.apache.samza.util.NoOpMetricsRegistry; import org.junit.Assert; @@ -38,7 +36,7 @@ public void testSystemFactoryCreateAndStartProducer() { } private Config createEventHubConfig() { - return MockEventHubConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING); + return MockEventHubConfigFactory.getEventHubConfig(EventHubSystemProducer.PartitioningMethod.EVENT_HUB_HASHING); } @Test @@ -76,9 +74,11 @@ public void testSend() { @Test public void testReceive() throws ServiceBusException { - EventHubClientWrapper wrapper = - new EventHubClientWrapper(EventHubClientWrapper.PartitioningMethod.EVENT_HUB_HASHING, 8, - EVENTHUB_NAMESPACE, EVENTHUB_ENTITY1, EVENTHUB_KEY_NAME, EVENTHUB_KEY); + EventHubClientFactory clientFactory = new EventHubClientFactory(); + EventHubClientWrapper wrapper = clientFactory + .getEventHubClient(EVENTHUB_NAMESPACE, EVENTHUB_ENTITY1, EVENTHUB_KEY_NAME, EVENTHUB_KEY, + new EventHubConfig(createEventHubConfig(), SYSTEM_NAME)); + wrapper.init(); EventHubClient client = wrapper.getEventHubClient(); PartitionReceiver receiver = client.createReceiverSync(EventHubClient.DEFAULT_CONSUMER_GROUP_NAME, "0", @@ -103,7 +103,7 @@ private void receiveMessages(PartitionReceiver receiver, int numMessages) throws @Test public void testSendToSpecificPartition() { - Config eventHubConfig = MockEventHubConfigFactory.getEventHubConfig(EventHubClientWrapper.PartitioningMethod.PARTITION_KEY_AS_PARTITION); + Config eventHubConfig = MockEventHubConfigFactory.getEventHubConfig(EventHubSystemProducer.PartitioningMethod.PARTITION_KEY_AS_PARTITION); EventHubSystemFactory systemFactory = new EventHubSystemFactory(); SystemProducer systemProducer = systemFactory.getProducer(SYSTEM_NAME, eventHubConfig, new NoOpMetricsRegistry()); From f2d3aa3564803512d00b5159f2bd4ef56603a7fa Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Fri, 29 Sep 2017 14:26:58 -0700 Subject: [PATCH 08/34] fix code style, added timeout configs --- .../system/eventhub/EventDataWrapper.java | 33 ++- .../eventhub/EventHubClientFactory.java | 19 ++ .../eventhub/EventHubClientWrapper.java | 19 ++ .../samza/system/eventhub/EventHubConfig.java | 91 +++++-- .../eventhub/EventHubSystemFactory.java | 19 ++ .../eventhub/SamzaEventHubClientWrapper.java | 51 ++-- .../eventhub/admin/EventHubSystemAdmin.java | 87 ++++--- .../consumer/EventHubEntityConnection.java | 110 +++++---- .../EventHubEntityConnectionFactory.java | 19 ++ .../consumer/EventHubSystemConsumer.java | 163 ++++++------ .../eventhub/metrics/SamzaHistogram.java | 43 +++- .../producer/EventHubSystemProducer.java | 233 ++++++++++-------- .../samza/system/eventhub/MockEventData.java | 27 +- .../eventhub/MockEventHubConfigFactory.java | 19 ++ .../system/eventhub/TestMetricsRegistry.java | 41 ++- .../admin/TestEventHubSystemAdmin.java | 27 +- .../MockEventHubEntityConnectionFactory.java | 44 +++- .../consumer/TestEventHubSystemConsumer.java | 23 +- .../producer/TestEventHubSystemProducer.java | 19 ++ 19 files changed, 744 insertions(+), 343 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java index 5fc95441d6..f2daf96783 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub; import com.microsoft.azure.eventhubs.EventData; @@ -6,16 +25,16 @@ * Simpler wrapper of {@link EventData} events with the decrypted payload */ public class EventDataWrapper { - private final EventData _eventData; - private final byte[] _body; + private final EventData eventData; + private final byte[] body; public EventDataWrapper(EventData eventData, byte[] body) { - _eventData = eventData; - _body = body; + this.eventData = eventData; + this.body = body; } public EventData getEventData() { - return _eventData; + return eventData; } /** @@ -23,12 +42,12 @@ public EventData getEventData() { * just returns the body of the message. */ public byte[] getDecryptedBody() { - return _body; + return body; } @Override public String toString() { - return "EventDataWrapper: body: " + (new String(_body)) + ", EventData " + _eventData; + return "EventDataWrapper: body: " + (new String(body)) + ", EventData " + eventData; } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientFactory.java index 50cd54a14d..4a1d6205d4 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientFactory.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub; public class EventHubClientFactory { diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java index 903abc101a..12c98aabd6 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub; import com.microsoft.azure.eventhubs.EventHubClient; diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java index 0f76ad888d..7a602235f3 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub; import com.microsoft.azure.eventhubs.EventHubClient; @@ -7,6 +26,7 @@ import org.apache.samza.serializers.SerdeFactory; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; +import java.time.Duration; import java.util.List; import java.util.Map; import java.util.Objects; @@ -37,16 +57,19 @@ public class EventHubConfig extends MapConfig { .PartitioningMethod.EVENT_HUB_HASHING.name(); public static final String CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = "systems.%s.eventhubs.send.key"; - public static final String DEFAULT_CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = Boolean.toString(false); + public static final Boolean DEFAULT_CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = false; + + public static final String CONFIG_CONNECTION_SHUTDOWN_TIMEOUT_MILLIS = "systems.%s.eventhubs.shutdown.timeout"; + public static final long DEFAULT_CONFIG_CONNECTION_SHUTDOWN_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); - public static final String CONFIG_GET_RUNTIME_INFO_TIMEOUT_MILLIS = "systems.%s.eventhubs.getruntime.timeout"; - public static final String DEFAULT_CONFIG_GET_RUNTIME_INFO_TIMEOUT_MILLIS = Integer.toString(1000); + public static final String CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS = "systems.%s.eventhubs.runtime.info.timeout"; + public static final long DEFAULT_CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); - private final String _system; + private final String systemName; public EventHubConfig(Map config, String systemName) { super(config); - _system = systemName; + this.systemName = systemName; } /** @@ -56,7 +79,7 @@ public EventHubConfig(Map config, String systemName) { * @return list of stream names */ public List getStreamList() { - return getList(String.format(CONFIG_STREAM_LIST, _system)); + return getList(String.format(CONFIG_STREAM_LIST, systemName)); } /** @@ -66,7 +89,7 @@ public List getStreamList() { * @return EventHubs namespace */ public String getStreamNamespace(String streamName) { - return get(String.format(CONFIG_STREAM_NAMESPACE, _system, streamName)); + return get(String.format(CONFIG_STREAM_NAMESPACE, systemName, streamName)); } /** @@ -76,7 +99,7 @@ public String getStreamNamespace(String streamName) { * @return EventHubs entity path */ public String getStreamEntityPath(String streamName) { - return get(String.format(CONFIG_STREAM_ENTITYPATH, _system, streamName)); + return get(String.format(CONFIG_STREAM_ENTITYPATH, systemName, streamName)); } /** @@ -86,7 +109,7 @@ public String getStreamEntityPath(String streamName) { * @return EventHubs SAS key name */ public String getStreamSasKeyName(String streamName) { - return get(String.format(CONFIG_STREAM_SAS_KEY_NAME, _system, streamName)); + return get(String.format(CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName)); } /** @@ -96,15 +119,15 @@ public String getStreamSasKeyName(String streamName) { * @return EventHubs SAS token */ public String getStreamSasToken(String streamName) { - return get(String.format(CONFIG_STREAM_SAS_TOKEN, _system, streamName)); + return get(String.format(CONFIG_STREAM_SAS_TOKEN, systemName, streamName)); } public Optional> getSerde(String streamName) { Serde serde = null; - String serdeFactoryClassName = this.get(String.format(CONFIG_STREAM_SERDE_FACTORY, _system, streamName)); + String serdeFactoryClassName = this.get(String.format(CONFIG_STREAM_SERDE_FACTORY, systemName, streamName)); if (!StringUtils.isEmpty(serdeFactoryClassName)) { SerdeFactory factory = EventHubSystemFactory.getSerdeFactory(serdeFactoryClassName); - serde = factory.getSerde(streamName, this.subset(String.format(CONFIG_STREAM_SERDE_PREFIX, _system, streamName))); + serde = factory.getSerde(streamName, this.subset(String.format(CONFIG_STREAM_SERDE_PREFIX, systemName, streamName))); } return Optional.ofNullable(serde); } @@ -116,7 +139,7 @@ public Optional> getSerde(String streamName) { * @return EventHubs consumer group */ public String getStreamConsumerGroup(String streamName) { - return get(String.format(CONFIG_STREAM_CONSUMER_GROUP, _system, streamName), DEFAULT_CONFIG_STREAM_CONSUMER_GROUP); + return get(String.format(CONFIG_STREAM_CONSUMER_GROUP, systemName, streamName), DEFAULT_CONFIG_STREAM_CONSUMER_GROUP); } /** @@ -126,18 +149,18 @@ public String getStreamConsumerGroup(String streamName) { * @return Starting position when no checkpoints */ public StartPosition getStartPosition(String streamName) { - String startPositionStr = get(String.format(CONFIG_STREAM_CONSUMER_START_POSITION, _system, streamName), + String startPositionStr = get(String.format(CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION); return StartPosition.valueOf(startPositionStr.toUpperCase()); } /** - * Get the partition method of the system. By default partitioning is handed by EventHub. + * Get the partition method of the systemName. By default partitioning is handed by EventHub. * * @return The method the producer should use to partition the outgoing data */ public EventHubSystemProducer.PartitioningMethod getPartitioningMethod() { - String partitioningMethod = get(String.format(CONFIG_PRODUCER_PARTITION_METHOD, _system), + String partitioningMethod = get(String.format(CONFIG_PRODUCER_PARTITION_METHOD, systemName), DEFAULT_CONFIG_PRODUCER_PARTITION_METHOD); return EventHubSystemProducer.PartitioningMethod.valueOf(partitioningMethod); @@ -149,19 +172,47 @@ public EventHubSystemProducer.PartitioningMethod getPartitioningMethod() { * @return Boolean, is send key included */ public Boolean getSendKeyInEventProperties() { - String isSendKeyIncluded = get(String.format(CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, _system), - DEFAULT_CONFIG_SEND_KEY_IN_EVENT_PROPERTIES); + String isSendKeyIncluded = get(String.format(CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, systemName)); + if (isSendKeyIncluded == null) { + return DEFAULT_CONFIG_SEND_KEY_IN_EVENT_PROPERTIES; + } return Boolean.valueOf(isSendKeyIncluded); } + /** + * Get the timeout for terminating the connection to EventHub client + * + * @return long, timeout in millis for the shutdown of EventHub Connection + */ + public long getShutdownWaitTimeMS() { + String timeoutStr = get(String.format(CONFIG_CONNECTION_SHUTDOWN_TIMEOUT_MILLIS, systemName)); + if (timeoutStr == null) { + return DEFAULT_CONFIG_CONNECTION_SHUTDOWN_TIMEOUT_MILLIS; + } + return Long.valueOf(timeoutStr); + } + + /** + * Get the timeout for the getRuntimeInfo request to EventHub client + * + * @return long, timeout in millis for fetching RuntimeInfo + */ + public long getRuntimeInfoWaitTimeMS() { + String timeoutStr = get(String.format(CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS, systemName)); + if (timeoutStr == null) { + return DEFAULT_CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS; + } + return Long.valueOf(timeoutStr); + } + @Override public int hashCode() { - return Objects.hash(super.hashCode(), _system); + return Objects.hash(super.hashCode(), systemName); } @Override public boolean equals(Object obj) { - return super.equals(obj) && _system.equals(((EventHubConfig) obj)._system); + return super.equals(obj) && systemName.equals(((EventHubConfig) obj).systemName); } public enum StartPosition { diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java index a77237f2d0..8901feb4a9 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub; import org.apache.samza.SamzaException; diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientWrapper.java index 6ed4c15d6e..4195fc573a 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientWrapper.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientWrapper.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub; import com.microsoft.azure.eventhubs.EventHubClient; @@ -18,52 +37,52 @@ public class SamzaEventHubClientWrapper implements EventHubClientWrapper { private static final Logger LOG = LoggerFactory.getLogger(SamzaEventHubClientWrapper.class.getName()); private static final String EVENTHUB_REMOTE_HOST_FORMAT = "%s.servicebus.windows.net"; - private EventHubClient _eventHubClient; + private EventHubClient eventHubClient; - private final String _eventHubNamespace; - private final String _entityPath; - private final String _sasKeyName; - private final String _sasKey; + private final String eventHubNamespace; + private final String entityPath; + private final String sasKeyName; + private final String sasKey; public SamzaEventHubClientWrapper(String eventHubNamespace, String entityPath, String sasKeyName, String sasKey) { - _eventHubNamespace = eventHubNamespace; - _entityPath = entityPath; - _sasKeyName = sasKeyName; - _sasKey = sasKey; + this.eventHubNamespace = eventHubNamespace; + this.entityPath = entityPath; + this.sasKeyName = sasKeyName; + this.sasKey = sasKey; } public void init() { - String remoteHost = String.format(EVENTHUB_REMOTE_HOST_FORMAT, _eventHubNamespace); + String remoteHost = String.format(EVENTHUB_REMOTE_HOST_FORMAT, eventHubNamespace); try { ConnectionStringBuilder connectionStringBuilder = - new ConnectionStringBuilder(_eventHubNamespace, _entityPath, _sasKeyName, _sasKey); + new ConnectionStringBuilder(eventHubNamespace, entityPath, sasKeyName, sasKey); - _eventHubClient = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString()); + eventHubClient = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString()); } catch (IOException ioe) { throw new IllegalStateException( "Failed to connect to remote host " + remoteHost + ":" + ClientConstants.AMQPS_PORT, ioe); } catch (ServiceBusException e) { String msg = String.format("Creation of event hub client failed for eventHub %s %s %s %s with exception", - _entityPath, _eventHubNamespace, _sasKeyName, _sasKey); + entityPath, eventHubNamespace, sasKeyName, sasKey); LOG.error(msg, e); throw new SamzaException(msg, e); } } public EventHubClient getEventHubClient() { - return _eventHubClient; + return eventHubClient; } public void close(long timeoutMS) { if (timeoutMS <= 0) { try { - _eventHubClient.closeSync(); + eventHubClient.closeSync(); } catch (ServiceBusException e) { LOG.warn("Closing the event hub client failed ", e); } } else { - CompletableFuture future = _eventHubClient.close(); + CompletableFuture future = eventHubClient.close(); try { future.get(timeoutMS, TimeUnit.MILLISECONDS); } catch (ExecutionException | InterruptedException | TimeoutException e) { diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index 370018a4ae..9394cd248b 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub.admin; import com.microsoft.azure.eventhubs.EventHubRuntimeInformation; @@ -17,17 +36,20 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public class EventHubSystemAdmin implements SystemAdmin { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemAdmin.class); - private final EventHubClientFactory _eventHubClientFactory = new EventHubClientFactory(); - private String _systemName; - private EventHubConfig _config; - private Map _eventHubClients = new HashMap<>(); + private final EventHubClientFactory eventHubClientFactory = new EventHubClientFactory(); + private String systemName; + private EventHubConfig eventHubConfig; + private Map eventHubClients = new HashMap<>(); public EventHubSystemAdmin(String systemName, EventHubConfig config) { - _systemName = systemName; - _config = config; + this.systemName = systemName; + eventHubConfig = config; } private static String getNextOffset(String currentOffset) { @@ -49,32 +71,37 @@ public Map getSystemStreamMetadata(Set str Map requestedMetadata = new HashMap<>(); Map> ehRuntimeInfos = new HashMap<>(); streamNames.forEach((streamName) -> { - if (!_eventHubClients.containsKey(streamName)) { - _eventHubClients.put(streamName, _eventHubClientFactory - .getEventHubClient(_config.getStreamNamespace(streamName), _config.getStreamEntityPath(streamName), - _config.getStreamSasKeyName(streamName), _config.getStreamSasToken(streamName), _config)); - _eventHubClients.get(streamName).init(); - } - ehRuntimeInfos.put(streamName, - _eventHubClients.get(streamName).getEventHubClient().getRuntimeInformation()); - }); + if (!eventHubClients.containsKey(streamName)) { + eventHubClients.put(streamName, eventHubClientFactory + .getEventHubClient(eventHubConfig.getStreamNamespace(streamName), eventHubConfig.getStreamEntityPath(streamName), + eventHubConfig.getStreamSasKeyName(streamName), eventHubConfig.getStreamSasToken(streamName), eventHubConfig)); + eventHubClients.get(streamName).init(); + } + ehRuntimeInfos.put(streamName, + eventHubClients.get(streamName).getEventHubClient().getRuntimeInformation()); + }); ehRuntimeInfos.forEach((streamName, ehRuntimeInfo) -> { - try { - EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(); // TODO: timeout - Map sspMetadataMap = new HashMap<>(); - for (String partition : ehInfo.getPartitionIds()) { //TODO getPartitionRuntimeInformation - sspMetadataMap.put(new Partition(Integer.parseInt(partition)), - new SystemStreamMetadata.SystemStreamPartitionMetadata(EventHubSystemConsumer.START_OF_STREAM, - EventHubSystemConsumer.END_OF_STREAM, EventHubSystemConsumer.END_OF_STREAM)); + try { + EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(eventHubConfig.getRuntimeInfoWaitTimeMS(), TimeUnit.MILLISECONDS); + Map sspMetadataMap = new HashMap<>(); + for (String partition : ehInfo.getPartitionIds()) { + sspMetadataMap.put(new Partition(Integer.parseInt(partition)), + new SystemStreamMetadata.SystemStreamPartitionMetadata(EventHubSystemConsumer.START_OF_STREAM, + EventHubSystemConsumer.END_OF_STREAM, EventHubSystemConsumer.END_OF_STREAM)); + } + requestedMetadata.put(streamName, new SystemStreamMetadata(streamName, sspMetadataMap)); + } catch (InterruptedException | ExecutionException e) { + String msg = String.format("Error while fetching EventHubRuntimeInfo for System:%s, Stream:%s", + systemName, streamName); + LOG.error(msg, e); + throw new SamzaException(msg); + } catch (TimeoutException e) { + String msg = String.format("Timed out while fetching EventHubRuntimeInfo for System:%s, Stream:%s", + systemName, streamName); + LOG.error(msg, e); + throw new SamzaException(msg); } - requestedMetadata.put(streamName, new SystemStreamMetadata(streamName, sspMetadataMap)); - } catch (Exception e) { - String msg = String.format("Error while fetching EventHubRuntimeInfo for System:%s, Stream:%s", - _systemName, streamName); - LOG.error(msg); - throw new SamzaException(msg); - } - }); + }); return requestedMetadata; } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java index 1ab1504449..08cdc79c18 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub.consumer; import com.microsoft.azure.eventhubs.PartitionReceiveHandler; @@ -8,7 +27,6 @@ import org.apache.samza.system.eventhub.EventHubClientFactory; import org.apache.samza.system.eventhub.EventHubClientWrapper; import org.apache.samza.system.eventhub.EventHubConfig; -import org.apache.samza.system.eventhub.SamzaEventHubClientWrapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -18,97 +36,97 @@ public class EventHubEntityConnection { private static final Logger LOG = LoggerFactory.getLogger(EventHubEntityConnection.class); - final Map _offsets = new TreeMap<>(); - final Map _handlers = new TreeMap<>(); - private final String _namespace; - private final String _entityPath; - private final String _sasKeyName; - private final String _sasKey; - private final String _consumerName; - private final EventHubConfig _config; - private final Map _receivers = new TreeMap<>(); - private EventHubClientWrapper _ehClientWrapper; - private final EventHubClientFactory _eventHubClientFactory = new EventHubClientFactory(); - private boolean _isStarted = false; + final Map offsets = new TreeMap<>(); + final Map handlers = new TreeMap<>(); + private final String namespace; + private final String entityPath; + private final String sasKeyName; + private final String sasKey; + private final String consumerName; + private final EventHubConfig config; + private final Map receivers = new TreeMap<>(); + private EventHubClientWrapper ehClientWrapper; + private final EventHubClientFactory eventHubClientFactory = new EventHubClientFactory(); + private boolean isStarted = false; EventHubEntityConnection(String namespace, String entityPath, String sasKeyName, String sasKey, String consumerName, EventHubConfig config) { - _namespace = namespace; - _entityPath = entityPath; - _sasKeyName = sasKeyName; - _sasKey = sasKey; - _consumerName = consumerName; - _config = config; + this.namespace = namespace; + this.entityPath = entityPath; + this.sasKeyName = sasKeyName; + this.sasKey = sasKey; + this.consumerName = consumerName; + this.config = config; } // add partitions and handlers for this connection. This can be called multiple times // for multiple partitions, but needs to be called before connectAndStart() synchronized void addPartition(int partitionId, String offset, PartitionReceiveHandler handler) { - if (_isStarted) { + if (isStarted) { LOG.warn("Trying to add partition when the connection has already started."); return; } - _offsets.put(partitionId, offset); - _handlers.put(partitionId, handler); + offsets.put(partitionId, offset); + handlers.put(partitionId, handler); } // establish the connection and start consuming events synchronized void connectAndStart() { - _isStarted = true; + isStarted = true; try { - LOG.info(String.format("Starting connection for namespace=%s, entity=%s ", _namespace, _entityPath)); + LOG.info(String.format("Starting connection for namespace=%s, entity=%s ", namespace, entityPath)); // upon the instantiation of the client, the connection will be established - _ehClientWrapper = _eventHubClientFactory - .getEventHubClient(_namespace, _entityPath, _sasKeyName, _sasKey, _config); - _ehClientWrapper.init(); - for (Map.Entry entry : _offsets.entrySet()) { + ehClientWrapper = eventHubClientFactory + .getEventHubClient(namespace, entityPath, sasKeyName, sasKey, config); + ehClientWrapper.init(); + for (Map.Entry entry : offsets.entrySet()) { Integer id = entry.getKey(); String offset = entry.getValue(); try { PartitionReceiver receiver; if (StringUtil.isNullOrWhiteSpace(offset)) { throw new SamzaException( - String.format("Invalid offset %s namespace=%s, entity=%s", offset, _namespace, _entityPath)); + String.format("Invalid offset %s namespace=%s, entity=%s", offset, namespace, entityPath)); } if (offset.equals(EventHubSystemConsumer.END_OF_STREAM)) { - receiver = _ehClientWrapper.getEventHubClient() - .createReceiverSync(_consumerName, id.toString(), Instant.now()); + receiver = ehClientWrapper.getEventHubClient() + .createReceiverSync(consumerName, id.toString(), Instant.now()); } else { - receiver = _ehClientWrapper.getEventHubClient() - .createReceiverSync(_consumerName, id.toString(), offset, + receiver = ehClientWrapper.getEventHubClient() + .createReceiverSync(consumerName, id.toString(), offset, !offset.equals(EventHubSystemConsumer.START_OF_STREAM)); } - receiver.setReceiveHandler(_handlers.get(id)); - _receivers.put(id, receiver); + receiver.setReceiveHandler(handlers.get(id)); + receivers.put(id, receiver); } catch (Exception e) { throw new SamzaException( String.format("Failed to create receiver for EventHubs: namespace=%s, entity=%s, partitionId=%d", - _namespace, _entityPath, id), e); + namespace, entityPath, id), e); } } } catch (Exception e) { throw new SamzaException( String.format("Failed to create connection to EventHubs: namespace=%s, entity=%s", - _namespace, _entityPath), + namespace, entityPath), e); } - LOG.info(String.format("Connection successfully started for namespace=%s, entity=%s ", _namespace, _entityPath)); + LOG.info(String.format("Connection successfully started for namespace=%s, entity=%s ", namespace, entityPath)); } synchronized void stop() { - LOG.info(String.format("Stopping connection for namespace=%s, entity=%s ", _namespace, _entityPath)); + LOG.info(String.format("Stopping connection for namespace=%s, entity=%s ", namespace, entityPath)); try { - for (PartitionReceiver receiver : _receivers.values()) { + for (PartitionReceiver receiver : receivers.values()) { receiver.closeSync(); } - _ehClientWrapper.close(0); + ehClientWrapper.close(0); } catch (ServiceBusException e) { throw new SamzaException( - String.format("Failed to stop connection for namespace=%s, entity=%s ", _namespace, _entityPath), e); + String.format("Failed to stop connection for namespace=%s, entity=%s ", namespace, entityPath), e); } - _isStarted = false; - _offsets.clear(); - _handlers.clear(); - LOG.info(String.format("Connection for namespace=%s, entity=%s stopped", _namespace, _entityPath)); + isStarted = false; + offsets.clear(); + handlers.clear(); + LOG.info(String.format("Connection for namespace=%s, entity=%s stopped", namespace, entityPath)); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java index 80bd699153..900377fef3 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub.consumer; import org.apache.samza.system.eventhub.EventHubConfig; diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index cc22b4f9cd..b4b4edb455 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub.consumer; import com.microsoft.azure.eventhubs.EventData; @@ -74,47 +93,47 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemConsumer.class); private static final int MAX_EVENT_COUNT_PER_PARTITION_POLL = 50; private static final int BLOCKING_QUEUE_SIZE = 100; - private static Counter _aggEventReadRate = null; - private static Counter _aggEventByteReadRate = null; - private static SamzaHistogram _aggReadLatency = null; - private static Counter _aggReadErrors = null; - private final Map _connections = new HashMap<>(); - private final Map> _serdes = new HashMap<>(); - private final EventHubConfig _config; - private Map _eventReadRates; - private Map _eventByteReadRates; - private Map _readLatencies; - private Map _readErrors; + private static Counter aggEventReadRate = null; + private static Counter aggEventByteReadRate = null; + private static SamzaHistogram aggReadLatency = null; + private static Counter aggReadErrors = null; + private final Map connections = new HashMap<>(); + private final Map> serdes = new HashMap<>(); + private final EventHubConfig config; + private Map eventReadRates; + private Map eventByteReadRates; + private Map readLatencies; + private Map readErrors; public EventHubSystemConsumer(EventHubConfig config, EventHubEntityConnectionFactory connectionFactory, MetricsRegistry registry) { super(registry, System::currentTimeMillis); - _config = config; + this.config = config; List streamList = config.getStreamList(); streamList.forEach(stream -> { - _connections.put(stream, connectionFactory.createConnection( - config.getStreamNamespace(stream), config.getStreamEntityPath(stream), - config.getStreamSasKeyName(stream), config.getStreamSasToken(stream), - config.getStreamConsumerGroup(stream), config)); - _serdes.put(stream, config.getSerde(stream).orElse(null)); - }); - _eventReadRates = streamList.stream() + connections.put(stream, connectionFactory.createConnection( + config.getStreamNamespace(stream), config.getStreamEntityPath(stream), + config.getStreamSasKeyName(stream), config.getStreamSasToken(stream), + config.getStreamConsumerGroup(stream), config)); + serdes.put(stream, config.getSerde(stream).orElse(null)); + }); + eventReadRates = streamList.stream() .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_READ_RATE))); - _eventByteReadRates = streamList.stream() + eventByteReadRates = streamList.stream() .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_BYTE_READ_RATE))); - _readLatencies = streamList.stream() + readLatencies = streamList.stream() .collect(Collectors.toMap(Function.identity(), x -> new SamzaHistogram(registry, x, READ_LATENCY))); - _readErrors = + readErrors = streamList.stream().collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, READ_ERRORS))); // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. synchronized (AGGREGATE) { - if (_aggEventReadRate == null) { - _aggEventReadRate = registry.newCounter(AGGREGATE, EVENT_READ_RATE); - _aggEventByteReadRate = registry.newCounter(AGGREGATE, EVENT_BYTE_READ_RATE); - _aggReadLatency = new SamzaHistogram(registry, AGGREGATE, READ_LATENCY); - _aggReadErrors = registry.newCounter(AGGREGATE, READ_ERRORS); + if (aggEventReadRate == null) { + aggEventReadRate = registry.newCounter(AGGREGATE, EVENT_READ_RATE); + aggEventByteReadRate = registry.newCounter(AGGREGATE, EVENT_BYTE_READ_RATE); + aggReadLatency = new SamzaHistogram(registry, AGGREGATE, READ_LATENCY); + aggReadErrors = registry.newCounter(AGGREGATE, READ_ERRORS); } } } @@ -123,13 +142,13 @@ public EventHubSystemConsumer(EventHubConfig config, EventHubEntityConnectionFac public void register(SystemStreamPartition systemStreamPartition, String offset) { super.register(systemStreamPartition, offset); String stream = systemStreamPartition.getStream(); - EventHubEntityConnection connection = _connections.get(stream); + EventHubEntityConnection connection = connections.get(stream); if (connection == null) { throw new SamzaException("No EventHub connection for " + stream); } if (StringUtil.isNullOrWhiteSpace(offset)) { - switch (_config.getStartPosition(systemStreamPartition.getStream())) { + switch (config.getStartPosition(systemStreamPartition.getStream())) { case EARLIEST: offset = START_OF_STREAM; break; @@ -138,43 +157,43 @@ public void register(SystemStreamPartition systemStreamPartition, String offset) break; default: throw new SamzaException( - "Unknown starting position config " + _config.getStartPosition(systemStreamPartition.getStream())); + "Unknown starting position config " + config.getStartPosition(systemStreamPartition.getStream())); } } connection.addPartition(systemStreamPartition.getPartition().getPartitionId(), offset, - new PartitionReceiverHandlerImpl(systemStreamPartition, _eventReadRates.get(stream), - _eventByteReadRates.get(stream), _readLatencies.get(stream), _readErrors.get(stream), - _serdes.get(stream))); + new PartitionReceiverHandlerImpl(systemStreamPartition, eventReadRates.get(stream), + eventByteReadRates.get(stream), readLatencies.get(stream), readErrors.get(stream), + serdes.get(stream))); } @Override public void start() { - _connections.values().forEach(EventHubEntityConnection::connectAndStart); + connections.values().forEach(EventHubEntityConnection::connectAndStart); } @Override public void stop() { - _connections.values().forEach(EventHubEntityConnection::stop); + connections.values().forEach(EventHubEntityConnection::stop); } private class PartitionReceiverHandlerImpl extends PartitionReceiveHandler { - private final Counter _eventReadRate; - private final Counter _eventByteReadRate; - private final SamzaHistogram _readLatency; - private final Counter _errors; - private final Serde _serde; - SystemStreamPartition _ssp; + private final Counter eventReadRate; + private final Counter eventByteReadRate; + private final SamzaHistogram readLatency; + private final Counter errors; + private final Serde serde; + SystemStreamPartition ssp; PartitionReceiverHandlerImpl(SystemStreamPartition ssp, Counter eventReadRate, Counter eventByteReadRate, SamzaHistogram readLatency, Counter readErrors, Serde serde) { super(MAX_EVENT_COUNT_PER_PARTITION_POLL); - _ssp = ssp; - _eventReadRate = eventReadRate; - _eventByteReadRate = eventByteReadRate; - _readLatency = readLatency; - _errors = readErrors; - _serde = serde; + this.ssp = ssp; + this.eventReadRate = eventReadRate; + this.eventByteReadRate = eventByteReadRate; + this.readLatency = readLatency; + errors = readErrors; + this.serde = serde; } @Override @@ -182,41 +201,41 @@ public void onReceive(Iterable events) { if (events != null) { events.forEach(event -> { - byte[] decryptedBody = event.getBody(); - if (_serde != null) { - decryptedBody = _serde.fromBytes(decryptedBody); - } - EventDataWrapper wrappedEvent = new EventDataWrapper(event, decryptedBody); - try { - updateMetrics(event); - // note that the partition key can be null - put(_ssp, new IncomingMessageEnvelope(_ssp, event.getSystemProperties().getOffset(), - event.getSystemProperties().getPartitionKey(), wrappedEvent)); - } catch (Exception e) { - String msg = String.format("Exception while adding the event from ssp %s to dispatch queue.", _ssp); - LOG.error(msg, e); - throw new SamzaException(msg, e); - } - }); + byte[] decryptedBody = event.getBody(); + if (serde != null) { + decryptedBody = serde.fromBytes(decryptedBody); + } + EventDataWrapper wrappedEvent = new EventDataWrapper(event, decryptedBody); + try { + updateMetrics(event); + // note that the partition key can be null + put(ssp, new IncomingMessageEnvelope(ssp, event.getSystemProperties().getOffset(), + event.getSystemProperties().getPartitionKey(), wrappedEvent)); + } catch (Exception e) { + String msg = String.format("Exception while adding the event from ssp %s to dispatch queue.", ssp); + LOG.error(msg, e); + throw new SamzaException(msg, e); + } + }); } } private void updateMetrics(EventData event) { - _eventReadRate.inc(); - _aggEventReadRate.inc(); - _eventByteReadRate.inc(event.getBodyLength()); - _aggEventByteReadRate.inc(event.getBodyLength()); + eventReadRate.inc(); + aggEventReadRate.inc(); + eventByteReadRate.inc(event.getBodyLength()); + aggEventByteReadRate.inc(event.getBodyLength()); long latencyMs = Duration.between(Instant.now(), event.getSystemProperties().getEnqueuedTime()).toMillis(); - _readLatency.update(latencyMs); - _aggReadLatency.update(latencyMs); + readLatency.update(latencyMs); + aggReadLatency.update(latencyMs); } @Override public void onError(Throwable throwable) { // TODO error handling - _errors.inc(); - _aggReadErrors.inc(); - LOG.error(String.format("Received error from event hub connection (ssp=%s): ", _ssp), throwable); + errors.inc(); + aggReadErrors.inc(); + LOG.error(String.format("Received error from event hub connection (ssp=%s): ", ssp), throwable); } } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java index 1f1bd8ce63..631ed0429a 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub.metrics; import com.codahale.metrics.ExponentiallyDecayingReservoir; @@ -15,28 +34,28 @@ public class SamzaHistogram { private static final List DEFAULT_HISTOGRAM_PERCENTILES = Arrays.asList(50D, 99D); - private final MetricsRegistry _registry; - private final Histogram _histogram; - private final List _percentiles; - private final Map> _gauges; + private final MetricsRegistry registry; + private final Histogram histogram; + private final List percentiles; + private final Map> gauges; public SamzaHistogram(MetricsRegistry registry, String group, String name) { this(registry, group, name, DEFAULT_HISTOGRAM_PERCENTILES); } public SamzaHistogram(MetricsRegistry registry, String group, String name, List percentiles) { - _registry = registry; - _histogram = new Histogram(new ExponentiallyDecayingReservoir()); - _percentiles = percentiles; - _gauges = _percentiles.stream() + this.registry = registry; + histogram = new Histogram(new ExponentiallyDecayingReservoir()); + this.percentiles = percentiles; + gauges = this.percentiles.stream() .filter(x -> x > 0 && x <= 100) .collect( - Collectors.toMap(Function.identity(), x -> _registry.newGauge(group, name + "_" + String.valueOf(0), 0D))); + Collectors.toMap(Function.identity(), x -> this.registry.newGauge(group, name + "_" + String.valueOf(0), 0D))); } public void update(long value) { - _histogram.update(value); - Snapshot values = _histogram.getSnapshot(); - _percentiles.stream().forEach(x -> _gauges.get(x).set(values.getValue(x / 100))); + histogram.update(value); + Snapshot values = histogram.getSnapshot(); + percentiles.stream().forEach(x -> gauges.get(x).set(values.getValue(x / 100))); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 70072ff20d..7633dfdd47 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub.producer; import com.microsoft.azure.eventhubs.EventData; @@ -25,6 +44,7 @@ public class EventHubSystemProducer implements SystemProducer { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemProducer.class.getName()); + private static final long FLUSH_SLEEP_TIME_MILLIS = 1000; public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; @@ -35,132 +55,132 @@ public class EventHubSystemProducer implements SystemProducer { private static final String SEND_ERRORS = "sendErrors"; private static final String SEND_LATENCY = "sendLatency"; private static final String SEND_CALLBACK_LATENCY = "sendCallbackLatency"; - private static Counter _aggEventWriteRate = null; - private static Counter _aggEventByteWriteRate = null; - private static Counter _aggSendErrors = null; - private static SamzaHistogram _aggSendLatency = null; - private static SamzaHistogram _aggSendCallbackLatency = null; - private HashMap _eventWriteRate = new HashMap<>(); - private HashMap _eventByteWriteRate = new HashMap<>(); - private HashMap _sendLatency = new HashMap<>(); - private HashMap _sendCallbackLatency = new HashMap<>(); - private HashMap _sendErrors = new HashMap<>(); - public static final Duration SHUTDOWN_WAIT_TIME = Duration.ofMinutes(1L); - - private final EventHubClientFactory _eventHubClientFactory = new EventHubClientFactory(); - private final EventHubConfig _config; - private final String _systemName; - private final MetricsRegistry _registry; - private final PartitioningMethod _partitioningMethod; - - private Throwable _sendExceptionOnCallback; - private boolean _isStarted; + private static Counter aggEventWriteRate = null; + private static Counter aggEventByteWriteRate = null; + private static Counter aggSendErrors = null; + private static SamzaHistogram aggSendLatency = null; + private static SamzaHistogram aggSendCallbackLatency = null; + private HashMap eventWriteRate = new HashMap<>(); + private HashMap eventByteWriteRate = new HashMap<>(); + private HashMap sendLatency = new HashMap<>(); + private HashMap sendCallbackLatency = new HashMap<>(); + private HashMap sendErrors = new HashMap<>(); + + private final EventHubClientFactory eventHubClientFactory = new EventHubClientFactory(); + private final EventHubConfig config; + private final String systemName; + private final MetricsRegistry registry; + private final PartitioningMethod partitioningMethod; + + private Throwable sendExceptionOnCallback; + private boolean isStarted; // Map of the system name to the event hub client. - private Map _eventHubClients = new HashMap<>(); - private Map> _streamPartitionSenders = new HashMap<>(); + private Map eventHubClients = new HashMap<>(); + private Map> streamPartitionSenders = new HashMap<>(); - private long _messageId; - private Map> _serdes = new HashMap<>(); + // Running count for the next message Id + private long messageId; + private Map> serdes = new HashMap<>(); - private Map> _pendingFutures = new ConcurrentHashMap<>(); + private Map> pendingFutures = new ConcurrentHashMap<>(); public EventHubSystemProducer(String systemName, EventHubConfig config, MetricsRegistry registry) { - _messageId = 0; - _systemName = systemName; - _config = config; - _registry = registry; - _partitioningMethod = _config.getPartitioningMethod(); + messageId = 0; + this.systemName = systemName; + this.config = config; + this.registry = registry; + partitioningMethod = this.config.getPartitioningMethod(); } @Override public synchronized void start() { LOG.info("Starting system producer."); - for (String eventHub : _eventHubClients.keySet()) { - _eventWriteRate.put(eventHub, _registry.newCounter(eventHub, EVENT_WRITE_RATE)); - _eventByteWriteRate.put(eventHub, _registry.newCounter(eventHub, EVENT_BYTE_WRITE_RATE)); - _sendLatency.put(eventHub, new SamzaHistogram(_registry, eventHub, SEND_LATENCY)); - _sendCallbackLatency.put(eventHub, new SamzaHistogram(_registry, eventHub, SEND_CALLBACK_LATENCY)); - _sendErrors.put(eventHub, _registry.newCounter(eventHub, SEND_ERRORS)); + for (String eventHub : eventHubClients.keySet()) { + eventWriteRate.put(eventHub, registry.newCounter(eventHub, EVENT_WRITE_RATE)); + eventByteWriteRate.put(eventHub, registry.newCounter(eventHub, EVENT_BYTE_WRITE_RATE)); + sendLatency.put(eventHub, new SamzaHistogram(registry, eventHub, SEND_LATENCY)); + sendCallbackLatency.put(eventHub, new SamzaHistogram(registry, eventHub, SEND_CALLBACK_LATENCY)); + sendErrors.put(eventHub, registry.newCounter(eventHub, SEND_ERRORS)); } // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. synchronized (AGGREGATE) { - if (_aggEventWriteRate == null) { - _aggEventWriteRate = _registry.newCounter(AGGREGATE, EVENT_WRITE_RATE); - _aggEventByteWriteRate = _registry.newCounter(AGGREGATE, EVENT_BYTE_WRITE_RATE); - _aggSendLatency = new SamzaHistogram(_registry, AGGREGATE, SEND_LATENCY); - _aggSendCallbackLatency = new SamzaHistogram(_registry, AGGREGATE, SEND_CALLBACK_LATENCY); - _aggSendErrors = _registry.newCounter(AGGREGATE, SEND_ERRORS); + if (aggEventWriteRate == null) { + aggEventWriteRate = registry.newCounter(AGGREGATE, EVENT_WRITE_RATE); + aggEventByteWriteRate = registry.newCounter(AGGREGATE, EVENT_BYTE_WRITE_RATE); + aggSendLatency = new SamzaHistogram(registry, AGGREGATE, SEND_LATENCY); + aggSendCallbackLatency = new SamzaHistogram(registry, AGGREGATE, SEND_CALLBACK_LATENCY); + aggSendErrors = registry.newCounter(AGGREGATE, SEND_ERRORS); } } - _isStarted = true; + isStarted = true; } @Override public synchronized void stop() { LOG.info("Stopping system producer."); - _streamPartitionSenders.values().forEach((streamPartitionSender) -> { - List> futures = new ArrayList<>(); - streamPartitionSender.forEach((key, value) -> futures.add(value.close())); - CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); - try { - future.get(SHUTDOWN_WAIT_TIME.toMillis(), TimeUnit.MILLISECONDS); - } catch (ExecutionException | InterruptedException | TimeoutException e) { - LOG.warn("Closing the partition sender failed ", e); - } - }); - _eventHubClients.values().forEach(ehClient -> ehClient.close(SHUTDOWN_WAIT_TIME.toMillis())); - _eventHubClients.clear(); + streamPartitionSenders.values().forEach((streamPartitionSender) -> { + List> futures = new ArrayList<>(); + streamPartitionSender.forEach((key, value) -> futures.add(value.close())); + CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); + try { + future.get(config.getShutdownWaitTimeMS(), TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + LOG.warn("Closing the partition sender failed ", e); + } + }); + eventHubClients.values().forEach(ehClient -> ehClient.close(config.getShutdownWaitTimeMS())); + eventHubClients.clear(); } @Override public synchronized void register(String streamName) { LOG.info("Trying to register {}.", streamName); - if (_isStarted) { + if (isStarted) { String msg = "Cannot register once the producer is started."; LOG.error(msg); throw new SamzaException(msg); } - EventHubClientWrapper ehClient = _eventHubClientFactory - .getEventHubClient(_config.getStreamNamespace(streamName), _config.getStreamEntityPath(streamName), - _config.getStreamSasKeyName(streamName), _config.getStreamSasToken(streamName), _config); + EventHubClientWrapper ehClient = eventHubClientFactory + .getEventHubClient(config.getStreamNamespace(streamName), config.getStreamEntityPath(streamName), + config.getStreamSasKeyName(streamName), config.getStreamSasToken(streamName), config); ehClient.init(); - _eventHubClients.put(streamName, ehClient); - _streamPartitionSenders.put(streamName, new HashMap<>()); - _config.getSerde(streamName).ifPresent(x -> _serdes.put(streamName, x)); + eventHubClients.put(streamName, ehClient); + streamPartitionSenders.put(streamName, new HashMap<>()); + config.getSerde(streamName).ifPresent(x -> serdes.put(streamName, x)); } @Override public synchronized void send(String destination, OutgoingMessageEnvelope envelope) { - if (!_isStarted) { + if (!isStarted) { throw new SamzaException("Trying to call send before the producer is started."); } - if (!_eventHubClients.containsKey(destination)) { + if (!eventHubClients.containsKey(destination)) { String msg = String.format("Trying to send event to a destination {%s} that is not registered.", destination); LOG.error(msg); throw new SamzaException(msg); } - if (_sendExceptionOnCallback != null) { - SamzaException e = new SamzaException(_sendExceptionOnCallback); - _sendExceptionOnCallback = null; - _pendingFutures.clear(); + if (sendExceptionOnCallback != null) { + SamzaException e = new SamzaException(sendExceptionOnCallback); + sendExceptionOnCallback = null; + pendingFutures.clear(); LOG.error("One of the previous sends failed."); throw e; } EventData eventData = createEventData(destination, envelope); - _eventWriteRate.get(destination).inc(); - _aggEventWriteRate.inc(); - _eventByteWriteRate.get(destination).inc(eventData.getBodyLength()); - _aggEventByteWriteRate.inc(eventData.getBodyLength()); - EventHubClientWrapper ehClient = _eventHubClients.get(destination); + eventWriteRate.get(destination).inc(); + aggEventWriteRate.inc(); + eventByteWriteRate.get(destination).inc(eventData.getBodyLength()); + aggEventByteWriteRate.inc(eventData.getBodyLength()); + EventHubClientWrapper ehClient = eventHubClients.get(destination); Instant startTime = Instant.now(); @@ -169,39 +189,39 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo Instant endTime = Instant.now(); long latencyMs = Duration.between(startTime, endTime).toMillis(); - _sendLatency.get(destination).update(latencyMs); - _aggSendLatency.update(latencyMs); + sendLatency.get(destination).update(latencyMs); + aggSendLatency.update(latencyMs); - long messageId = ++_messageId; + long messageId = ++this.messageId; // Rotate the messageIds if (messageId == Long.MAX_VALUE) { - _messageId = 0; + this.messageId = 0; } - _pendingFutures.put(messageId, sendResult); + pendingFutures.put(messageId, sendResult); // Auto remove the future from the list when they are complete. - sendResult.handle(((aVoid, throwable) -> { - long callbackLatencyMs = Duration.between(endTime, Instant.now()).toMillis(); - _sendCallbackLatency.get(destination).update(callbackLatencyMs); - _aggSendCallbackLatency.update(callbackLatencyMs); - if (throwable != null) { - _sendErrors.get(destination).inc(); - _aggSendErrors.inc(); - LOG.error("Send message to event hub: {} failed with exception: ", destination, throwable); - _sendExceptionOnCallback = throwable; - } - _pendingFutures.remove(messageId); - return aVoid; - })); + sendResult.handle((aVoid, throwable) -> { + long callbackLatencyMs = Duration.between(endTime, Instant.now()).toMillis(); + sendCallbackLatency.get(destination).update(callbackLatencyMs); + aggSendCallbackLatency.update(callbackLatencyMs); + if (throwable != null) { + sendErrors.get(destination).inc(); + aggSendErrors.inc(); + LOG.error("Send message to event hub: {} failed with exception: ", destination, throwable); + sendExceptionOnCallback = throwable; + } + pendingFutures.remove(messageId); + return aVoid; + }); } private CompletableFuture sendToEventHub(String streamName, EventData eventData, Object partitionKey, EventHubClient eventHubClient) { - if (_partitioningMethod == PartitioningMethod.EVENT_HUB_HASHING) { + if (partitioningMethod == PartitioningMethod.EVENT_HUB_HASHING) { return eventHubClient.send(eventData, convertPartitionKeyToString(partitionKey)); - } else if (_partitioningMethod == PartitioningMethod.PARTITION_KEY_AS_PARTITION) { + } else if (partitioningMethod == PartitioningMethod.PARTITION_KEY_AS_PARTITION) { if (!(partitionKey instanceof Integer)) { String msg = "Partition key should be of type Integer"; LOG.error(msg); @@ -211,7 +231,7 @@ private CompletableFuture sendToEventHub(String streamName, EventData even PartitionSender sender = getPartitionSender(streamName, (int) partitionKey, eventHubClient); return sender.send(eventData); } else { - throw new SamzaException("Unknown partitioning method " + _partitioningMethod); + throw new SamzaException("Unknown partitioning method " + partitioningMethod); } } @@ -228,10 +248,11 @@ private String convertPartitionKeyToString(Object partitionKey) { } private PartitionSender getPartitionSender(String streamName, int partition, EventHubClient eventHubClient) { - Map partitionSenders = _streamPartitionSenders.get(streamName); + Map partitionSenders = streamPartitionSenders.get(streamName); if (!partitionSenders.containsKey(partition)) { try { - int numPartitions = eventHubClient.getRuntimeInformation().get().getPartitionCount(); + int numPartitions = eventHubClient.getRuntimeInformation() + .get(config.getRuntimeInfoWaitTimeMS(), TimeUnit.MILLISECONDS).getPartitionCount(); PartitionSender partitionSender = eventHubClient.createPartitionSenderSync(String.valueOf(partition % numPartitions)); partitionSenders.put(partition, partitionSender); @@ -239,7 +260,7 @@ private PartitionSender getPartitionSender(String streamName, int partition, Eve String msg = "Creation of partition sender failed with exception"; LOG.error(msg, e); throw new SamzaException(msg, e); - } catch (InterruptedException | ExecutionException e) { + } catch (InterruptedException | ExecutionException | TimeoutException e) { String msg = "Failed to fetch number of Event Hub partitions for partition sender creation"; LOG.error(msg, e); throw new SamzaException(msg, e); @@ -250,7 +271,7 @@ private PartitionSender getPartitionSender(String streamName, int partition, Eve } private EventData createEventData(String streamName, OutgoingMessageEnvelope envelope) { - Optional> serde = Optional.ofNullable(_serdes.getOrDefault(streamName, null)); + Optional> serde = Optional.ofNullable(serdes.getOrDefault(streamName, null)); byte[] eventValue = (byte[]) envelope.getMessage(); if (serde.isPresent()) { eventValue = serde.get().toBytes(eventValue); @@ -260,7 +281,7 @@ private EventData createEventData(String streamName, OutgoingMessageEnvelope env eventData.getProperties().put(PRODUCE_TIMESTAMP, Long.toString(System.currentTimeMillis())); - if (_config.getSendKeyInEventProperties()) { + if (config.getSendKeyInEventProperties()) { String keyValue = ""; if (envelope.getKey() != null) { keyValue = (envelope.getKey() instanceof byte[]) ? new String((byte[]) envelope.getKey()) @@ -273,11 +294,11 @@ private EventData createEventData(String streamName, OutgoingMessageEnvelope env @Override public void flush(String source) { - LOG.info("Trying to flush pending {} sends messages: {}", _pendingFutures.size(), _pendingFutures.keySet()); + LOG.info("Trying to flush pending {} sends messages: {}", pendingFutures.size(), pendingFutures.keySet()); // Wait till all the pending sends are complete. - while (!_pendingFutures.isEmpty()) { + while (!pendingFutures.isEmpty()) { try { - Thread.sleep(1000); + Thread.sleep(FLUSH_SLEEP_TIME_MILLIS); } catch (InterruptedException e) { String msg = "Flush failed with error"; LOG.error(msg, e); @@ -285,10 +306,10 @@ public void flush(String source) { } } - if (_sendExceptionOnCallback != null) { + if (sendExceptionOnCallback != null) { String msg = "Sending one of the message failed during flush"; - Throwable throwable = _sendExceptionOnCallback; - _sendExceptionOnCallback = null; + Throwable throwable = sendExceptionOnCallback; + sendExceptionOnCallback = null; LOG.error(msg, throwable); throw new SamzaException(msg, throwable); } @@ -297,7 +318,7 @@ public void flush(String source) { } Collection> getPendingFutures() { - return _pendingFutures.values(); + return pendingFutures.values(); } public enum PartitioningMethod { diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java index 8b5b4e7db0..b5b55dc467 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventData.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub; import com.microsoft.azure.eventhubs.EventData; @@ -7,15 +26,15 @@ public class MockEventData extends EventData { - EventData.SystemProperties _overridedSystemProperties; + private EventData.SystemProperties overridedSystemProperties; - public MockEventData(byte[] data, String partitionKey, String offset) { + private MockEventData(byte[] data, String partitionKey, String offset) { super(data); HashMap properties = new HashMap<>(); properties.put("x-opt-offset", offset); properties.put("x-opt-partition-key", partitionKey); properties.put("x-opt-enqueued-time", new Date(System.currentTimeMillis())); - _overridedSystemProperties = new SystemProperties(properties); + overridedSystemProperties = new SystemProperties(properties); } public static List generateEventData(int numEvents) { @@ -33,6 +52,6 @@ public static List generateEventData(int numEvents) { @Override public EventData.SystemProperties getSystemProperties() { - return _overridedSystemProperties; + return overridedSystemProperties; } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java index efa96aa287..e15317e754 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub; import org.apache.samza.config.Config; diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java index 5523398de4..a421cbd221 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub; import org.apache.commons.collections4.map.HashedMap; @@ -12,35 +31,35 @@ public class TestMetricsRegistry implements MetricsRegistry { - private Map> _counters = new HashedMap<>(); - private Map>> _gauges = new HashedMap<>(); + private Map> counters = new HashedMap<>(); + private Map>> gauges = new HashedMap<>(); public List getCounters(String groupName) { - return _counters.get(groupName); + return counters.get(groupName); } public List> getGauges(String groupName) { - return _gauges.get(groupName); + return gauges.get(groupName); } @Override public Counter newCounter(String group, String name) { - if (!_counters.containsKey(group)) { - _counters.put(group, new ArrayList<>()); + if (!counters.containsKey(group)) { + counters.put(group, new ArrayList<>()); } Counter c = new Counter(name); - _counters.get(group).add(c); + counters.get(group).add(c); return c; } @Override public Gauge newGauge(String group, String name, T value) { - if (!_gauges.containsKey(group)) { - _gauges.put(group, new ArrayList<>()); + if (!gauges.containsKey(group)) { + gauges.put(group, new ArrayList<>()); } - Gauge g = new Gauge(name, value); - _gauges.get(group).add(g); + Gauge g = new Gauge<>(name, value); + gauges.get(group).add(g); return g; } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java index 37744b5396..b5ccc6c317 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub.admin; import junit.framework.Assert; @@ -79,10 +98,10 @@ public void testGetStreamMetadata() { Assert.assertTrue(partitionMetadataMap.size() >= MIN_EVENTHUB_ENTITY_PARTITION); Assert.assertTrue(partitionMetadataMap.size() <= MAX_EVENTHUB_ENTITY_PARTITION); partitionMetadataMap.forEach((partition, metadata) -> { - Assert.assertEquals(EventHubSystemConsumer.START_OF_STREAM, metadata.getOldestOffset()); - Assert.assertEquals(EventHubSystemConsumer.END_OF_STREAM, metadata.getNewestOffset()); - Assert.assertEquals(EventHubSystemConsumer.END_OF_STREAM, metadata.getUpcomingOffset()); - }); + Assert.assertEquals(EventHubSystemConsumer.START_OF_STREAM, metadata.getOldestOffset()); + Assert.assertEquals(EventHubSystemConsumer.END_OF_STREAM, metadata.getNewestOffset()); + Assert.assertEquals(EventHubSystemConsumer.END_OF_STREAM, metadata.getUpcomingOffset()); + }); } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java index 49763f2512..3f64b2b989 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java @@ -1,8 +1,26 @@ +/* +* 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.samza.system.eventhub.consumer; import com.microsoft.azure.eventhubs.EventData; import com.microsoft.azure.eventhubs.PartitionReceiveHandler; -import org.apache.samza.system.eventhub.EventHubClientFactory; import org.apache.samza.system.eventhub.EventHubConfig; import java.util.List; @@ -10,36 +28,36 @@ class MockEventHubEntityConnectionFactory extends EventHubEntityConnectionFactory { - private final Map>> _eventData; + private final Map>> eventData; MockEventHubEntityConnectionFactory(Map>> eventData) { - _eventData = eventData; + this.eventData = eventData; } @Override EventHubEntityConnection createConnection(String namespace, String entityPath, String sasKeyName, String sasKey, String consumerName, EventHubConfig eventHubConfig) { - return new MockEventHubEntityConnection(entityPath, _eventData.get(entityPath)); + return new MockEventHubEntityConnection(entityPath, eventData.get(entityPath)); } private class MockEventHubEntityConnection extends EventHubEntityConnection { - private final Map> _eventData; + private final Map> eventData; MockEventHubEntityConnection(String entity, Map> eventData) { super(null, entity, null, null, null, null); assert eventData != null : "No event data found for entity:" + entity; - _eventData = eventData; + this.eventData = eventData; } @Override void connectAndStart() { - _offsets.keySet().forEach(partitionId -> { - List events = _eventData.get(partitionId); - PartitionReceiveHandler partitionHandler = _handlers.get(partitionId); - assert events != null : String.format("partition %d not found", partitionId); - assert partitionHandler != null : String.format("handler %d not registered", partitionId); - partitionHandler.onReceive(events); - }); + offsets.keySet().forEach(partitionId -> { + List events = eventData.get(partitionId); + PartitionReceiveHandler partitionHandler = handlers.get(partitionId); + assert events != null : String.format("partition %d not found", partitionId); + assert partitionHandler != null : String.format("handler %d not registered", partitionId); + partitionHandler.onReceive(events); + }); } @Override diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index 7ffbb19232..8ce17e642a 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub.consumer; @@ -18,8 +37,8 @@ import java.util.stream.Collectors; public class TestEventHubSystemConsumer { - private final String MOCK_ENTITY_1 = "mocktopic1"; - private final String MOCK_ENTITY_2 = "mocktopic2"; + private static final String MOCK_ENTITY_1 = "mocktopic1"; + private static final String MOCK_ENTITY_2 = "mocktopic2"; private void verifyEvents(List messages, List eventDataList) { Assert.assertEquals(messages.size(), eventDataList.size()); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java index 7cbef24712..c6045977d6 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -1,3 +1,22 @@ +/* +* 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.samza.system.eventhub.producer; import com.microsoft.azure.eventhubs.EventData; From 2c22671ceddff9c7aa400ac196e931749bcb396f Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Fri, 29 Sep 2017 14:52:13 -0700 Subject: [PATCH 09/34] removed depricated EventData methods --- .../system/eventhub/consumer/EventHubSystemConsumer.java | 7 ++++--- .../system/eventhub/producer/EventHubSystemProducer.java | 6 +++--- .../eventhub/consumer/TestEventHubSystemConsumer.java | 2 +- .../eventhub/producer/TestEventHubSystemProducer.java | 2 +- 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index b4b4edb455..9fa021b53a 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -201,7 +201,7 @@ public void onReceive(Iterable events) { if (events != null) { events.forEach(event -> { - byte[] decryptedBody = event.getBody(); + byte[] decryptedBody = event.getBytes(); if (serde != null) { decryptedBody = serde.fromBytes(decryptedBody); } @@ -221,10 +221,11 @@ public void onReceive(Iterable events) { } private void updateMetrics(EventData event) { + int eventDataLength = event.getBytes() == null ? 0 : event.getBytes().length; eventReadRate.inc(); aggEventReadRate.inc(); - eventByteReadRate.inc(event.getBodyLength()); - aggEventByteReadRate.inc(event.getBodyLength()); + eventByteReadRate.inc(eventDataLength); + aggEventByteReadRate.inc(eventDataLength); long latencyMs = Duration.between(Instant.now(), event.getSystemProperties().getEnqueuedTime()).toMillis(); readLatency.update(latencyMs); aggReadLatency.update(latencyMs); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 7633dfdd47..ca78d5f80a 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -175,11 +175,11 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo } EventData eventData = createEventData(destination, envelope); - + int eventDataLength = eventData.getBytes() == null ? 0 : eventData.getBytes().length; eventWriteRate.get(destination).inc(); aggEventWriteRate.inc(); - eventByteWriteRate.get(destination).inc(eventData.getBodyLength()); - aggEventByteWriteRate.inc(eventData.getBodyLength()); + eventByteWriteRate.get(destination).inc(eventDataLength); + aggEventByteWriteRate.inc(eventDataLength); EventHubClientWrapper ehClient = eventHubClients.get(destination); Instant startTime = Instant.now(); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index 8ce17e642a..2df4bc9e18 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -46,7 +46,7 @@ private void verifyEvents(List messages, List Date: Fri, 29 Sep 2017 15:29:31 -0700 Subject: [PATCH 10/34] throw on missing connection configs, fixed typos --- .../system/eventhub/EventDataWrapper.java | 2 +- .../samza/system/eventhub/EventHubConfig.java | 18 ++++++++++++++---- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java index f2daf96783..08880ba197 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java @@ -38,7 +38,7 @@ public EventData getEventData() { } /** - * @return the body of decrypted body of the message. In case not encryption is setup for this topic + * @return the body of decrypted body of the message. In case no encryption is setup for this topic, * just returns the body of the message. */ public byte[] getDecryptedBody() { diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java index 7a602235f3..f149af2309 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -21,6 +21,8 @@ import com.microsoft.azure.eventhubs.EventHubClient; import org.apache.commons.lang3.StringUtils; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.SerdeFactory; @@ -72,6 +74,14 @@ public EventHubConfig(Map config, String systemName) { this.systemName = systemName; } + private String getRequiredConfigValue(String configKey, String streamName) { + String configValue = get(String.format(configKey, systemName, streamName), null); + if (configValue == null) { + throw new SamzaException(configKey + " is not configured."); + } + return configValue; + } + /** * Get the list of streams that are defined. Each stream has enough * information for connecting to a certain EventHub entity. @@ -89,7 +99,7 @@ public List getStreamList() { * @return EventHubs namespace */ public String getStreamNamespace(String streamName) { - return get(String.format(CONFIG_STREAM_NAMESPACE, systemName, streamName)); + return getRequiredConfigValue(CONFIG_STREAM_NAMESPACE, streamName); } /** @@ -99,7 +109,7 @@ public String getStreamNamespace(String streamName) { * @return EventHubs entity path */ public String getStreamEntityPath(String streamName) { - return get(String.format(CONFIG_STREAM_ENTITYPATH, systemName, streamName)); + return getRequiredConfigValue(CONFIG_STREAM_ENTITYPATH, streamName); } /** @@ -109,7 +119,7 @@ public String getStreamEntityPath(String streamName) { * @return EventHubs SAS key name */ public String getStreamSasKeyName(String streamName) { - return get(String.format(CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName)); + return getRequiredConfigValue(CONFIG_STREAM_SAS_KEY_NAME, streamName); } /** @@ -119,7 +129,7 @@ public String getStreamSasKeyName(String streamName) { * @return EventHubs SAS token */ public String getStreamSasToken(String streamName) { - return get(String.format(CONFIG_STREAM_SAS_TOKEN, systemName, streamName)); + return getRequiredConfigValue(CONFIG_STREAM_SAS_TOKEN, streamName); } public Optional> getSerde(String streamName) { From 196963b6fa31f103734e7ecf6a60a51d1675511f Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Fri, 29 Sep 2017 15:42:11 -0700 Subject: [PATCH 11/34] removed unused import --- .../java/org/apache/samza/system/eventhub/EventHubConfig.java | 1 - 1 file changed, 1 deletion(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java index f149af2309..5886b9e0a7 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -22,7 +22,6 @@ import com.microsoft.azure.eventhubs.EventHubClient; import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; -import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.SerdeFactory; From 4343678e7ce740285f61ecd9ccf97b864a16bffd Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 2 Oct 2017 13:17:13 -0700 Subject: [PATCH 12/34] more accurate fetch for newestOffset --- .../eventhub/admin/EventHubSystemAdmin.java | 66 ++++++++++++++----- .../admin/TestEventHubSystemAdmin.java | 4 +- 2 files changed, 54 insertions(+), 16 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index 9394cd248b..ddc5513bc0 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -19,11 +19,13 @@ package org.apache.samza.system.eventhub.admin; +import com.microsoft.azure.eventhubs.EventHubPartitionRuntimeInformation; import com.microsoft.azure.eventhubs.EventHubRuntimeInformation; import org.apache.samza.Partition; import org.apache.samza.SamzaException; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemStreamMetadata; +import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata; import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.eventhub.EventHubClientFactory; import org.apache.samza.system.eventhub.EventHubClientWrapper; @@ -71,25 +73,16 @@ public Map getSystemStreamMetadata(Set str Map requestedMetadata = new HashMap<>(); Map> ehRuntimeInfos = new HashMap<>(); streamNames.forEach((streamName) -> { - if (!eventHubClients.containsKey(streamName)) { - eventHubClients.put(streamName, eventHubClientFactory - .getEventHubClient(eventHubConfig.getStreamNamespace(streamName), eventHubConfig.getStreamEntityPath(streamName), - eventHubConfig.getStreamSasKeyName(streamName), eventHubConfig.getStreamSasToken(streamName), eventHubConfig)); - eventHubClients.get(streamName).init(); - } + EventHubClientWrapper eventHubClientWrapper = getEventHubClient(streamName); ehRuntimeInfos.put(streamName, - eventHubClients.get(streamName).getEventHubClient().getRuntimeInformation()); + eventHubClientWrapper.getEventHubClient().getRuntimeInformation()); }); ehRuntimeInfos.forEach((streamName, ehRuntimeInfo) -> { try { - EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(eventHubConfig.getRuntimeInfoWaitTimeMS(), TimeUnit.MILLISECONDS); - Map sspMetadataMap = new HashMap<>(); - for (String partition : ehInfo.getPartitionIds()) { - sspMetadataMap.put(new Partition(Integer.parseInt(partition)), - new SystemStreamMetadata.SystemStreamPartitionMetadata(EventHubSystemConsumer.START_OF_STREAM, - EventHubSystemConsumer.END_OF_STREAM, EventHubSystemConsumer.END_OF_STREAM)); - } - requestedMetadata.put(streamName, new SystemStreamMetadata(streamName, sspMetadataMap)); + EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(eventHubConfig.getRuntimeInfoWaitTimeMS(), + TimeUnit.MILLISECONDS); + requestedMetadata.put(streamName, new SystemStreamMetadata(streamName, + getPartitionMetadata(streamName, ehInfo.getPartitionIds()))); } catch (InterruptedException | ExecutionException e) { String msg = String.format("Error while fetching EventHubRuntimeInfo for System:%s, Stream:%s", systemName, streamName); @@ -105,6 +98,49 @@ public Map getSystemStreamMetadata(Set str return requestedMetadata; } + private EventHubClientWrapper getEventHubClient(String streamName) { + if (!eventHubClients.containsKey(streamName)) { + eventHubClients.put(streamName, eventHubClientFactory + .getEventHubClient(eventHubConfig.getStreamNamespace(streamName), + eventHubConfig.getStreamEntityPath(streamName), eventHubConfig.getStreamSasKeyName(streamName), + eventHubConfig.getStreamSasToken(streamName), eventHubConfig)); + eventHubClients.get(streamName).init(); + } + return eventHubClients.get(streamName); + } + + private Map getPartitionMetadata(String streamName, String[] partitionIds) { + EventHubClientWrapper eventHubClientWrapper = getEventHubClient(streamName); + Map sspMetadataMap = new HashMap<>(); + Map> ehRuntimeInfos = new HashMap<>(); + for (String partition : partitionIds) { + ehRuntimeInfos.put(partition, eventHubClientWrapper.getEventHubClient() + .getPartitionRuntimeInformation(partition)); + } + ehRuntimeInfos.forEach((partitionId, ehPartitionRuntimeInfo) -> { + try { + EventHubPartitionRuntimeInformation ehPartitionInfo = ehPartitionRuntimeInfo + .get(eventHubConfig.getRuntimeInfoWaitTimeMS(), TimeUnit.MILLISECONDS); + sspMetadataMap.put(new Partition(Integer.parseInt(partitionId)), + new SystemStreamPartitionMetadata(EventHubSystemConsumer.START_OF_STREAM, + ehPartitionInfo.getLastEnqueuedOffset(), EventHubSystemConsumer.END_OF_STREAM)); + } catch (InterruptedException | ExecutionException e) { + String msg = String.format( + "Error while fetching EventHubPartitionRuntimeInfo for System:%s, Stream:%s, Partition:%s", + systemName, streamName, partitionId); + LOG.error(msg, e); + throw new SamzaException(msg); + } catch (TimeoutException e) { + String msg = String.format( + "Timed out while fetching EventHubRuntimeInfo for System:%s, Stream:%s, , Partition:%s", + systemName, streamName, partitionId); + LOG.error(msg, e); + throw new SamzaException(msg); + } + }); + return sspMetadataMap; + } + @Override public void createChangelogStream(String streamName, int numOfPartitions) { throw new UnsupportedOperationException("Event Hubs does not support change log stream."); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java index b5ccc6c317..7cb6ebdb64 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java @@ -99,7 +99,9 @@ public void testGetStreamMetadata() { Assert.assertTrue(partitionMetadataMap.size() <= MAX_EVENTHUB_ENTITY_PARTITION); partitionMetadataMap.forEach((partition, metadata) -> { Assert.assertEquals(EventHubSystemConsumer.START_OF_STREAM, metadata.getOldestOffset()); - Assert.assertEquals(EventHubSystemConsumer.END_OF_STREAM, metadata.getNewestOffset()); + Assert.assertNotSame(EventHubSystemConsumer.END_OF_STREAM, metadata.getNewestOffset()); + Assert.assertTrue(Long.parseLong(EventHubSystemConsumer.END_OF_STREAM) + <= Long.parseLong(metadata.getNewestOffset())); Assert.assertEquals(EventHubSystemConsumer.END_OF_STREAM, metadata.getUpcomingOffset()); }); } From 0f9065da0aec999ed6c64bab13b9398aa7381831 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 2 Oct 2017 17:45:28 -0700 Subject: [PATCH 13/34] leverage EH constant partition size to decrease request --- .../eventhub/admin/EventHubSystemAdmin.java | 46 +++++++++++-------- .../producer/EventHubSystemProducer.java | 11 +++-- 2 files changed, 33 insertions(+), 24 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index ddc5513bc0..4d1ebff401 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -48,6 +48,7 @@ public class EventHubSystemAdmin implements SystemAdmin { private String systemName; private EventHubConfig eventHubConfig; private Map eventHubClients = new HashMap<>(); + private Map streamPartitions = new HashMap<>(); public EventHubSystemAdmin(String systemName, EventHubConfig config) { this.systemName = systemName; @@ -73,32 +74,37 @@ public Map getSystemStreamMetadata(Set str Map requestedMetadata = new HashMap<>(); Map> ehRuntimeInfos = new HashMap<>(); streamNames.forEach((streamName) -> { - EventHubClientWrapper eventHubClientWrapper = getEventHubClient(streamName); - ehRuntimeInfos.put(streamName, - eventHubClientWrapper.getEventHubClient().getRuntimeInformation()); + if (!streamPartitions.containsKey(streamName)) { + EventHubClientWrapper eventHubClientWrapper = getStreamEventHubClient(streamName); + ehRuntimeInfos.put(streamName, + eventHubClientWrapper.getEventHubClient().getRuntimeInformation()); + } }); ehRuntimeInfos.forEach((streamName, ehRuntimeInfo) -> { - try { - EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(eventHubConfig.getRuntimeInfoWaitTimeMS(), - TimeUnit.MILLISECONDS); - requestedMetadata.put(streamName, new SystemStreamMetadata(streamName, - getPartitionMetadata(streamName, ehInfo.getPartitionIds()))); - } catch (InterruptedException | ExecutionException e) { - String msg = String.format("Error while fetching EventHubRuntimeInfo for System:%s, Stream:%s", - systemName, streamName); - LOG.error(msg, e); - throw new SamzaException(msg); - } catch (TimeoutException e) { - String msg = String.format("Timed out while fetching EventHubRuntimeInfo for System:%s, Stream:%s", - systemName, streamName); - LOG.error(msg, e); - throw new SamzaException(msg); + if (!streamPartitions.containsKey(streamName)) { + try { + EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(eventHubConfig.getRuntimeInfoWaitTimeMS(), + TimeUnit.MILLISECONDS); + streamPartitions.put(streamName, ehInfo.getPartitionIds()); + } catch (InterruptedException | ExecutionException e) { + String msg = String.format("Error while fetching EventHubRuntimeInfo for System:%s, Stream:%s", + systemName, streamName); + LOG.error(msg, e); + throw new SamzaException(msg); + } catch (TimeoutException e) { + String msg = String.format("Timed out while fetching EventHubRuntimeInfo for System:%s, Stream:%s", + systemName, streamName); + LOG.error(msg, e); + throw new SamzaException(msg); + } } + requestedMetadata.put(streamName, new SystemStreamMetadata(streamName, + getPartitionMetadata(streamName, streamPartitions.get(streamName)))); }); return requestedMetadata; } - private EventHubClientWrapper getEventHubClient(String streamName) { + private EventHubClientWrapper getStreamEventHubClient(String streamName) { if (!eventHubClients.containsKey(streamName)) { eventHubClients.put(streamName, eventHubClientFactory .getEventHubClient(eventHubConfig.getStreamNamespace(streamName), @@ -110,7 +116,7 @@ private EventHubClientWrapper getEventHubClient(String streamName) { } private Map getPartitionMetadata(String streamName, String[] partitionIds) { - EventHubClientWrapper eventHubClientWrapper = getEventHubClient(streamName); + EventHubClientWrapper eventHubClientWrapper = getStreamEventHubClient(streamName); Map sspMetadataMap = new HashMap<>(); Map> ehRuntimeInfos = new HashMap<>(); for (String partition : partitionIds) { diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index ca78d5f80a..6fb1266cc2 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -78,6 +78,7 @@ public class EventHubSystemProducer implements SystemProducer { // Map of the system name to the event hub client. private Map eventHubClients = new HashMap<>(); private Map> streamPartitionSenders = new HashMap<>(); + private Map streamPartitionCounts = new HashMap<>(); // Running count for the next message Id private long messageId; @@ -251,10 +252,12 @@ private PartitionSender getPartitionSender(String streamName, int partition, Eve Map partitionSenders = streamPartitionSenders.get(streamName); if (!partitionSenders.containsKey(partition)) { try { - int numPartitions = eventHubClient.getRuntimeInformation() - .get(config.getRuntimeInfoWaitTimeMS(), TimeUnit.MILLISECONDS).getPartitionCount(); - PartitionSender partitionSender = - eventHubClient.createPartitionSenderSync(String.valueOf(partition % numPartitions)); + if (!streamPartitionCounts.containsKey(streamName)) { + streamPartitionCounts.put(streamName, eventHubClient.getRuntimeInformation() + .get(config.getRuntimeInfoWaitTimeMS(), TimeUnit.MILLISECONDS).getPartitionCount()); + } + PartitionSender partitionSender = eventHubClient + .createPartitionSenderSync(String.valueOf(partition % streamPartitionCounts.get(streamName))); partitionSenders.put(partition, partitionSender); } catch (ServiceBusException e) { String msg = "Creation of partition sender failed with exception"; From 1ebc4672b38a4acaeefc6148f3f22bdaaead8b95 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Tue, 3 Oct 2017 15:18:11 -0700 Subject: [PATCH 14/34] added Event Hub consumer buffer capacity per SystemStreamPartition --- .../samza/system/eventhub/EventHubConfig.java | 17 +++++++++++++++++ .../consumer/EventHubSystemConsumer.java | 8 +++++++- 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java index 5886b9e0a7..cb58ef7fd7 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -66,6 +66,10 @@ public class EventHubConfig extends MapConfig { public static final String CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS = "systems.%s.eventhubs.runtime.info.timeout"; public static final long DEFAULT_CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); + public static final String CONFIG_CONSUMER_BUFFER_CAPACITY = "systems.%s.eventhubs.receive.queue.size"; + public static final int DEFAULT_CONFIG_CONSUMER_BUFFER_CAPACITY = 100; + + private final String systemName; public EventHubConfig(Map config, String systemName) { @@ -214,6 +218,19 @@ public long getRuntimeInfoWaitTimeMS() { return Long.valueOf(timeoutStr); } + /** + * Get the capacity of the Event Hub consumer buffer - the blocking queue used for storing messages + * + * @return int, number of buffered messages per SystemStreamPartition + */ + public int getConsumerBufferCapacity() { + String bufferCapacity = get(String.format(CONFIG_CONSUMER_BUFFER_CAPACITY, systemName)); + if (bufferCapacity == null) { + return DEFAULT_CONFIG_CONSUMER_BUFFER_CAPACITY; + } + return Integer.parseInt(bufferCapacity); + } + @Override public int hashCode() { return Objects.hash(super.hashCode(), systemName); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index 9fa021b53a..1c009e03df 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -41,6 +41,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; import java.util.function.Function; import java.util.stream.Collectors; @@ -92,7 +94,6 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { public static final String READ_ERRORS = "readErrors"; private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemConsumer.class); private static final int MAX_EVENT_COUNT_PER_PARTITION_POLL = 50; - private static final int BLOCKING_QUEUE_SIZE = 100; private static Counter aggEventReadRate = null; private static Counter aggEventByteReadRate = null; private static SamzaHistogram aggReadLatency = null; @@ -176,6 +177,11 @@ public void stop() { connections.values().forEach(EventHubEntityConnection::stop); } + @Override + protected BlockingQueue newBlockingQueue() { + return new LinkedBlockingQueue<>(config.getConsumerBufferCapacity()); + } + private class PartitionReceiverHandlerImpl extends PartitionReceiveHandler { private final Counter eventReadRate; From aab7f5126781d9a8158b02f18173d068f5b72a5c Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 9 Oct 2017 13:44:52 -0700 Subject: [PATCH 15/34] Removed EHEntityConnection, Moved EHWrapper factory to System ctors --- build.gradle | 6 +- ...java => EventHubClientWrapperFactory.java} | 6 +- .../eventhub/EventHubSystemFactory.java | 9 +- .../eventhub/admin/EventHubSystemAdmin.java | 14 +- .../consumer/EventHubEntityConnection.java | 132 ---------------- .../EventHubEntityConnectionFactory.java | 29 ---- .../consumer/EventHubSystemConsumer.java | 147 +++++++++++++----- .../producer/EventHubSystemProducer.java | 20 +-- .../admin/TestEventHubSystemAdmin.java | 2 +- .../consumer/MockEventHubClientFactory.java | 109 +++++++++++++ .../MockEventHubEntityConnectionFactory.java | 68 -------- .../consumer/TestEventHubSystemConsumer.java | 99 ++++++++---- .../producer/TestEventHubSystemProducer.java | 4 +- 13 files changed, 318 insertions(+), 327 deletions(-) rename samza-azure/src/main/java/org/apache/samza/system/eventhub/{EventHubClientFactory.java => EventHubClientWrapperFactory.java} (78%) delete mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java delete mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubClientFactory.java delete mode 100644 samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java diff --git a/build.gradle b/build.gradle index 959ddd0493..928007af94 100644 --- a/build.gradle +++ b/build.gradle @@ -199,13 +199,17 @@ project(':samza-azure') { compile project(":samza-core_$scalaVersion") compile "org.slf4j:slf4j-api:$slf4jVersion" testCompile "junit:junit:$junitVersion" + testCompile "org.mockito:mockito-all:$mockitoVersion" + testCompile "org.powermock:powermock-api-mockito:$powerMockVersion" + testCompile "org.powermock:powermock-core:$powerMockVersion" + testCompile "org.powermock:powermock-module-junit4:$powerMockVersion" } checkstyle { configFile = new File(rootDir, "checkstyle/checkstyle.xml") toolVersion = "$checkstyleVersion" } test { - exclude 'org/apache/samza/system/eventhub/**' + exclude 'org/apache/samza/system/eventhub/producer/**' } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapperFactory.java similarity index 78% rename from samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientFactory.java rename to samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapperFactory.java index 4a1d6205d4..82941905bf 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapperFactory.java @@ -19,9 +19,9 @@ package org.apache.samza.system.eventhub; -public class EventHubClientFactory { - public EventHubClientWrapper getEventHubClient(String eventHubNamespace, String entityPath, String sasKeyName, - String sasToken, EventHubConfig config) { +public class EventHubClientWrapperFactory { + public EventHubClientWrapper getEventHubClientWrapper(String eventHubNamespace, String entityPath, String sasKeyName, + String sasToken, EventHubConfig config) { return new SamzaEventHubClientWrapper(eventHubNamespace, entityPath, sasKeyName, sasToken); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java index 8901feb4a9..5b15fc36fb 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java @@ -28,7 +28,6 @@ import org.apache.samza.system.SystemFactory; import org.apache.samza.system.SystemProducer; import org.apache.samza.system.eventhub.admin.EventHubSystemAdmin; -import org.apache.samza.system.eventhub.consumer.EventHubEntityConnectionFactory; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; @@ -51,17 +50,17 @@ public static SerdeFactory getSerdeFactory(String serdeFactoryClassName) @Override public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { - return new EventHubSystemConsumer(new EventHubConfig(config, systemName), - new EventHubEntityConnectionFactory(), registry); + return new EventHubSystemConsumer(new EventHubConfig(config, systemName), new EventHubClientWrapperFactory(), registry); } @Override public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { - return new EventHubSystemProducer(systemName, new EventHubConfig(config, systemName), registry); + return new EventHubSystemProducer(systemName, new EventHubConfig(config, systemName), new EventHubClientWrapperFactory(), + registry); } @Override public SystemAdmin getAdmin(String systemName, Config config) { - return new EventHubSystemAdmin(systemName, new EventHubConfig(config, systemName)); + return new EventHubSystemAdmin(systemName, new EventHubConfig(config, systemName), new EventHubClientWrapperFactory()); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index 4d1ebff401..843cfc3f83 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -27,10 +27,10 @@ import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata; import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.system.eventhub.EventHubClientFactory; import org.apache.samza.system.eventhub.EventHubClientWrapper; import org.apache.samza.system.eventhub.EventHubConfig; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; +import org.apache.samza.system.eventhub.EventHubClientWrapperFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,15 +44,17 @@ public class EventHubSystemAdmin implements SystemAdmin { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemAdmin.class); - private final EventHubClientFactory eventHubClientFactory = new EventHubClientFactory(); + private final EventHubClientWrapperFactory eventHubClientWrapperFactory; private String systemName; private EventHubConfig eventHubConfig; private Map eventHubClients = new HashMap<>(); private Map streamPartitions = new HashMap<>(); - public EventHubSystemAdmin(String systemName, EventHubConfig config) { + public EventHubSystemAdmin(String systemName, EventHubConfig eventHubConfig, + EventHubClientWrapperFactory eventHubClientWrapperFactory) { this.systemName = systemName; - eventHubConfig = config; + this.eventHubConfig = eventHubConfig; + this.eventHubClientWrapperFactory = eventHubClientWrapperFactory; } private static String getNextOffset(String currentOffset) { @@ -106,8 +108,8 @@ public Map getSystemStreamMetadata(Set str private EventHubClientWrapper getStreamEventHubClient(String streamName) { if (!eventHubClients.containsKey(streamName)) { - eventHubClients.put(streamName, eventHubClientFactory - .getEventHubClient(eventHubConfig.getStreamNamespace(streamName), + eventHubClients.put(streamName, eventHubClientWrapperFactory + .getEventHubClientWrapper(eventHubConfig.getStreamNamespace(streamName), eventHubConfig.getStreamEntityPath(streamName), eventHubConfig.getStreamSasKeyName(streamName), eventHubConfig.getStreamSasToken(streamName), eventHubConfig)); eventHubClients.get(streamName).init(); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java deleted file mode 100644 index 08cdc79c18..0000000000 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnection.java +++ /dev/null @@ -1,132 +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. -*/ - -package org.apache.samza.system.eventhub.consumer; - -import com.microsoft.azure.eventhubs.PartitionReceiveHandler; -import com.microsoft.azure.eventhubs.PartitionReceiver; -import com.microsoft.azure.servicebus.ServiceBusException; -import com.microsoft.azure.servicebus.StringUtil; -import org.apache.samza.SamzaException; -import org.apache.samza.system.eventhub.EventHubClientFactory; -import org.apache.samza.system.eventhub.EventHubClientWrapper; -import org.apache.samza.system.eventhub.EventHubConfig; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.time.Instant; -import java.util.Map; -import java.util.TreeMap; - -public class EventHubEntityConnection { - private static final Logger LOG = LoggerFactory.getLogger(EventHubEntityConnection.class); - final Map offsets = new TreeMap<>(); - final Map handlers = new TreeMap<>(); - private final String namespace; - private final String entityPath; - private final String sasKeyName; - private final String sasKey; - private final String consumerName; - private final EventHubConfig config; - private final Map receivers = new TreeMap<>(); - private EventHubClientWrapper ehClientWrapper; - private final EventHubClientFactory eventHubClientFactory = new EventHubClientFactory(); - private boolean isStarted = false; - - EventHubEntityConnection(String namespace, String entityPath, String sasKeyName, String sasKey, - String consumerName, EventHubConfig config) { - this.namespace = namespace; - this.entityPath = entityPath; - this.sasKeyName = sasKeyName; - this.sasKey = sasKey; - this.consumerName = consumerName; - this.config = config; - } - - // add partitions and handlers for this connection. This can be called multiple times - // for multiple partitions, but needs to be called before connectAndStart() - synchronized void addPartition(int partitionId, String offset, PartitionReceiveHandler handler) { - if (isStarted) { - LOG.warn("Trying to add partition when the connection has already started."); - return; - } - offsets.put(partitionId, offset); - handlers.put(partitionId, handler); - } - - // establish the connection and start consuming events - synchronized void connectAndStart() { - isStarted = true; - try { - LOG.info(String.format("Starting connection for namespace=%s, entity=%s ", namespace, entityPath)); - // upon the instantiation of the client, the connection will be established - ehClientWrapper = eventHubClientFactory - .getEventHubClient(namespace, entityPath, sasKeyName, sasKey, config); - ehClientWrapper.init(); - for (Map.Entry entry : offsets.entrySet()) { - Integer id = entry.getKey(); - String offset = entry.getValue(); - try { - PartitionReceiver receiver; - if (StringUtil.isNullOrWhiteSpace(offset)) { - throw new SamzaException( - String.format("Invalid offset %s namespace=%s, entity=%s", offset, namespace, entityPath)); - } - if (offset.equals(EventHubSystemConsumer.END_OF_STREAM)) { - receiver = ehClientWrapper.getEventHubClient() - .createReceiverSync(consumerName, id.toString(), Instant.now()); - } else { - receiver = ehClientWrapper.getEventHubClient() - .createReceiverSync(consumerName, id.toString(), offset, - !offset.equals(EventHubSystemConsumer.START_OF_STREAM)); - } - receiver.setReceiveHandler(handlers.get(id)); - receivers.put(id, receiver); - } catch (Exception e) { - throw new SamzaException( - String.format("Failed to create receiver for EventHubs: namespace=%s, entity=%s, partitionId=%d", - namespace, entityPath, id), e); - } - } - } catch (Exception e) { - throw new SamzaException( - String.format("Failed to create connection to EventHubs: namespace=%s, entity=%s", - namespace, entityPath), - e); - } - LOG.info(String.format("Connection successfully started for namespace=%s, entity=%s ", namespace, entityPath)); - } - - synchronized void stop() { - LOG.info(String.format("Stopping connection for namespace=%s, entity=%s ", namespace, entityPath)); - try { - for (PartitionReceiver receiver : receivers.values()) { - receiver.closeSync(); - } - ehClientWrapper.close(0); - } catch (ServiceBusException e) { - throw new SamzaException( - String.format("Failed to stop connection for namespace=%s, entity=%s ", namespace, entityPath), e); - } - isStarted = false; - offsets.clear(); - handlers.clear(); - LOG.info(String.format("Connection for namespace=%s, entity=%s stopped", namespace, entityPath)); - } -} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java deleted file mode 100644 index 900377fef3..0000000000 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubEntityConnectionFactory.java +++ /dev/null @@ -1,29 +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. -*/ - -package org.apache.samza.system.eventhub.consumer; - -import org.apache.samza.system.eventhub.EventHubConfig; - -public class EventHubEntityConnectionFactory { - EventHubEntityConnection createConnection(String namespace, String entityPath, String sasKeyName, String sasKey, - String consumerName, EventHubConfig config) { - return new EventHubEntityConnection(namespace, entityPath, sasKeyName, sasKey, consumerName, config); - } -} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index 1c009e03df..eedfa9cb71 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -30,6 +30,8 @@ import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.eventhub.EventDataWrapper; +import org.apache.samza.system.eventhub.EventHubClientWrapperFactory; +import org.apache.samza.system.eventhub.EventHubClientWrapper; import org.apache.samza.system.eventhub.EventHubConfig; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; import org.apache.samza.util.BlockingEnvelopeMap; @@ -38,11 +40,11 @@ import java.time.Duration; import java.time.Instant; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.*; import java.util.function.Function; import java.util.stream.Collectors; @@ -84,6 +86,8 @@ * └───────────────────────────────────────────────┘ */ public class EventHubSystemConsumer extends BlockingEnvelopeMap { + private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemConsumer.class); + private static final int MAX_EVENT_COUNT_PER_PARTITION_POLL = 50; //TODO public static final String START_OF_STREAM = PartitionReceiver.START_OF_STREAM; // -1 public static final String END_OF_STREAM = "-2"; @@ -92,41 +96,53 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { public static final String EVENT_BYTE_READ_RATE = "eventByteReadRate"; public static final String READ_LATENCY = "readLatency"; public static final String READ_ERRORS = "readErrors"; - private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemConsumer.class); - private static final int MAX_EVENT_COUNT_PER_PARTITION_POLL = 50; + private static Counter aggEventReadRate = null; private static Counter aggEventByteReadRate = null; private static SamzaHistogram aggReadLatency = null; private static Counter aggReadErrors = null; - private final Map connections = new HashMap<>(); - private final Map> serdes = new HashMap<>(); - private final EventHubConfig config; + private Map eventReadRates; private Map eventByteReadRates; private Map readLatencies; private Map readErrors; - public EventHubSystemConsumer(EventHubConfig config, EventHubEntityConnectionFactory connectionFactory, + final Map streamPartitionHandlers = new HashMap<>(); + private final Map streamPartitionReceivers = new HashMap<>(); + private final Map streamEventHubClients = new HashMap<>(); + private final Map streamPartitionStartingOffsets = new HashMap<>(); + private boolean isStarted = false; + private final EventHubConfig config; + + public EventHubSystemConsumer(EventHubConfig config, EventHubClientWrapperFactory eventHubClientWrapperFactory, MetricsRegistry registry) { super(registry, System::currentTimeMillis); this.config = config; List streamList = config.getStreamList(); - streamList.forEach(stream -> { - connections.put(stream, connectionFactory.createConnection( - config.getStreamNamespace(stream), config.getStreamEntityPath(stream), - config.getStreamSasKeyName(stream), config.getStreamSasToken(stream), - config.getStreamConsumerGroup(stream), config)); - serdes.put(stream, config.getSerde(stream).orElse(null)); - }); + + // Create and initiate connections to Event Hubs + for (String streamName : streamList) { + String namespace = config.getStreamNamespace(streamName); + String entityPath = config.getStreamEntityPath(streamName); + String sasKeyName = config.getStreamSasKeyName(streamName); + String sasKey = config.getStreamSasToken(streamName); + LOG.info(String.format("Starting connection for namespace=%s, entity=%s ", namespace, entityPath)); + EventHubClientWrapper ehClientWrapper = eventHubClientWrapperFactory + .getEventHubClientWrapper(namespace, entityPath, sasKeyName, sasKey, config); + streamEventHubClients.put(streamName, ehClientWrapper); + ehClientWrapper.init(); + } + + // Initiate metrics eventReadRates = streamList.stream() .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_READ_RATE))); eventByteReadRates = streamList.stream() .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_BYTE_READ_RATE))); readLatencies = streamList.stream() .collect(Collectors.toMap(Function.identity(), x -> new SamzaHistogram(registry, x, READ_LATENCY))); - readErrors = - streamList.stream().collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, READ_ERRORS))); + readErrors = streamList.stream() + .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, READ_ERRORS))); // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. synchronized (AGGREGATE) { @@ -139,13 +155,82 @@ public EventHubSystemConsumer(EventHubConfig config, EventHubEntityConnectionFac } } + /** + * {@inheritDoc} + */ + @Override + public synchronized void start() { + isStarted = true; + // Create receivers for Event Hubs + for (Map.Entry entry : streamPartitionStartingOffsets.entrySet()) { + SystemStreamPartition ssp = entry.getKey(); + String streamName = ssp.getStream(); + Integer partitionId = ssp.getPartition().getPartitionId(); + String offset = entry.getValue(); + String consumerGroup = config.getStreamConsumerGroup(streamName); + String namespace = config.getStreamNamespace(streamName); + String entityPath = config.getStreamEntityPath(streamName); + EventHubClientWrapper ehClientWrapper = streamEventHubClients.get(streamName); + try { + PartitionReceiver receiver; + if (StringUtil.isNullOrWhiteSpace(offset)) { + throw new SamzaException( + String.format("Invalid offset %s system=%s, stream=%s", offset, namespace, entityPath)); + } + if (offset.equals(EventHubSystemConsumer.END_OF_STREAM)) { + receiver = ehClientWrapper.getEventHubClient() + .createReceiverSync(consumerGroup, partitionId.toString(), Instant.now()); + } else { + receiver = ehClientWrapper.getEventHubClient() + .createReceiverSync(consumerGroup, partitionId.toString(), offset, + !offset.equals(EventHubSystemConsumer.START_OF_STREAM)); + } + PartitionReceiveHandler handler = new PartitionReceiverHandlerImpl(ssp, eventReadRates.get(streamName), + eventByteReadRates.get(streamName), readLatencies.get(streamName), readErrors.get(streamName), + config.getSerde(streamName).orElse(null)); + streamPartitionHandlers.put(ssp, handler); + receiver.setReceiveHandler(handler); + streamPartitionReceivers.put(ssp, receiver); + } catch (Exception e) { + throw new SamzaException( + String.format("Failed to create receiver for EventHubs: namespace=%s, entity=%s, partitionId=%d", + namespace, entityPath, partitionId), e); + } + LOG.info(String.format("Connection successfully started for namespace=%s, entity=%s ", namespace, entityPath)); + + } + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized void stop() { + LOG.info("Stopping event hub system consumer..."); + List> futures = new ArrayList<>(); + streamPartitionReceivers.values().forEach((receiver) -> futures.add(receiver.close())); + CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); + try { + future.get(config.getRuntimeInfoWaitTimeMS(), TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + throw new SamzaException("Failed to close receivers", e); + } + streamEventHubClients.values().forEach(ehClient -> ehClient.close(config.getShutdownWaitTimeMS())); + streamEventHubClients.clear(); + streamPartitionStartingOffsets.clear(); + streamPartitionReceivers.clear(); + } + + /** + * {@inheritDoc} + */ @Override - public void register(SystemStreamPartition systemStreamPartition, String offset) { + public synchronized void register(SystemStreamPartition systemStreamPartition, String offset) { super.register(systemStreamPartition, offset); - String stream = systemStreamPartition.getStream(); - EventHubEntityConnection connection = connections.get(stream); - if (connection == null) { - throw new SamzaException("No EventHub connection for " + stream); + + if (isStarted) { + LOG.warn("Trying to add partition when the connection has already started."); + return; } if (StringUtil.isNullOrWhiteSpace(offset)) { @@ -161,20 +246,7 @@ public void register(SystemStreamPartition systemStreamPartition, String offset) "Unknown starting position config " + config.getStartPosition(systemStreamPartition.getStream())); } } - connection.addPartition(systemStreamPartition.getPartition().getPartitionId(), offset, - new PartitionReceiverHandlerImpl(systemStreamPartition, eventReadRates.get(stream), - eventByteReadRates.get(stream), readLatencies.get(stream), readErrors.get(stream), - serdes.get(stream))); - } - - @Override - public void start() { - connections.values().forEach(EventHubEntityConnection::connectAndStart); - } - - @Override - public void stop() { - connections.values().forEach(EventHubEntityConnection::stop); + streamPartitionStartingOffsets.put(systemStreamPartition, offset); } @Override @@ -227,7 +299,7 @@ public void onReceive(Iterable events) { } private void updateMetrics(EventData event) { - int eventDataLength = event.getBytes() == null ? 0 : event.getBytes().length; + int eventDataLength = event.getBytes() == null ? 0 : event.getBytes().length; eventReadRate.inc(); aggEventReadRate.inc(); eventByteReadRate.inc(eventDataLength); @@ -239,7 +311,6 @@ private void updateMetrics(EventData event) { @Override public void onError(Throwable throwable) { - // TODO error handling errors.inc(); aggReadErrors.inc(); LOG.error(String.format("Received error from event hub connection (ssp=%s): ", ssp), throwable); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 6fb1266cc2..4f69fd2bad 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -29,9 +29,9 @@ import org.apache.samza.serializers.Serde; import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemProducer; -import org.apache.samza.system.eventhub.EventHubClientFactory; import org.apache.samza.system.eventhub.EventHubClientWrapper; import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.EventHubClientWrapperFactory; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,7 +66,7 @@ public class EventHubSystemProducer implements SystemProducer { private HashMap sendCallbackLatency = new HashMap<>(); private HashMap sendErrors = new HashMap<>(); - private final EventHubClientFactory eventHubClientFactory = new EventHubClientFactory(); + private final EventHubClientWrapperFactory eventHubClientWrapperFactory; private final EventHubConfig config; private final String systemName; private final MetricsRegistry registry; @@ -86,12 +86,14 @@ public class EventHubSystemProducer implements SystemProducer { private Map> pendingFutures = new ConcurrentHashMap<>(); - public EventHubSystemProducer(String systemName, EventHubConfig config, MetricsRegistry registry) { + public EventHubSystemProducer(String systemName, EventHubConfig config, EventHubClientWrapperFactory eventHubClientWrapperFactory, + MetricsRegistry registry) { messageId = 0; this.systemName = systemName; this.config = config; this.registry = registry; partitioningMethod = this.config.getPartitioningMethod(); + this.eventHubClientWrapperFactory = eventHubClientWrapperFactory; } @Override @@ -121,7 +123,7 @@ public synchronized void start() { @Override public synchronized void stop() { - LOG.info("Stopping system producer."); + LOG.info("Stopping event hub system producer..."); streamPartitionSenders.values().forEach((streamPartitionSender) -> { List> futures = new ArrayList<>(); streamPartitionSender.forEach((key, value) -> futures.add(value.close())); @@ -129,7 +131,7 @@ public synchronized void stop() { try { future.get(config.getShutdownWaitTimeMS(), TimeUnit.MILLISECONDS); } catch (ExecutionException | InterruptedException | TimeoutException e) { - LOG.warn("Closing the partition sender failed ", e); + throw new SamzaException("Closing the partition sender failed ", e); } }); eventHubClients.values().forEach(ehClient -> ehClient.close(config.getShutdownWaitTimeMS())); @@ -145,8 +147,8 @@ public synchronized void register(String streamName) { throw new SamzaException(msg); } - EventHubClientWrapper ehClient = eventHubClientFactory - .getEventHubClient(config.getStreamNamespace(streamName), config.getStreamEntityPath(streamName), + EventHubClientWrapper ehClient = eventHubClientWrapperFactory + .getEventHubClientWrapper(config.getStreamNamespace(streamName), config.getStreamEntityPath(streamName), config.getStreamSasKeyName(streamName), config.getStreamSasToken(streamName), config); ehClient.init(); @@ -185,8 +187,8 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo Instant startTime = Instant.now(); - CompletableFuture sendResult; - sendResult = sendToEventHub(destination, eventData, envelope.getPartitionKey(), ehClient.getEventHubClient()); + CompletableFuture sendResult = sendToEventHub(destination, eventData, envelope.getPartitionKey(), + ehClient.getEventHubClient()); Instant endTime = Instant.now(); long latencyMs = Duration.between(startTime, endTime).toMillis(); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java index 7cb6ebdb64..5d1f5c1074 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java @@ -19,7 +19,6 @@ package org.apache.samza.system.eventhub.admin; -import junit.framework.Assert; import org.apache.samza.Partition; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemStreamMetadata; @@ -28,6 +27,7 @@ import org.apache.samza.system.eventhub.MockEventHubConfigFactory; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; +import org.junit.Assert; import org.junit.Test; import java.util.HashMap; diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubClientFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubClientFactory.java new file mode 100644 index 0000000000..1b5eeaf035 --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubClientFactory.java @@ -0,0 +1,109 @@ +/* +* 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.samza.system.eventhub.consumer; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.eventhubs.PartitionReceiveHandler; +import com.microsoft.azure.eventhubs.PartitionReceiver; +import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.system.eventhub.EventHubClientWrapper; +import org.apache.samza.system.eventhub.EventHubClientWrapperFactory; +import org.apache.samza.system.eventhub.EventHubConfig; +import org.junit.Assert; +import org.mockito.stubbing.Answer; +import org.powermock.api.mockito.PowerMockito; + +import java.time.Instant; +import java.util.List; +import java.util.Map; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyString; + +public class MockEventHubClientFactory extends EventHubClientWrapperFactory { + private Map> eventData; + + MockEventHubClientFactory(Map> eventData) { + this.eventData = eventData; + } + + @Override + public EventHubClientWrapper getEventHubClientWrapper(String eventHubNamespace, String entityPath, String sasKeyName, + String sasToken, EventHubConfig config) { + return new MockEventHubClientWrapper(); + } + + // Emulate EventHub sending data + void sendToHandlers(Map handlers) { + handlers.forEach((ssp, value) -> value.onReceive(eventData.get(ssp))); + } + + private class MockEventHubClientWrapper implements EventHubClientWrapper { + Boolean initiated = false; + EventHubClient mockEventHubClient = PowerMockito.mock(EventHubClient.class); + + MockEventHubClientWrapper() { + PartitionReceiver mockPartitionReceiver = PowerMockito.mock(PartitionReceiver.class); + + // Set mocks + PowerMockito.when(mockPartitionReceiver.setReceiveHandler(any())).then((Answer) invocationOnMock -> { + PartitionReceiveHandler handler = invocationOnMock.getArgumentAt(0, PartitionReceiveHandler.class); + if (handler == null) { + Assert.fail("Handler for setReceiverHandler was null"); + } + return null; + }); + + try { + PowerMockito.when(mockEventHubClient.createReceiverSync(anyString(), anyString(), any(Instant.class))) + .thenReturn(mockPartitionReceiver); + PowerMockito.when(mockEventHubClient.createReceiverSync(anyString(), anyString(), anyString(), anyBoolean())) + .thenReturn(mockPartitionReceiver); + } catch (Exception e) { + Assert.fail("Cannot create mockReceiverSync"); + } + } + + @Override + public void init() { + initiated = true; + } + + @Override + public EventHubClient getEventHubClient() { + if (!initiated) { + Assert.fail("Should have called init() on EventHubClient before getEventHubClientWrapper()"); + } + return mockEventHubClient; + } + + @Override + public void close(long timeoutMS) { + if (!initiated) { + Assert.fail("Should have called init() on EventHubClient before close()"); + } + initiated = false; + } + + } + +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java deleted file mode 100644 index 3f64b2b989..0000000000 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubEntityConnectionFactory.java +++ /dev/null @@ -1,68 +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. -*/ - -package org.apache.samza.system.eventhub.consumer; - -import com.microsoft.azure.eventhubs.EventData; -import com.microsoft.azure.eventhubs.PartitionReceiveHandler; -import org.apache.samza.system.eventhub.EventHubConfig; - -import java.util.List; -import java.util.Map; - -class MockEventHubEntityConnectionFactory extends EventHubEntityConnectionFactory { - - private final Map>> eventData; - - MockEventHubEntityConnectionFactory(Map>> eventData) { - this.eventData = eventData; - } - - @Override - EventHubEntityConnection createConnection(String namespace, String entityPath, String sasKeyName, String sasKey, - String consumerName, EventHubConfig eventHubConfig) { - return new MockEventHubEntityConnection(entityPath, eventData.get(entityPath)); - } - - private class MockEventHubEntityConnection extends EventHubEntityConnection { - private final Map> eventData; - - MockEventHubEntityConnection(String entity, Map> eventData) { - super(null, entity, null, null, null, null); - assert eventData != null : "No event data found for entity:" + entity; - this.eventData = eventData; - } - - @Override - void connectAndStart() { - offsets.keySet().forEach(partitionId -> { - List events = eventData.get(partitionId); - PartitionReceiveHandler partitionHandler = handlers.get(partitionId); - assert events != null : String.format("partition %d not found", partitionId); - assert partitionHandler != null : String.format("handler %d not registered", partitionId); - partitionHandler.onReceive(events); - }); - } - - @Override - void stop() { - // do nothing - } - } -} diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index 2df4bc9e18..6887d3035e 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -21,21 +21,27 @@ import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.eventhubs.PartitionReceiver; import org.apache.samza.Partition; import org.apache.samza.metrics.Counter; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.system.eventhub.EventDataWrapper; -import org.apache.samza.system.eventhub.EventHubConfig; -import org.apache.samza.system.eventhub.MockEventData; -import org.apache.samza.system.eventhub.TestMetricsRegistry; +import org.apache.samza.system.eventhub.*; import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; import java.util.*; import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.*; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({PartitionReceiver.class, EventHubClient.class}) public class TestEventHubSystemConsumer { private static final String MOCK_ENTITY_1 = "mocktopic1"; private static final String MOCK_ENTITY_2 = "mocktopic2"; @@ -59,25 +65,34 @@ public void testSinglePartitionConsumptionHappyPath() throws Exception { int partitionId = 0; TestMetricsRegistry testMetrics = new TestMetricsRegistry(); - Map>> eventData = new HashMap<>(); - Map> singleTopicEventData = new HashMap<>(); + Map> eventData = new HashMap<>(); + SystemStreamPartition ssp = new SystemStreamPartition(systemName, streamName, new Partition(partitionId)); + // create EventData List singlePartitionEventData = MockEventData.generateEventData(numEvents); - singleTopicEventData.put(partitionId, singlePartitionEventData); - eventData.put(MOCK_ENTITY_1, singleTopicEventData); - EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); + eventData.put(ssp, singlePartitionEventData); + // Set configs Map configMap = new HashMap<>(); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), streamName); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, systemName, streamName), EVENTHUB_NAMESPACE); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "latest"); - EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); - SystemStreamPartition ssp = new SystemStreamPartition(systemName, streamName, new Partition(partitionId)); + MockEventHubClientFactory eventHubClientWrapperFactory = new MockEventHubClientFactory(eventData); + + EventHubSystemConsumer consumer = + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), eventHubClientWrapperFactory, testMetrics); consumer.register(ssp, null); consumer.start(); + + // Mock received data from EventHub + eventHubClientWrapperFactory.sendToHandlers(consumer.streamPartitionHandlers); + List result = consumer.poll(Collections.singleton(ssp), 1000).get(ssp); + verifyEvents(result, singlePartitionEventData); Assert.assertEquals(testMetrics.getCounters(streamName).size(), 3); Assert.assertEquals(testMetrics.getGauges(streamName).size(), 2); @@ -96,27 +111,36 @@ public void testMultiPartitionConsumptionHappyPath() throws Exception { int partitionId1 = 0; int partitionId2 = 1; TestMetricsRegistry testMetrics = new TestMetricsRegistry(); + Map> eventData = new HashMap<>(); + SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId1)); + SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId2)); - Map>> eventData = new HashMap<>(); - Map> singleTopicEventData = new HashMap<>(); + // create EventData List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); List singlePartitionEventData2 = MockEventData.generateEventData(numEvents); - singleTopicEventData.put(partitionId1, singlePartitionEventData1); - singleTopicEventData.put(partitionId2, singlePartitionEventData2); - eventData.put(MOCK_ENTITY_1, singleTopicEventData); - EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); + eventData.put(ssp1, singlePartitionEventData1); + eventData.put(ssp2, singlePartitionEventData2); + // Set configs Map configMap = new HashMap<>(); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), streamName); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); - EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, systemName, streamName), EVENTHUB_NAMESPACE); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); - SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId1)); + MockEventHubClientFactory eventHubClientWrapperFactory = new MockEventHubClientFactory(eventData); + + EventHubSystemConsumer consumer = + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), eventHubClientWrapperFactory, + testMetrics); consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); - SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId2)); consumer.register(ssp2, EventHubSystemConsumer.START_OF_STREAM); consumer.start(); + + // Mock received data from EventHub + eventHubClientWrapperFactory.sendToHandlers(consumer.streamPartitionHandlers); + Set ssps = new HashSet<>(); ssps.add(ssp1); ssps.add(ssp2); @@ -141,31 +165,40 @@ public void testMultiStreamsConsumptionHappyPath() throws Exception { int numEvents = 10; // needs to be less than BLOCKING_QUEUE_SIZE int partitionId = 0; TestMetricsRegistry testMetrics = new TestMetricsRegistry(); + Map> eventData = new HashMap<>(); + SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName1, new Partition(partitionId)); + SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName2, new Partition(partitionId)); - Map>> eventData = new HashMap<>(); - Map> singleTopicEventData1 = new HashMap<>(); List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); - singleTopicEventData1.put(partitionId, singlePartitionEventData1); - eventData.put(MOCK_ENTITY_1, singleTopicEventData1); - Map> singleTopicEventData2 = new HashMap<>(); List singlePartitionEventData2 = MockEventData.generateEventData(numEvents); - singleTopicEventData2.put(partitionId, singlePartitionEventData2); - eventData.put(MOCK_ENTITY_2, singleTopicEventData2); - EventHubEntityConnectionFactory connectionFactory = new MockEventHubEntityConnectionFactory(eventData); + eventData.put(ssp1, singlePartitionEventData1); + eventData.put(ssp2, singlePartitionEventData2); Map configMap = new HashMap<>(); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), String.format("%s,%s", streamName1, streamName2)); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName1), MOCK_ENTITY_1); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, systemName, streamName1), EVENTHUB_NAMESPACE); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName1), EVENTHUB_KEY_NAME); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName1), EVENTHUB_KEY); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName2), MOCK_ENTITY_2); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, systemName, streamName2), EVENTHUB_NAMESPACE); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName2), EVENTHUB_KEY_NAME); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName2), EVENTHUB_KEY); + + MockEventHubClientFactory eventHubClientWrapperFactory = new MockEventHubClientFactory(eventData); + EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), connectionFactory, testMetrics); + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), eventHubClientWrapperFactory, + testMetrics); - SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName1, new Partition(partitionId)); consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); - SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName2, new Partition(partitionId)); consumer.register(ssp2, EventHubSystemConsumer.START_OF_STREAM); consumer.start(); + + // Mock received data from EventHub + eventHubClientWrapperFactory.sendToHandlers(consumer.streamPartitionHandlers); + Set ssps = new HashSet<>(); ssps.add(ssp1); ssps.add(ssp2); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java index 1d9afd6361..fe91d5a437 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -93,9 +93,9 @@ public void testSend() { @Test public void testReceive() throws ServiceBusException { - EventHubClientFactory clientFactory = new EventHubClientFactory(); + EventHubClientWrapperFactory clientFactory = new EventHubClientWrapperFactory(); EventHubClientWrapper wrapper = clientFactory - .getEventHubClient(EVENTHUB_NAMESPACE, EVENTHUB_ENTITY1, EVENTHUB_KEY_NAME, EVENTHUB_KEY, + .getEventHubClientWrapper(EVENTHUB_NAMESPACE, EVENTHUB_ENTITY1, EVENTHUB_KEY_NAME, EVENTHUB_KEY, new EventHubConfig(createEventHubConfig(), SYSTEM_NAME)); wrapper.init(); EventHubClient client = wrapper.getEventHubClient(); From bfb48b78aeac2f4cd77a9a63f1d953accb7f45d3 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 9 Oct 2017 14:12:38 -0700 Subject: [PATCH 16/34] ignore intergration test --- .../samza/system/eventhub/admin/TestEventHubSystemAdmin.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java index 5d1f5c1074..bc34ea7c08 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java @@ -28,6 +28,7 @@ import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; import java.util.HashMap; @@ -78,6 +79,7 @@ public void testGetNextOffset() { Assert.assertEquals("0", updatedOffsets.get(ssp2)); } + @Ignore("Integration Test") @Test public void testGetStreamMetadata() { EventHubSystemFactory eventHubSystemFactory = new EventHubSystemFactory(); From 097f355e6204c72c80db3641c0688be2162e4f55 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 9 Oct 2017 14:58:53 -0700 Subject: [PATCH 17/34] compatibility with upstream changes --- .../eventhub/admin/EventHubSystemAdmin.java | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index 843cfc3f83..ffff040c94 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -149,21 +149,6 @@ private Map getPartitionMetadata(Strin return sspMetadataMap; } - @Override - public void createChangelogStream(String streamName, int numOfPartitions) { - throw new UnsupportedOperationException("Event Hubs does not support change log stream."); - } - - @Override - public void validateChangelogStream(String streamName, int numOfPartitions) { - throw new UnsupportedOperationException("Event Hubs does not support change log stream."); - } - - @Override - public void createCoordinatorStream(String streamName) { - throw new UnsupportedOperationException("Event Hubs does not support coordinator stream."); - } - @Override public Integer offsetComparator(String offset1, String offset2) { try { From 41016b7a509e24fe853c00296022dd441018c251 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 9 Oct 2017 15:47:11 -0700 Subject: [PATCH 18/34] moved per stream Serde setting to EHSystemFactory --- .../samza/system/eventhub/EventHubConfig.java | 17 ---------- .../eventhub/EventHubSystemFactory.java | 31 +++++++++---------- .../consumer/EventHubSystemConsumer.java | 6 ++-- .../producer/EventHubSystemProducer.java | 10 +++--- .../consumer/TestEventHubSystemConsumer.java | 16 ++++++++-- 5 files changed, 36 insertions(+), 44 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java index cb58ef7fd7..e9d671e501 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -20,18 +20,14 @@ package org.apache.samza.system.eventhub; import com.microsoft.azure.eventhubs.EventHubClient; -import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; import org.apache.samza.config.MapConfig; -import org.apache.samza.serializers.Serde; -import org.apache.samza.serializers.SerdeFactory; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; import java.time.Duration; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; public class EventHubConfig extends MapConfig { public static final String CONFIG_STREAM_LIST = "systems.%s.stream.list"; @@ -44,9 +40,6 @@ public class EventHubConfig extends MapConfig { public static final String CONFIG_STREAM_SAS_TOKEN = "systems.%s.streams.%s.eventhubs.sas.token"; - public static final String CONFIG_STREAM_SERDE_FACTORY = "systems.%s.streams.%s.eventhubs.serdeFactory"; - public static final String CONFIG_STREAM_SERDE_PREFIX = "systems.%s.streams.%s.eventhubs.serde."; - public static final String CONFIG_STREAM_CONSUMER_GROUP = "systems.%s.streams.%s.eventhubs.consumer.group"; public static final String DEFAULT_CONFIG_STREAM_CONSUMER_GROUP = EventHubClient.DEFAULT_CONSUMER_GROUP_NAME; @@ -135,16 +128,6 @@ public String getStreamSasToken(String streamName) { return getRequiredConfigValue(CONFIG_STREAM_SAS_TOKEN, streamName); } - public Optional> getSerde(String streamName) { - Serde serde = null; - String serdeFactoryClassName = this.get(String.format(CONFIG_STREAM_SERDE_FACTORY, systemName, streamName)); - if (!StringUtils.isEmpty(serdeFactoryClassName)) { - SerdeFactory factory = EventHubSystemFactory.getSerdeFactory(serdeFactoryClassName); - serde = factory.getSerde(streamName, this.subset(String.format(CONFIG_STREAM_SERDE_PREFIX, systemName, streamName))); - } - return Optional.ofNullable(serde); - } - /** * Get the EventHubs consumer group used for consumption for the stream * diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java index 5b15fc36fb..3fba5afddf 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java @@ -19,10 +19,10 @@ package org.apache.samza.system.eventhub; -import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.metrics.MetricsRegistry; -import org.apache.samza.serializers.SerdeFactory; +import org.apache.samza.serializers.ByteSerde; +import org.apache.samza.serializers.Serde; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemConsumer; import org.apache.samza.system.SystemFactory; @@ -31,31 +31,30 @@ import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; -import java.lang.reflect.Constructor; +import java.util.HashMap; +import java.util.List; +import java.util.Map; public class EventHubSystemFactory implements SystemFactory { - @SuppressWarnings("unchecked") - public static SerdeFactory getSerdeFactory(String serdeFactoryClassName) { - SerdeFactory factory; - try { - Class> classObj = (Class>) Class.forName(serdeFactoryClassName); - Constructor> ctor = classObj.getDeclaredConstructor(); - factory = ctor.newInstance(); - } catch (Exception e) { - throw new SamzaException("Failed to create Serde Factory for: " + serdeFactoryClassName, e); - } - return factory; + private Map> getSerdesMap(EventHubConfig config) { + Map> serdes = new HashMap<>(); + List streamList = config.getStreamList(); + streamList.forEach((streamName) -> serdes.put(streamName, new ByteSerde())); + return serdes; } @Override public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { - return new EventHubSystemConsumer(new EventHubConfig(config, systemName), new EventHubClientWrapperFactory(), registry); + EventHubConfig eventHubConfig = new EventHubConfig(config, systemName); + return new EventHubSystemConsumer(eventHubConfig, new EventHubClientWrapperFactory(), getSerdesMap(eventHubConfig), + registry); } @Override public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { - return new EventHubSystemProducer(systemName, new EventHubConfig(config, systemName), new EventHubClientWrapperFactory(), + EventHubConfig eventHubConfig = new EventHubConfig(config, systemName); + return new EventHubSystemProducer(eventHubConfig, new EventHubClientWrapperFactory(), getSerdesMap(eventHubConfig), registry); } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index eedfa9cb71..adf2d2bafa 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -111,14 +111,16 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { private final Map streamPartitionReceivers = new HashMap<>(); private final Map streamEventHubClients = new HashMap<>(); private final Map streamPartitionStartingOffsets = new HashMap<>(); + private Map> serdes; private boolean isStarted = false; private final EventHubConfig config; public EventHubSystemConsumer(EventHubConfig config, EventHubClientWrapperFactory eventHubClientWrapperFactory, - MetricsRegistry registry) { + Map> serdes, MetricsRegistry registry) { super(registry, System::currentTimeMillis); this.config = config; + this.serdes = serdes; List streamList = config.getStreamList(); // Create and initiate connections to Event Hubs @@ -187,7 +189,7 @@ public synchronized void start() { } PartitionReceiveHandler handler = new PartitionReceiverHandlerImpl(ssp, eventReadRates.get(streamName), eventByteReadRates.get(streamName), readLatencies.get(streamName), readErrors.get(streamName), - config.getSerde(streamName).orElse(null)); + serdes.getOrDefault(streamName, null)); streamPartitionHandlers.put(ssp, handler); receiver.setReceiveHandler(handler); streamPartitionReceivers.put(ssp, receiver); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 4f69fd2bad..b3144f0560 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -68,7 +68,6 @@ public class EventHubSystemProducer implements SystemProducer { private final EventHubClientWrapperFactory eventHubClientWrapperFactory; private final EventHubConfig config; - private final String systemName; private final MetricsRegistry registry; private final PartitioningMethod partitioningMethod; @@ -82,18 +81,18 @@ public class EventHubSystemProducer implements SystemProducer { // Running count for the next message Id private long messageId; - private Map> serdes = new HashMap<>(); + private Map> serdes; private Map> pendingFutures = new ConcurrentHashMap<>(); - public EventHubSystemProducer(String systemName, EventHubConfig config, EventHubClientWrapperFactory eventHubClientWrapperFactory, - MetricsRegistry registry) { + public EventHubSystemProducer(EventHubConfig config, EventHubClientWrapperFactory eventHubClientWrapperFactory, + Map> serdes, MetricsRegistry registry) { messageId = 0; - this.systemName = systemName; this.config = config; this.registry = registry; partitioningMethod = this.config.getPartitioningMethod(); this.eventHubClientWrapperFactory = eventHubClientWrapperFactory; + this.serdes = serdes; } @Override @@ -154,7 +153,6 @@ public synchronized void register(String streamName) { ehClient.init(); eventHubClients.put(streamName, ehClient); streamPartitionSenders.put(streamName, new HashMap<>()); - config.getSerde(streamName).ifPresent(x -> serdes.put(streamName, x)); } @Override diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index 6887d3035e..554ef78202 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -25,6 +25,8 @@ import com.microsoft.azure.eventhubs.PartitionReceiver; import org.apache.samza.Partition; import org.apache.samza.metrics.Counter; +import org.apache.samza.serializers.ByteSerde; +import org.apache.samza.serializers.Serde; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.eventhub.*; @@ -67,6 +69,8 @@ public void testSinglePartitionConsumptionHappyPath() throws Exception { TestMetricsRegistry testMetrics = new TestMetricsRegistry(); Map> eventData = new HashMap<>(); SystemStreamPartition ssp = new SystemStreamPartition(systemName, streamName, new Partition(partitionId)); + Map> serdes = new HashMap<>(); + serdes.put(streamName, new ByteSerde()); // create EventData List singlePartitionEventData = MockEventData.generateEventData(numEvents); @@ -84,7 +88,8 @@ public void testSinglePartitionConsumptionHappyPath() throws Exception { MockEventHubClientFactory eventHubClientWrapperFactory = new MockEventHubClientFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), eventHubClientWrapperFactory, testMetrics); + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), eventHubClientWrapperFactory, serdes, + testMetrics); consumer.register(ssp, null); consumer.start(); @@ -114,6 +119,8 @@ public void testMultiPartitionConsumptionHappyPath() throws Exception { Map> eventData = new HashMap<>(); SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId1)); SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId2)); + Map> serdes = new HashMap<>(); + serdes.put(streamName, new ByteSerde()); // create EventData List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); @@ -132,7 +139,7 @@ public void testMultiPartitionConsumptionHappyPath() throws Exception { MockEventHubClientFactory eventHubClientWrapperFactory = new MockEventHubClientFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), eventHubClientWrapperFactory, + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), eventHubClientWrapperFactory, serdes, testMetrics); consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); consumer.register(ssp2, EventHubSystemConsumer.START_OF_STREAM); @@ -168,6 +175,9 @@ public void testMultiStreamsConsumptionHappyPath() throws Exception { Map> eventData = new HashMap<>(); SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName1, new Partition(partitionId)); SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName2, new Partition(partitionId)); + Map> serdes = new HashMap<>(); + serdes.put(streamName1, new ByteSerde()); + serdes.put(streamName2, new ByteSerde()); List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); List singlePartitionEventData2 = MockEventData.generateEventData(numEvents); @@ -189,7 +199,7 @@ public void testMultiStreamsConsumptionHappyPath() throws Exception { MockEventHubClientFactory eventHubClientWrapperFactory = new MockEventHubClientFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), eventHubClientWrapperFactory, + new EventHubSystemConsumer(new EventHubConfig(configMap, systemName), eventHubClientWrapperFactory, serdes, testMetrics); consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); From 61e3dbb24e505514fae0a5e30b44cc7cb6d7b721 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 9 Oct 2017 16:08:54 -0700 Subject: [PATCH 19/34] extract method of partition key determination --- .../system/eventhub/producer/EventHubSystemProducer.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index b3144f0560..13a6e619d0 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -185,7 +185,7 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo Instant startTime = Instant.now(); - CompletableFuture sendResult = sendToEventHub(destination, eventData, envelope.getPartitionKey(), + CompletableFuture sendResult = sendToEventHub(destination, eventData, getEnvelopePartitionId(envelope), ehClient.getEventHubClient()); Instant endTime = Instant.now(); @@ -236,6 +236,10 @@ private CompletableFuture sendToEventHub(String streamName, EventData even } } + private Object getEnvelopePartitionId(OutgoingMessageEnvelope envelope) { + return envelope.getPartitionKey(); + } + private String convertPartitionKeyToString(Object partitionKey) { if (partitionKey instanceof String) { return (String) partitionKey; From 10399287b2017ab026681bef976cae07ad06ae78 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Wed, 11 Oct 2017 11:00:37 -0700 Subject: [PATCH 20/34] Consistent timeout results --- .../system/eventhub/consumer/EventHubSystemConsumer.java | 8 +++++--- .../system/eventhub/producer/EventHubSystemProducer.java | 6 ++++-- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index adf2d2bafa..f84692bb20 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -213,9 +213,11 @@ public synchronized void stop() { streamPartitionReceivers.values().forEach((receiver) -> futures.add(receiver.close())); CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); try { - future.get(config.getRuntimeInfoWaitTimeMS(), TimeUnit.MILLISECONDS); - } catch (ExecutionException | InterruptedException | TimeoutException e) { - throw new SamzaException("Failed to close receivers", e); + future.get(config.getShutdownWaitTimeMS(), TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException e) { + LOG.warn("Failed to close receivers", e); + } catch (TimeoutException e) { + LOG.warn("Closing the partition sender timed out ", e); } streamEventHubClients.values().forEach(ehClient -> ehClient.close(config.getShutdownWaitTimeMS())); streamEventHubClients.clear(); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 13a6e619d0..9c369fe5d5 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -129,8 +129,10 @@ public synchronized void stop() { CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); try { future.get(config.getShutdownWaitTimeMS(), TimeUnit.MILLISECONDS); - } catch (ExecutionException | InterruptedException | TimeoutException e) { - throw new SamzaException("Closing the partition sender failed ", e); + } catch (ExecutionException | InterruptedException e) { + LOG.warn("Closing the partition sender failed ", e); + } catch (TimeoutException e) { + LOG.warn("Closing the partition sender timed out ", e); } }); eventHubClients.values().forEach(ehClient -> ehClient.close(config.getShutdownWaitTimeMS())); From e3e21f65cc287e7d2843fd4e7840893a5de486bb Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Thu, 12 Oct 2017 17:08:07 -0700 Subject: [PATCH 21/34] Simple pr comments addressed --- build.gradle | 3 +- gradle/wrapper/gradle-wrapper.properties | 4 +- .../system/eventhub/EventDataWrapper.java | 7 +- .../samza/system/eventhub/EventHubConfig.java | 100 ++++------- .../eventhub/EventHubSystemFactory.java | 17 +- ...tWrapper.java => SamzaEventHubClient.java} | 13 +- ...y.java => SamzaEventHubClientFactory.java} | 13 +- ...pper.java => SamzaEventHubClientImpl.java} | 34 ++-- .../eventhub/admin/EventHubSystemAdmin.java | 93 +++++----- .../consumer/EventHubSystemConsumer.java | 148 +++++++-------- .../eventhub/metrics/SamzaHistogram.java | 15 +- .../producer/EventHubSystemProducer.java | 168 +++++++++--------- .../eventhub/MockEventHubConfigFactory.java | 6 +- ...va => MockSamzaEventHubClientFactory.java} | 22 +-- .../consumer/TestEventHubSystemConsumer.java | 14 +- ...TestEventHubSystemConsumerIntegration.java | 77 ++++++++ ...estEventHubSystemProducerIntegration.java} | 15 +- 17 files changed, 392 insertions(+), 357 deletions(-) rename samza-azure/src/main/java/org/apache/samza/system/eventhub/{EventHubClientWrapper.java => SamzaEventHubClient.java} (79%) rename samza-azure/src/main/java/org/apache/samza/system/eventhub/{EventHubClientWrapperFactory.java => SamzaEventHubClientFactory.java} (60%) rename samza-azure/src/main/java/org/apache/samza/system/eventhub/{SamzaEventHubClientWrapper.java => SamzaEventHubClientImpl.java} (69%) rename samza-azure/src/test/java/org/apache/samza/system/eventhub/{consumer/MockEventHubClientFactory.java => MockSamzaEventHubClientFactory.java} (77%) create mode 100644 samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumerIntegration.java rename samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/{TestEventHubSystemProducer.java => TestEventHubSystemProducerIntegration.java} (91%) diff --git a/build.gradle b/build.gradle index 928007af94..c2bb5ecfdd 100644 --- a/build.gradle +++ b/build.gradle @@ -209,7 +209,8 @@ project(':samza-azure') { toolVersion = "$checkstyleVersion" } test { - exclude 'org/apache/samza/system/eventhub/producer/**' + exclude 'org/apache/samza/system/eventhub/producer/*Integration*' + exclude 'org/apache/samza/system/eventhub/consumer/*Integration*' } } diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index f742bf3ea0..55720c38e2 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,6 @@ -#Thu Sep 07 09:55:54 PDT 2017 +#Mon Oct 31 23:13:44 PDT 2016 distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-2.8-all.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-2.8-bin.zip diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java index 08880ba197..1c2993673e 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java @@ -41,13 +41,8 @@ public EventData getEventData() { * @return the body of decrypted body of the message. In case no encryption is setup for this topic, * just returns the body of the message. */ - public byte[] getDecryptedBody() { + public byte[] getBody() { return body; } - @Override - public String toString() { - return "EventDataWrapper: body: " + (new String(body)) + ", EventData " + eventData; - } - } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java index e9d671e501..b3b0d9c296 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -20,16 +20,20 @@ package org.apache.samza.system.eventhub; import com.microsoft.azure.eventhubs.EventHubClient; -import org.apache.samza.SamzaException; import org.apache.samza.config.MapConfig; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; import java.time.Duration; import java.util.List; import java.util.Map; -import java.util.Objects; public class EventHubConfig extends MapConfig { + + public enum StartPosition { + EARLIEST, + LATEST + } + public static final String CONFIG_STREAM_LIST = "systems.%s.stream.list"; public static final String CONFIG_STREAM_NAMESPACE = "systems.%s.streams.%s.eventhubs.namespace"; @@ -53,9 +57,6 @@ public class EventHubConfig extends MapConfig { public static final String CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = "systems.%s.eventhubs.send.key"; public static final Boolean DEFAULT_CONFIG_SEND_KEY_IN_EVENT_PROPERTIES = false; - public static final String CONFIG_CONNECTION_SHUTDOWN_TIMEOUT_MILLIS = "systems.%s.eventhubs.shutdown.timeout"; - public static final long DEFAULT_CONFIG_CONNECTION_SHUTDOWN_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); - public static final String CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS = "systems.%s.eventhubs.runtime.info.timeout"; public static final long DEFAULT_CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); @@ -63,88 +64,84 @@ public class EventHubConfig extends MapConfig { public static final int DEFAULT_CONFIG_CONSUMER_BUFFER_CAPACITY = 100; - private final String systemName; - - public EventHubConfig(Map config, String systemName) { + public EventHubConfig(Map config) { super(config); - this.systemName = systemName; - } - - private String getRequiredConfigValue(String configKey, String streamName) { - String configValue = get(String.format(configKey, systemName, streamName), null); - if (configValue == null) { - throw new SamzaException(configKey + " is not configured."); - } - return configValue; } /** * Get the list of streams that are defined. Each stream has enough * information for connecting to a certain EventHub entity. * + * @param systemName name of the system * @return list of stream names */ - public List getStreamList() { + public List getStreams(String systemName) { return getList(String.format(CONFIG_STREAM_LIST, systemName)); } /** * Get the EventHubs namespace for the stream * + * @param systemName name of the system * @param streamName name of stream * @return EventHubs namespace */ - public String getStreamNamespace(String streamName) { - return getRequiredConfigValue(CONFIG_STREAM_NAMESPACE, streamName); + public String getStreamNamespace(String systemName, String streamName) { + return get(String.format(CONFIG_STREAM_NAMESPACE, systemName, streamName)); } /** * Get the EventHubs entity path (topic name) for the stream * + * @param systemName name of the system * @param streamName name of stream * @return EventHubs entity path */ - public String getStreamEntityPath(String streamName) { - return getRequiredConfigValue(CONFIG_STREAM_ENTITYPATH, streamName); + public String getStreamEntityPath(String systemName, String streamName) { + return get(String.format(CONFIG_STREAM_ENTITYPATH, systemName, streamName)); } /** * Get the EventHubs SAS (Shared Access Signature) key name for the stream * + * @param systemName name of the system * @param streamName name of stream * @return EventHubs SAS key name */ - public String getStreamSasKeyName(String streamName) { - return getRequiredConfigValue(CONFIG_STREAM_SAS_KEY_NAME, streamName); + public String getStreamSasKeyName(String systemName, String streamName) { + return get(String.format(CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName)); } /** * Get the EventHubs SAS (Shared Access Signature) token for the stream * + * @param systemName name of the system * @param streamName name of stream * @return EventHubs SAS token */ - public String getStreamSasToken(String streamName) { - return getRequiredConfigValue(CONFIG_STREAM_SAS_TOKEN, streamName); + public String getStreamSasToken(String systemName, String streamName) { + return get(String.format(CONFIG_STREAM_SAS_TOKEN, systemName, streamName)); } /** * Get the EventHubs consumer group used for consumption for the stream * + * @param systemName name of the system * @param streamName name of stream * @return EventHubs consumer group */ - public String getStreamConsumerGroup(String streamName) { + public String getStreamConsumerGroup(String systemName, String streamName) { return get(String.format(CONFIG_STREAM_CONSUMER_GROUP, systemName, streamName), DEFAULT_CONFIG_STREAM_CONSUMER_GROUP); } /** * Get the start position when there is no checkpoints. By default the consumer starts from latest (end of stream) * + * @param systemName name of the system * @param streamName name of the stream * @return Starting position when no checkpoints */ - public StartPosition getStartPosition(String streamName) { + public StartPosition getStartPosition(String systemName, String streamName) { String startPositionStr = get(String.format(CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION); return StartPosition.valueOf(startPositionStr.toUpperCase()); @@ -153,9 +150,10 @@ public StartPosition getStartPosition(String streamName) { /** * Get the partition method of the systemName. By default partitioning is handed by EventHub. * + * @param systemName name of the system * @return The method the producer should use to partition the outgoing data */ - public EventHubSystemProducer.PartitioningMethod getPartitioningMethod() { + public EventHubSystemProducer.PartitioningMethod getPartitioningMethod(String systemName) { String partitioningMethod = get(String.format(CONFIG_PRODUCER_PARTITION_METHOD, systemName), DEFAULT_CONFIG_PRODUCER_PARTITION_METHOD); return EventHubSystemProducer.PartitioningMethod.valueOf(partitioningMethod); @@ -165,9 +163,10 @@ public EventHubSystemProducer.PartitioningMethod getPartitioningMethod() { /** * Returns true if the OutgoingMessageEnvelope key should be sent in the outgoing envelope, false otherwise * + * @param systemName name of the system * @return Boolean, is send key included */ - public Boolean getSendKeyInEventProperties() { + public Boolean getSendKeyInEventProperties(String systemName) { String isSendKeyIncluded = get(String.format(CONFIG_SEND_KEY_IN_EVENT_PROPERTIES, systemName)); if (isSendKeyIncluded == null) { return DEFAULT_CONFIG_SEND_KEY_IN_EVENT_PROPERTIES; @@ -175,38 +174,24 @@ public Boolean getSendKeyInEventProperties() { return Boolean.valueOf(isSendKeyIncluded); } - /** - * Get the timeout for terminating the connection to EventHub client - * - * @return long, timeout in millis for the shutdown of EventHub Connection - */ - public long getShutdownWaitTimeMS() { - String timeoutStr = get(String.format(CONFIG_CONNECTION_SHUTDOWN_TIMEOUT_MILLIS, systemName)); - if (timeoutStr == null) { - return DEFAULT_CONFIG_CONNECTION_SHUTDOWN_TIMEOUT_MILLIS; - } - return Long.valueOf(timeoutStr); - } - /** * Get the timeout for the getRuntimeInfo request to EventHub client * + * @param systemName name of the systems * @return long, timeout in millis for fetching RuntimeInfo */ - public long getRuntimeInfoWaitTimeMS() { - String timeoutStr = get(String.format(CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS, systemName)); - if (timeoutStr == null) { - return DEFAULT_CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS; - } - return Long.valueOf(timeoutStr); + public long getRuntimeInfoWaitTimeMS(String systemName) { + return getLong(String.format(CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS, systemName), + DEFAULT_CONFIG_FETCH_RUNTIME_INFO_TIMEOUT_MILLIS); } /** * Get the capacity of the Event Hub consumer buffer - the blocking queue used for storing messages * + * @param systemName name of the system * @return int, number of buffered messages per SystemStreamPartition */ - public int getConsumerBufferCapacity() { + public int getConsumerBufferCapacity(String systemName) { String bufferCapacity = get(String.format(CONFIG_CONSUMER_BUFFER_CAPACITY, systemName)); if (bufferCapacity == null) { return DEFAULT_CONFIG_CONSUMER_BUFFER_CAPACITY; @@ -214,19 +199,4 @@ public int getConsumerBufferCapacity() { return Integer.parseInt(bufferCapacity); } - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), systemName); - } - - @Override - public boolean equals(Object obj) { - return super.equals(obj) && systemName.equals(((EventHubConfig) obj).systemName); - } - - public enum StartPosition { - EARLIEST, - LATEST - } - } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java index 3fba5afddf..79d31c4cc8 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java @@ -37,29 +37,30 @@ public class EventHubSystemFactory implements SystemFactory { - private Map> getSerdesMap(EventHubConfig config) { + private Map> getSerdesMap(EventHubConfig config, String systemName) { Map> serdes = new HashMap<>(); - List streamList = config.getStreamList(); + List streamList = config.getStreams(systemName); streamList.forEach((streamName) -> serdes.put(streamName, new ByteSerde())); return serdes; } @Override public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { - EventHubConfig eventHubConfig = new EventHubConfig(config, systemName); - return new EventHubSystemConsumer(eventHubConfig, new EventHubClientWrapperFactory(), getSerdesMap(eventHubConfig), - registry); + EventHubConfig eventHubConfig = new EventHubConfig(config); + return new EventHubSystemConsumer(eventHubConfig, systemName, new SamzaEventHubClientFactory(), + getSerdesMap(eventHubConfig, systemName), registry); } @Override public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { - EventHubConfig eventHubConfig = new EventHubConfig(config, systemName); - return new EventHubSystemProducer(eventHubConfig, new EventHubClientWrapperFactory(), getSerdesMap(eventHubConfig), + EventHubConfig eventHubConfig = new EventHubConfig(config); + return new EventHubSystemProducer(eventHubConfig, systemName, new SamzaEventHubClientFactory(), + getSerdesMap(eventHubConfig, systemName), registry); } @Override public SystemAdmin getAdmin(String systemName, Config config) { - return new EventHubSystemAdmin(systemName, new EventHubConfig(config, systemName), new EventHubClientWrapperFactory()); + return new EventHubSystemAdmin(systemName, new EventHubConfig(config), new SamzaEventHubClientFactory()); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClient.java similarity index 79% rename from samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java rename to samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClient.java index 12c98aabd6..0a9c61863b 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapper.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClient.java @@ -21,7 +21,10 @@ import com.microsoft.azure.eventhubs.EventHubClient; -public interface EventHubClientWrapper { +/** + * Wraps the {@link EventHubClient} with lifestyle hooks for initialization and close + */ +public interface SamzaEventHubClient { /** * Initiate the connection to EventHub */ @@ -37,9 +40,9 @@ public interface EventHubClientWrapper { /** * Timed synchronous connection close to the EventHub. * - * @param timeoutMS - * Time in Milliseconds to wait for individual components to - * shutdown before moving to the next stage. + * @param timeoutMs + * Time in Milliseconds to wait for individual components to + * shutdown before moving to the next stage. */ - void close(long timeoutMS); + void close(long timeoutMs); } \ No newline at end of file diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapperFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientFactory.java similarity index 60% rename from samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapperFactory.java rename to samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientFactory.java index 82941905bf..ece49d8ff8 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientWrapperFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientFactory.java @@ -19,9 +19,14 @@ package org.apache.samza.system.eventhub; -public class EventHubClientWrapperFactory { - public EventHubClientWrapper getEventHubClientWrapper(String eventHubNamespace, String entityPath, String sasKeyName, - String sasToken, EventHubConfig config) { - return new SamzaEventHubClientWrapper(eventHubNamespace, entityPath, sasKeyName, sasToken); +public class SamzaEventHubClientFactory { + public SamzaEventHubClient getSamzaEventHubClient(String systemName, String streamName, EventHubConfig config) { + + String eventHubNamespace = config.getStreamNamespace(systemName, streamName); + String entityPath = config.getStreamEntityPath(systemName, streamName); + String sasKeyName = config.getStreamSasKeyName(systemName, streamName); + String sasToken = config.getStreamSasToken(systemName, streamName); + + return new SamzaEventHubClientImpl(eventHubNamespace, entityPath, sasKeyName, sasToken); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientImpl.java similarity index 69% rename from samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientWrapper.java rename to samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientImpl.java index 4195fc573a..f4856ce129 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientWrapper.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientImpl.java @@ -29,12 +29,10 @@ import java.io.IOException; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -public class SamzaEventHubClientWrapper implements EventHubClientWrapper { - private static final Logger LOG = LoggerFactory.getLogger(SamzaEventHubClientWrapper.class.getName()); +public class SamzaEventHubClientImpl implements SamzaEventHubClient { + private static final Logger LOG = LoggerFactory.getLogger(SamzaEventHubClientImpl.class.getName()); private static final String EVENTHUB_REMOTE_HOST_FORMAT = "%s.servicebus.windows.net"; private EventHubClient eventHubClient; @@ -44,7 +42,7 @@ public class SamzaEventHubClientWrapper implements EventHubClientWrapper { private final String sasKeyName; private final String sasKey; - public SamzaEventHubClientWrapper(String eventHubNamespace, String entityPath, String sasKeyName, String sasKey) { + public SamzaEventHubClientImpl(String eventHubNamespace, String entityPath, String sasKeyName, String sasKey) { this.eventHubNamespace = eventHubNamespace; this.entityPath = entityPath; this.sasKeyName = sasKeyName; @@ -58,12 +56,9 @@ public void init() { new ConnectionStringBuilder(eventHubNamespace, entityPath, sasKeyName, sasKey); eventHubClient = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString()); - } catch (IOException ioe) { - throw new IllegalStateException( - "Failed to connect to remote host " + remoteHost + ":" + ClientConstants.AMQPS_PORT, ioe); - } catch (ServiceBusException e) { - String msg = String.format("Creation of event hub client failed for eventHub %s %s %s %s with exception", - entityPath, eventHubNamespace, sasKeyName, sasKey); + } catch (IOException | ServiceBusException e) { + String msg = String.format("Creation of EventHub client failed for eventHub %s %s %s %s on remote host %s:%d", + entityPath, eventHubNamespace, sasKeyName, sasKey, remoteHost, ClientConstants.AMQPS_PORT); LOG.error(msg, e); throw new SamzaException(msg, e); } @@ -75,21 +70,16 @@ public EventHubClient getEventHubClient() { public void close(long timeoutMS) { - if (timeoutMS <= 0) { - try { + try { + if (timeoutMS <= 0) { eventHubClient.closeSync(); - } catch (ServiceBusException e) { - LOG.warn("Closing the event hub client failed ", e); - } - } else { - CompletableFuture future = eventHubClient.close(); - try { + } else { + CompletableFuture future = eventHubClient.close(); future.get(timeoutMS, TimeUnit.MILLISECONDS); - } catch (ExecutionException | InterruptedException | TimeoutException e) { - LOG.warn("Closing the event hub client failed ", e); } + } catch (Exception e) { + LOG.warn("Closing the event hub client failed ", e); } - } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index ffff040c94..53b0bfd90e 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -27,10 +27,10 @@ import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata; import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.system.eventhub.EventHubClientWrapper; +import org.apache.samza.system.eventhub.SamzaEventHubClient; import org.apache.samza.system.eventhub.EventHubConfig; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; -import org.apache.samza.system.eventhub.EventHubClientWrapperFactory; +import org.apache.samza.system.eventhub.SamzaEventHubClientFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,17 +44,17 @@ public class EventHubSystemAdmin implements SystemAdmin { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemAdmin.class); - private final EventHubClientWrapperFactory eventHubClientWrapperFactory; + private final SamzaEventHubClientFactory samzaEventHubClientFactory; private String systemName; private EventHubConfig eventHubConfig; - private Map eventHubClients = new HashMap<>(); + private Map eventHubClients = new HashMap<>(); private Map streamPartitions = new HashMap<>(); public EventHubSystemAdmin(String systemName, EventHubConfig eventHubConfig, - EventHubClientWrapperFactory eventHubClientWrapperFactory) { + SamzaEventHubClientFactory samzaEventHubClientFactory) { this.systemName = systemName; this.eventHubConfig = eventHubConfig; - this.eventHubClientWrapperFactory = eventHubClientWrapperFactory; + this.samzaEventHubClientFactory = samzaEventHubClientFactory; } private static String getNextOffset(String currentOffset) { @@ -67,6 +67,7 @@ private static String getNextOffset(String currentOffset) { @Override public Map getOffsetsAfter(Map offsets) { Map results = new HashMap<>(); + offsets.forEach((partition, offset) -> results.put(partition, getNextOffset(offset))); return results; } @@ -75,74 +76,79 @@ public Map getOffsetsAfter(Map getSystemStreamMetadata(Set streamNames) { Map requestedMetadata = new HashMap<>(); Map> ehRuntimeInfos = new HashMap<>(); + streamNames.forEach((streamName) -> { if (!streamPartitions.containsKey(streamName)) { - EventHubClientWrapper eventHubClientWrapper = getStreamEventHubClient(streamName); - ehRuntimeInfos.put(streamName, - eventHubClientWrapper.getEventHubClient().getRuntimeInformation()); + SamzaEventHubClient samzaEventHubClient = getOrCreateStreamEventHubClient(streamName); + CompletableFuture runtimeInfo = samzaEventHubClient.getEventHubClient() + .getRuntimeInformation(); + + ehRuntimeInfos.put(streamName, runtimeInfo); } }); ehRuntimeInfos.forEach((streamName, ehRuntimeInfo) -> { if (!streamPartitions.containsKey(streamName)) { try { - EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(eventHubConfig.getRuntimeInfoWaitTimeMS(), - TimeUnit.MILLISECONDS); + long timeoutMs = eventHubConfig.getRuntimeInfoWaitTimeMS(systemName); + EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(timeoutMs, TimeUnit.MILLISECONDS); + streamPartitions.put(streamName, ehInfo.getPartitionIds()); - } catch (InterruptedException | ExecutionException e) { + } catch (InterruptedException | ExecutionException | TimeoutException e) { String msg = String.format("Error while fetching EventHubRuntimeInfo for System:%s, Stream:%s", systemName, streamName); - LOG.error(msg, e); - throw new SamzaException(msg); - } catch (TimeoutException e) { - String msg = String.format("Timed out while fetching EventHubRuntimeInfo for System:%s, Stream:%s", - systemName, streamName); - LOG.error(msg, e); throw new SamzaException(msg); } } - requestedMetadata.put(streamName, new SystemStreamMetadata(streamName, - getPartitionMetadata(streamName, streamPartitions.get(streamName)))); + String[] partitionIds = streamPartitions.get(streamName); + Map sspMetadataMap = getPartitionMetadata(streamName, partitionIds); + SystemStreamMetadata systemStreamMetadata = new SystemStreamMetadata(streamName, sspMetadataMap); + requestedMetadata.put(streamName, systemStreamMetadata); }); return requestedMetadata; } - private EventHubClientWrapper getStreamEventHubClient(String streamName) { + private SamzaEventHubClient getOrCreateStreamEventHubClient(String streamName) { if (!eventHubClients.containsKey(streamName)) { - eventHubClients.put(streamName, eventHubClientWrapperFactory - .getEventHubClientWrapper(eventHubConfig.getStreamNamespace(streamName), - eventHubConfig.getStreamEntityPath(streamName), eventHubConfig.getStreamSasKeyName(streamName), - eventHubConfig.getStreamSasToken(streamName), eventHubConfig)); - eventHubClients.get(streamName).init(); + SamzaEventHubClient samzaEventHubClient = samzaEventHubClientFactory + .getSamzaEventHubClient(systemName, streamName, eventHubConfig); + + samzaEventHubClient.init(); + eventHubClients.put(streamName, samzaEventHubClient); } return eventHubClients.get(streamName); } private Map getPartitionMetadata(String streamName, String[] partitionIds) { - EventHubClientWrapper eventHubClientWrapper = getStreamEventHubClient(streamName); + SamzaEventHubClient samzaEventHubClient = getOrCreateStreamEventHubClient(streamName); Map sspMetadataMap = new HashMap<>(); Map> ehRuntimeInfos = new HashMap<>(); + for (String partition : partitionIds) { - ehRuntimeInfos.put(partition, eventHubClientWrapper.getEventHubClient() - .getPartitionRuntimeInformation(partition)); + CompletableFuture partitionRuntimeInfo = samzaEventHubClient + .getEventHubClient() + .getPartitionRuntimeInformation(partition); + + ehRuntimeInfos.put(partition, partitionRuntimeInfo); } + ehRuntimeInfos.forEach((partitionId, ehPartitionRuntimeInfo) -> { try { - EventHubPartitionRuntimeInformation ehPartitionInfo = ehPartitionRuntimeInfo - .get(eventHubConfig.getRuntimeInfoWaitTimeMS(), TimeUnit.MILLISECONDS); - sspMetadataMap.put(new Partition(Integer.parseInt(partitionId)), - new SystemStreamPartitionMetadata(EventHubSystemConsumer.START_OF_STREAM, - ehPartitionInfo.getLastEnqueuedOffset(), EventHubSystemConsumer.END_OF_STREAM)); - } catch (InterruptedException | ExecutionException e) { + long timeoutMs = eventHubConfig.getRuntimeInfoWaitTimeMS(systemName); + EventHubPartitionRuntimeInformation ehPartitionInfo = ehPartitionRuntimeInfo.get(timeoutMs, TimeUnit.MILLISECONDS); + + String startingOffset = EventHubSystemConsumer.START_OF_STREAM; + String newestOffset = ehPartitionInfo.getLastEnqueuedOffset(); + String upcomingOffset = EventHubSystemConsumer.END_OF_STREAM; + SystemStreamPartitionMetadata sspMetadata = new SystemStreamPartitionMetadata(startingOffset, newestOffset, + upcomingOffset); + + Partition partition = new Partition(Integer.parseInt(partitionId)); + + sspMetadataMap.put(partition, sspMetadata); + } catch (InterruptedException | ExecutionException | TimeoutException e) { String msg = String.format( "Error while fetching EventHubPartitionRuntimeInfo for System:%s, Stream:%s, Partition:%s", systemName, streamName, partitionId); - LOG.error(msg, e); - throw new SamzaException(msg); - } catch (TimeoutException e) { - String msg = String.format( - "Timed out while fetching EventHubRuntimeInfo for System:%s, Stream:%s, , Partition:%s", - systemName, streamName, partitionId); - LOG.error(msg, e); throw new SamzaException(msg); } }); @@ -151,6 +157,9 @@ private Map getPartitionMetadata(Strin @Override public Integer offsetComparator(String offset1, String offset2) { + if (offset1 == null || offset2 == null) { + return null; + } try { if (offset1.equals(EventHubSystemConsumer.END_OF_STREAM)) { return offset2.equals(EventHubSystemConsumer.END_OF_STREAM) ? 0 : 1; diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index f84692bb20..e5565c2495 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -22,7 +22,7 @@ import com.microsoft.azure.eventhubs.EventData; import com.microsoft.azure.eventhubs.PartitionReceiveHandler; import com.microsoft.azure.eventhubs.PartitionReceiver; -import com.microsoft.azure.servicebus.StringUtil; +import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.MetricsRegistry; @@ -30,8 +30,8 @@ import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.eventhub.EventDataWrapper; -import org.apache.samza.system.eventhub.EventHubClientWrapperFactory; -import org.apache.samza.system.eventhub.EventHubClientWrapper; +import org.apache.samza.system.eventhub.SamzaEventHubClientFactory; +import org.apache.samza.system.eventhub.SamzaEventHubClient; import org.apache.samza.system.eventhub.EventHubConfig; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; import org.apache.samza.util.BlockingEnvelopeMap; @@ -44,7 +44,12 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.*; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; @@ -87,7 +92,8 @@ */ public class EventHubSystemConsumer extends BlockingEnvelopeMap { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemConsumer.class); - private static final int MAX_EVENT_COUNT_PER_PARTITION_POLL = 50; //TODO + private static final int MAX_EVENT_COUNT_PER_PARTITION_POLL = 50; + private static final long DEFAULT_SHUTDOWN_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); public static final String START_OF_STREAM = PartitionReceiver.START_OF_STREAM; // -1 public static final String END_OF_STREAM = "-2"; @@ -102,48 +108,46 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { private static SamzaHistogram aggReadLatency = null; private static Counter aggReadErrors = null; - private Map eventReadRates; - private Map eventByteReadRates; - private Map readLatencies; - private Map readErrors; + private final Map eventReadRates; + private final Map eventByteReadRates; + private final Map readLatencies; + private final Map readErrors; final Map streamPartitionHandlers = new HashMap<>(); private final Map streamPartitionReceivers = new HashMap<>(); - private final Map streamEventHubClients = new HashMap<>(); + private final Map streamEventHubClients = new HashMap<>(); private final Map streamPartitionStartingOffsets = new HashMap<>(); - private Map> serdes; + private final Map> serdes; private boolean isStarted = false; private final EventHubConfig config; + private final String systemName; - public EventHubSystemConsumer(EventHubConfig config, EventHubClientWrapperFactory eventHubClientWrapperFactory, + + public EventHubSystemConsumer(EventHubConfig config, String systemName, + SamzaEventHubClientFactory samzaEventHubClientFactory, Map> serdes, MetricsRegistry registry) { super(registry, System::currentTimeMillis); this.config = config; + this.systemName = systemName; this.serdes = serdes; - List streamList = config.getStreamList(); - + List streamNames = config.getStreams(systemName); // Create and initiate connections to Event Hubs - for (String streamName : streamList) { - String namespace = config.getStreamNamespace(streamName); - String entityPath = config.getStreamEntityPath(streamName); - String sasKeyName = config.getStreamSasKeyName(streamName); - String sasKey = config.getStreamSasToken(streamName); - LOG.info(String.format("Starting connection for namespace=%s, entity=%s ", namespace, entityPath)); - EventHubClientWrapper ehClientWrapper = eventHubClientWrapperFactory - .getEventHubClientWrapper(namespace, entityPath, sasKeyName, sasKey, config); + for (String streamName : streamNames) { + SamzaEventHubClient ehClientWrapper = samzaEventHubClientFactory + .getSamzaEventHubClient(systemName, streamName, config); streamEventHubClients.put(streamName, ehClientWrapper); ehClientWrapper.init(); } // Initiate metrics - eventReadRates = streamList.stream() + eventReadRates = streamNames.stream() .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_READ_RATE))); - eventByteReadRates = streamList.stream() + eventByteReadRates = streamNames.stream() .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, EVENT_BYTE_READ_RATE))); - readLatencies = streamList.stream() + readLatencies = streamNames.stream() .collect(Collectors.toMap(Function.identity(), x -> new SamzaHistogram(registry, x, READ_LATENCY))); - readErrors = streamList.stream() + readErrors = streamNames.stream() .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, READ_ERRORS))); // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. @@ -157,9 +161,30 @@ public EventHubSystemConsumer(EventHubConfig config, EventHubClientWrapperFactor } } - /** - * {@inheritDoc} - */ + @Override + public synchronized void register(SystemStreamPartition systemStreamPartition, String offset) { + super.register(systemStreamPartition, offset); + + if (isStarted) { + throw new SamzaException("Trying to add partition when the connection has already started."); + } + + if (StringUtils.isBlank(offset)) { + switch (config.getStartPosition(systemName, systemStreamPartition.getStream())) { + case EARLIEST: + offset = START_OF_STREAM; + break; + case LATEST: + offset = END_OF_STREAM; + break; + default: + throw new SamzaException("Unknown starting position config " + + config.getStartPosition(systemName, systemStreamPartition.getStream())); + } + } + streamPartitionStartingOffsets.put(systemStreamPartition, offset); + } + @Override public synchronized void start() { isStarted = true; @@ -169,16 +194,12 @@ public synchronized void start() { String streamName = ssp.getStream(); Integer partitionId = ssp.getPartition().getPartitionId(); String offset = entry.getValue(); - String consumerGroup = config.getStreamConsumerGroup(streamName); - String namespace = config.getStreamNamespace(streamName); - String entityPath = config.getStreamEntityPath(streamName); - EventHubClientWrapper ehClientWrapper = streamEventHubClients.get(streamName); + String consumerGroup = config.getStreamConsumerGroup(systemName, streamName); + String namespace = config.getStreamNamespace(systemName, streamName); + String entityPath = config.getStreamEntityPath(systemName, streamName); + SamzaEventHubClient ehClientWrapper = streamEventHubClients.get(streamName); try { PartitionReceiver receiver; - if (StringUtil.isNullOrWhiteSpace(offset)) { - throw new SamzaException( - String.format("Invalid offset %s system=%s, stream=%s", offset, namespace, entityPath)); - } if (offset.equals(EventHubSystemConsumer.END_OF_STREAM)) { receiver = ehClientWrapper.getEventHubClient() .createReceiverSync(consumerGroup, partitionId.toString(), Instant.now()); @@ -191,6 +212,7 @@ public synchronized void start() { eventByteReadRates.get(streamName), readLatencies.get(streamName), readErrors.get(streamName), serdes.getOrDefault(streamName, null)); streamPartitionHandlers.put(ssp, handler); + receiver.setReceiveHandler(handler); streamPartitionReceivers.put(ssp, receiver); } catch (Exception e) { @@ -203,9 +225,6 @@ public synchronized void start() { } } - /** - * {@inheritDoc} - */ @Override public synchronized void stop() { LOG.info("Stopping event hub system consumer..."); @@ -213,57 +232,24 @@ public synchronized void stop() { streamPartitionReceivers.values().forEach((receiver) -> futures.add(receiver.close())); CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); try { - future.get(config.getShutdownWaitTimeMS(), TimeUnit.MILLISECONDS); - } catch (ExecutionException | InterruptedException e) { + future.get(DEFAULT_SHUTDOWN_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { LOG.warn("Failed to close receivers", e); - } catch (TimeoutException e) { - LOG.warn("Closing the partition sender timed out ", e); } - streamEventHubClients.values().forEach(ehClient -> ehClient.close(config.getShutdownWaitTimeMS())); - streamEventHubClients.clear(); - streamPartitionStartingOffsets.clear(); - streamPartitionReceivers.clear(); - } - - /** - * {@inheritDoc} - */ - @Override - public synchronized void register(SystemStreamPartition systemStreamPartition, String offset) { - super.register(systemStreamPartition, offset); - - if (isStarted) { - LOG.warn("Trying to add partition when the connection has already started."); - return; - } - - if (StringUtil.isNullOrWhiteSpace(offset)) { - switch (config.getStartPosition(systemStreamPartition.getStream())) { - case EARLIEST: - offset = START_OF_STREAM; - break; - case LATEST: - offset = END_OF_STREAM; - break; - default: - throw new SamzaException( - "Unknown starting position config " + config.getStartPosition(systemStreamPartition.getStream())); - } - } - streamPartitionStartingOffsets.put(systemStreamPartition, offset); + streamEventHubClients.values().forEach(ehClient -> ehClient.close(DEFAULT_SHUTDOWN_TIMEOUT_MILLIS)); } @Override protected BlockingQueue newBlockingQueue() { - return new LinkedBlockingQueue<>(config.getConsumerBufferCapacity()); + return new LinkedBlockingQueue<>(config.getConsumerBufferCapacity(systemName)); } - private class PartitionReceiverHandlerImpl extends PartitionReceiveHandler { + protected class PartitionReceiverHandlerImpl extends PartitionReceiveHandler { private final Counter eventReadRate; private final Counter eventByteReadRate; private final SamzaHistogram readLatency; - private final Counter errors; + private final Counter errorRate; private final Serde serde; SystemStreamPartition ssp; @@ -274,7 +260,7 @@ private class PartitionReceiverHandlerImpl extends PartitionReceiveHandler { this.eventReadRate = eventReadRate; this.eventByteReadRate = eventByteReadRate; this.readLatency = readLatency; - errors = readErrors; + errorRate = readErrors; this.serde = serde; } @@ -315,7 +301,7 @@ private void updateMetrics(EventData event) { @Override public void onError(Throwable throwable) { - errors.inc(); + errorRate.inc(); aggReadErrors.inc(); LOG.error(String.format("Received error from event hub connection (ssp=%s): ", ssp), throwable); } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java index 631ed0429a..7d6d40832d 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/metrics/SamzaHistogram.java @@ -31,10 +31,12 @@ import java.util.function.Function; import java.util.stream.Collectors; - +/** + * Creates a {@link Histogram} metric using {@link ExponentiallyDecayingReservoir} + * Keeps a {@link Gauge} for each percentile + */ public class SamzaHistogram { private static final List DEFAULT_HISTOGRAM_PERCENTILES = Arrays.asList(50D, 99D); - private final MetricsRegistry registry; private final Histogram histogram; private final List percentiles; private final Map> gauges; @@ -44,18 +46,17 @@ public SamzaHistogram(MetricsRegistry registry, String group, String name) { } public SamzaHistogram(MetricsRegistry registry, String group, String name, List percentiles) { - this.registry = registry; - histogram = new Histogram(new ExponentiallyDecayingReservoir()); + this.histogram = new Histogram(new ExponentiallyDecayingReservoir()); this.percentiles = percentiles; - gauges = this.percentiles.stream() + this.gauges = this.percentiles.stream() .filter(x -> x > 0 && x <= 100) .collect( - Collectors.toMap(Function.identity(), x -> this.registry.newGauge(group, name + "_" + String.valueOf(0), 0D))); + Collectors.toMap(Function.identity(), x -> registry.newGauge(group, name + "_" + String.valueOf(0), 0D))); } public void update(long value) { histogram.update(value); Snapshot values = histogram.getSnapshot(); - percentiles.stream().forEach(x -> gauges.get(x).set(values.getValue(x / 100))); + percentiles.forEach(x -> gauges.get(x).set(values.getValue(x / 100))); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 9c369fe5d5..a277870603 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -29,22 +29,31 @@ import org.apache.samza.serializers.Serde; import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemProducer; -import org.apache.samza.system.eventhub.EventHubClientWrapper; +import org.apache.samza.system.eventhub.SamzaEventHubClient; import org.apache.samza.system.eventhub.EventHubConfig; -import org.apache.samza.system.eventhub.EventHubClientWrapperFactory; +import org.apache.samza.system.eventhub.SamzaEventHubClientFactory; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.charset.Charset; import java.time.Duration; -import java.time.Instant; -import java.util.*; -import java.util.concurrent.*; +import java.util.HashMap; +import java.util.Collection; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.TimeUnit; public class EventHubSystemProducer implements SystemProducer { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemProducer.class.getName()); private static final long FLUSH_SLEEP_TIME_MILLIS = 1000; + private static final long DEFAULT_SHUTDOWN_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; @@ -60,41 +69,65 @@ public class EventHubSystemProducer implements SystemProducer { private static Counter aggSendErrors = null; private static SamzaHistogram aggSendLatency = null; private static SamzaHistogram aggSendCallbackLatency = null; - private HashMap eventWriteRate = new HashMap<>(); - private HashMap eventByteWriteRate = new HashMap<>(); - private HashMap sendLatency = new HashMap<>(); - private HashMap sendCallbackLatency = new HashMap<>(); - private HashMap sendErrors = new HashMap<>(); - private final EventHubClientWrapperFactory eventHubClientWrapperFactory; + public enum PartitioningMethod { + EVENT_HUB_HASHING, + PARTITION_KEY_AS_PARTITION, + } + + private final HashMap eventWriteRate = new HashMap<>(); + private final HashMap eventByteWriteRate = new HashMap<>(); + private final HashMap sendLatency = new HashMap<>(); + private final HashMap sendCallbackLatency = new HashMap<>(); + private final HashMap sendErrors = new HashMap<>(); + + private final SamzaEventHubClientFactory samzaEventHubClientFactory; private final EventHubConfig config; private final MetricsRegistry registry; private final PartitioningMethod partitioningMethod; + private final String systemName; private Throwable sendExceptionOnCallback; private boolean isStarted; // Map of the system name to the event hub client. - private Map eventHubClients = new HashMap<>(); - private Map> streamPartitionSenders = new HashMap<>(); - private Map streamPartitionCounts = new HashMap<>(); + private final Map eventHubClients = new HashMap<>(); + private final Map> streamPartitionSenders = new HashMap<>(); + private final Map streamPartitionCounts = new HashMap<>(); // Running count for the next message Id private long messageId; private Map> serdes; - private Map> pendingFutures = new ConcurrentHashMap<>(); + private final Map> pendingFutures = new ConcurrentHashMap<>(); - public EventHubSystemProducer(EventHubConfig config, EventHubClientWrapperFactory eventHubClientWrapperFactory, + public EventHubSystemProducer(EventHubConfig config, String systemName, + SamzaEventHubClientFactory samzaEventHubClientFactory, Map> serdes, MetricsRegistry registry) { messageId = 0; this.config = config; this.registry = registry; - partitioningMethod = this.config.getPartitioningMethod(); - this.eventHubClientWrapperFactory = eventHubClientWrapperFactory; + this.systemName = systemName; + this.partitioningMethod = config.getPartitioningMethod(systemName); + this.samzaEventHubClientFactory = samzaEventHubClientFactory; this.serdes = serdes; } + @Override + public synchronized void register(String streamName) { + LOG.info("Trying to register {}.", streamName); + if (isStarted) { + String msg = "Cannot register once the producer is started."; + throw new SamzaException(msg); + } + + SamzaEventHubClient ehClient = samzaEventHubClientFactory.getSamzaEventHubClient(systemName, streamName, config); + + ehClient.init(); + eventHubClients.put(streamName, ehClient); + streamPartitionSenders.put(streamName, new HashMap<>()); + } + @Override public synchronized void start() { LOG.info("Starting system producer."); @@ -120,43 +153,6 @@ public synchronized void start() { isStarted = true; } - @Override - public synchronized void stop() { - LOG.info("Stopping event hub system producer..."); - streamPartitionSenders.values().forEach((streamPartitionSender) -> { - List> futures = new ArrayList<>(); - streamPartitionSender.forEach((key, value) -> futures.add(value.close())); - CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); - try { - future.get(config.getShutdownWaitTimeMS(), TimeUnit.MILLISECONDS); - } catch (ExecutionException | InterruptedException e) { - LOG.warn("Closing the partition sender failed ", e); - } catch (TimeoutException e) { - LOG.warn("Closing the partition sender timed out ", e); - } - }); - eventHubClients.values().forEach(ehClient -> ehClient.close(config.getShutdownWaitTimeMS())); - eventHubClients.clear(); - } - - @Override - public synchronized void register(String streamName) { - LOG.info("Trying to register {}.", streamName); - if (isStarted) { - String msg = "Cannot register once the producer is started."; - LOG.error(msg); - throw new SamzaException(msg); - } - - EventHubClientWrapper ehClient = eventHubClientWrapperFactory - .getEventHubClientWrapper(config.getStreamNamespace(streamName), config.getStreamEntityPath(streamName), - config.getStreamSasKeyName(streamName), config.getStreamSasToken(streamName), config); - - ehClient.init(); - eventHubClients.put(streamName, ehClient); - streamPartitionSenders.put(streamName, new HashMap<>()); - } - @Override public synchronized void send(String destination, OutgoingMessageEnvelope envelope) { if (!isStarted) { @@ -165,16 +161,13 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo if (!eventHubClients.containsKey(destination)) { String msg = String.format("Trying to send event to a destination {%s} that is not registered.", destination); - LOG.error(msg); throw new SamzaException(msg); } if (sendExceptionOnCallback != null) { - SamzaException e = new SamzaException(sendExceptionOnCallback); sendExceptionOnCallback = null; pendingFutures.clear(); - LOG.error("One of the previous sends failed."); - throw e; + throw new SamzaException(sendExceptionOnCallback); } EventData eventData = createEventData(destination, envelope); @@ -183,15 +176,15 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo aggEventWriteRate.inc(); eventByteWriteRate.get(destination).inc(eventDataLength); aggEventByteWriteRate.inc(eventDataLength); - EventHubClientWrapper ehClient = eventHubClients.get(destination); + SamzaEventHubClient ehClient = eventHubClients.get(destination); - Instant startTime = Instant.now(); + long beforeSendTimeMs = System.currentTimeMillis(); CompletableFuture sendResult = sendToEventHub(destination, eventData, getEnvelopePartitionId(envelope), ehClient.getEventHubClient()); - Instant endTime = Instant.now(); - long latencyMs = Duration.between(startTime, endTime).toMillis(); + long afterSendTimeMs = System.currentTimeMillis(); + long latencyMs = afterSendTimeMs - beforeSendTimeMs; sendLatency.get(destination).update(latencyMs); aggSendLatency.update(latencyMs); @@ -206,7 +199,7 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo // Auto remove the future from the list when they are complete. sendResult.handle((aVoid, throwable) -> { - long callbackLatencyMs = Duration.between(endTime, Instant.now()).toMillis(); + long callbackLatencyMs = System.currentTimeMillis() - afterSendTimeMs; sendCallbackLatency.get(destination).update(callbackLatencyMs); aggSendCallbackLatency.update(callbackLatencyMs); if (throwable != null) { @@ -227,11 +220,10 @@ private CompletableFuture sendToEventHub(String streamName, EventData even } else if (partitioningMethod == PartitioningMethod.PARTITION_KEY_AS_PARTITION) { if (!(partitionKey instanceof Integer)) { String msg = "Partition key should be of type Integer"; - LOG.error(msg); throw new SamzaException(msg); } - PartitionSender sender = getPartitionSender(streamName, (int) partitionKey, eventHubClient); + PartitionSender sender = getOrCreatePartitionSender(streamName, (int) partitionKey, eventHubClient); return sender.send(eventData); } else { throw new SamzaException("Unknown partitioning method " + partitioningMethod); @@ -254,24 +246,26 @@ private String convertPartitionKeyToString(Object partitionKey) { } } - private PartitionSender getPartitionSender(String streamName, int partition, EventHubClient eventHubClient) { + private PartitionSender getOrCreatePartitionSender(String streamName, int partition, EventHubClient eventHubClient) { Map partitionSenders = streamPartitionSenders.get(streamName); if (!partitionSenders.containsKey(partition)) { try { if (!streamPartitionCounts.containsKey(streamName)) { - streamPartitionCounts.put(streamName, eventHubClient.getRuntimeInformation() - .get(config.getRuntimeInfoWaitTimeMS(), TimeUnit.MILLISECONDS).getPartitionCount()); + long timeoutMs = config.getRuntimeInfoWaitTimeMS(systemName); + Integer numPartitions = eventHubClient.getRuntimeInformation().get(timeoutMs, TimeUnit.MILLISECONDS) + .getPartitionCount(); + + streamPartitionCounts.put(streamName, numPartitions); } - PartitionSender partitionSender = eventHubClient - .createPartitionSenderSync(String.valueOf(partition % streamPartitionCounts.get(streamName))); + String partitionId = String.valueOf(partition % streamPartitionCounts.get(streamName)); + + PartitionSender partitionSender = eventHubClient.createPartitionSenderSync(partitionId); partitionSenders.put(partition, partitionSender); - } catch (ServiceBusException e) { + } catch (ServiceBusException | IllegalArgumentException e) { String msg = "Creation of partition sender failed with exception"; - LOG.error(msg, e); throw new SamzaException(msg, e); } catch (InterruptedException | ExecutionException | TimeoutException e) { String msg = "Failed to fetch number of Event Hub partitions for partition sender creation"; - LOG.error(msg, e); throw new SamzaException(msg, e); } } @@ -290,7 +284,7 @@ private EventData createEventData(String streamName, OutgoingMessageEnvelope env eventData.getProperties().put(PRODUCE_TIMESTAMP, Long.toString(System.currentTimeMillis())); - if (config.getSendKeyInEventProperties()) { + if (config.getSendKeyInEventProperties(systemName)) { String keyValue = ""; if (envelope.getKey() != null) { keyValue = (envelope.getKey() instanceof byte[]) ? new String((byte[]) envelope.getKey()) @@ -310,7 +304,6 @@ public void flush(String source) { Thread.sleep(FLUSH_SLEEP_TIME_MILLIS); } catch (InterruptedException e) { String msg = "Flush failed with error"; - LOG.error(msg, e); throw new SamzaException(msg, e); } } @@ -319,19 +312,30 @@ public void flush(String source) { String msg = "Sending one of the message failed during flush"; Throwable throwable = sendExceptionOnCallback; sendExceptionOnCallback = null; - LOG.error(msg, throwable); throw new SamzaException(msg, throwable); } LOG.info("Flush succeeded."); } - Collection> getPendingFutures() { - return pendingFutures.values(); + @Override + public synchronized void stop() { + LOG.info("Stopping event hub system producer..."); + streamPartitionSenders.values().forEach((streamPartitionSender) -> { + List> futures = new ArrayList<>(); + streamPartitionSender.forEach((key, value) -> futures.add(value.close())); + CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); + try { + future.get(DEFAULT_SHUTDOWN_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + LOG.warn("Closing the partition sender failed ", e); + } + }); + eventHubClients.values().forEach(ehClient -> ehClient.close(DEFAULT_SHUTDOWN_TIMEOUT_MILLIS)); + eventHubClients.clear(); } - public enum PartitioningMethod { - EVENT_HUB_HASHING, - PARTITION_KEY_AS_PARTITION, + Collection> getPendingFutures() { + return pendingFutures.values(); } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java index e15317e754..bf944b3f52 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java @@ -31,7 +31,7 @@ public class MockEventHubConfigFactory { public static final String STREAM_NAME1 = "test_stream1"; public static final String STREAM_NAME2 = "test_stream2"; - // Add target Event Hub connection information here + // Add target Event Hub connection information for integration test here public static final String EVENTHUB_NAMESPACE = ""; public static final String EVENTHUB_KEY_NAME = ""; public static final String EVENTHUB_KEY = ""; @@ -44,14 +44,14 @@ public class MockEventHubConfigFactory { public static Config getEventHubConfig(EventHubSystemProducer.PartitioningMethod partitioningMethod) { HashMap mapConfig = new HashMap<>(); mapConfig.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, SYSTEM_NAME), partitioningMethod.toString()); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME1 + "," + STREAM_NAME2); + mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_NAMESPACE); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY_NAME); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME1), "earliest"); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, SYSTEM_NAME), STREAM_NAME2); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_NAMESPACE); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_ENTITY2); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY_NAME); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubClientFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockSamzaEventHubClientFactory.java similarity index 77% rename from samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubClientFactory.java rename to samza-azure/src/test/java/org/apache/samza/system/eventhub/MockSamzaEventHubClientFactory.java index 1b5eeaf035..8b36dea923 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/MockEventHubClientFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockSamzaEventHubClientFactory.java @@ -17,16 +17,13 @@ * under the License. */ -package org.apache.samza.system.eventhub.consumer; +package org.apache.samza.system.eventhub; import com.microsoft.azure.eventhubs.EventData; import com.microsoft.azure.eventhubs.EventHubClient; import com.microsoft.azure.eventhubs.PartitionReceiveHandler; import com.microsoft.azure.eventhubs.PartitionReceiver; import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.system.eventhub.EventHubClientWrapper; -import org.apache.samza.system.eventhub.EventHubClientWrapperFactory; -import org.apache.samza.system.eventhub.EventHubConfig; import org.junit.Assert; import org.mockito.stubbing.Answer; import org.powermock.api.mockito.PowerMockito; @@ -39,29 +36,28 @@ import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyString; -public class MockEventHubClientFactory extends EventHubClientWrapperFactory { +public class MockSamzaEventHubClientFactory extends SamzaEventHubClientFactory { private Map> eventData; - MockEventHubClientFactory(Map> eventData) { + public MockSamzaEventHubClientFactory(Map> eventData) { this.eventData = eventData; } @Override - public EventHubClientWrapper getEventHubClientWrapper(String eventHubNamespace, String entityPath, String sasKeyName, - String sasToken, EventHubConfig config) { - return new MockEventHubClientWrapper(); + public SamzaEventHubClient getSamzaEventHubClient(String systemName, String streamName, EventHubConfig config) { + return new MockSamzaEventHubClient(); } // Emulate EventHub sending data - void sendToHandlers(Map handlers) { + public void sendToHandlers(Map handlers) { handlers.forEach((ssp, value) -> value.onReceive(eventData.get(ssp))); } - private class MockEventHubClientWrapper implements EventHubClientWrapper { + private class MockSamzaEventHubClient implements SamzaEventHubClient { Boolean initiated = false; EventHubClient mockEventHubClient = PowerMockito.mock(EventHubClient.class); - MockEventHubClientWrapper() { + MockSamzaEventHubClient() { PartitionReceiver mockPartitionReceiver = PowerMockito.mock(PartitionReceiver.class); // Set mocks @@ -91,7 +87,7 @@ public void init() { @Override public EventHubClient getEventHubClient() { if (!initiated) { - Assert.fail("Should have called init() on EventHubClient before getEventHubClientWrapper()"); + Assert.fail("Should have called init() on EventHubClient before getEventHubClient()"); } return mockEventHubClient; } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index 554ef78202..d2bd063000 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -54,7 +54,7 @@ private void verifyEvents(List messages, List 0) { + List result = consumer.poll(Collections.singleton(ssp), 2000).get(ssp); + numEvents = result == null ? 0 : result.size(); + if (numEvents > 0) { + EventDataWrapper eventData = (EventDataWrapper) result.get(0).getMessage(); + System.out.println("System properties: " + eventData.getEventData().getSystemProperties()); + System.out.println("Message: " + new String(eventData.getEventData().getBody())); + break; + } + System.out.println("Retries left: " + numRetries); + } + Assert.assertTrue(numEvents > 0); + } + +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducerIntegration.java similarity index 91% rename from samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java rename to samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducerIntegration.java index fe91d5a437..d511d3aa5b 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducerIntegration.java @@ -25,9 +25,7 @@ import com.microsoft.azure.servicebus.ServiceBusException; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; -import org.apache.samza.system.OutgoingMessageEnvelope; -import org.apache.samza.system.SystemProducer; -import org.apache.samza.system.SystemStream; +import org.apache.samza.system.*; import org.apache.samza.system.eventhub.*; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.apache.samza.util.NoOpMetricsRegistry; @@ -38,8 +36,8 @@ import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.*; -public class TestEventHubSystemProducer { - private static final Logger LOG = LoggerFactory.getLogger(TestEventHubSystemProducer.class.getName()); +public class TestEventHubSystemProducerIntegration { + private static final Logger LOG = LoggerFactory.getLogger(TestEventHubSystemProducerIntegration.class.getName()); @Test public void testSystemFactoryCreateAndStartProducer() { @@ -93,10 +91,9 @@ public void testSend() { @Test public void testReceive() throws ServiceBusException { - EventHubClientWrapperFactory clientFactory = new EventHubClientWrapperFactory(); - EventHubClientWrapper wrapper = clientFactory - .getEventHubClientWrapper(EVENTHUB_NAMESPACE, EVENTHUB_ENTITY1, EVENTHUB_KEY_NAME, EVENTHUB_KEY, - new EventHubConfig(createEventHubConfig(), SYSTEM_NAME)); + SamzaEventHubClientFactory clientFactory = new SamzaEventHubClientFactory(); + SamzaEventHubClient wrapper = clientFactory + .getSamzaEventHubClient(SYSTEM_NAME, STREAM_NAME1, new EventHubConfig(createEventHubConfig())); wrapper.init(); EventHubClient client = wrapper.getEventHubClient(); PartitionReceiver receiver = From f514ecd015674bc4035f8b2fc094d47afa45c95f Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Fri, 13 Oct 2017 17:35:32 -0700 Subject: [PATCH 22/34] Moved create logic for Producer to start, added unit tests for Producer --- build.gradle | 4 +- .../producer/EventHubSystemProducer.java | 65 ++++---- .../MockSamzaEventHubClientFactory.java | 100 +++++++++-- ....java => ITestEventHubSystemConsumer.java} | 2 +- .../consumer/TestEventHubSystemConsumer.java | 6 +- ....java => ITestEventHubSystemProducer.java} | 4 +- .../producer/TestEventHubSystemProducer.java | 155 ++++++++++++++++++ 7 files changed, 284 insertions(+), 52 deletions(-) rename samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/{TestEventHubSystemConsumerIntegration.java => ITestEventHubSystemConsumer.java} (98%) rename samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/{TestEventHubSystemProducerIntegration.java => ITestEventHubSystemProducer.java} (97%) create mode 100644 samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java diff --git a/build.gradle b/build.gradle index c2bb5ecfdd..90192ce007 100644 --- a/build.gradle +++ b/build.gradle @@ -209,8 +209,8 @@ project(':samza-azure') { toolVersion = "$checkstyleVersion" } test { - exclude 'org/apache/samza/system/eventhub/producer/*Integration*' - exclude 'org/apache/samza/system/eventhub/consumer/*Integration*' + exclude 'org/apache/samza/system/eventhub/producer/*ITest*' + exclude 'org/apache/samza/system/eventhub/consumer/*ITest*' } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index a277870603..ce3165ae01 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -93,7 +93,6 @@ public enum PartitioningMethod { // Map of the system name to the event hub client. private final Map eventHubClients = new HashMap<>(); private final Map> streamPartitionSenders = new HashMap<>(); - private final Map streamPartitionCounts = new HashMap<>(); // Running count for the next message Id private long messageId; @@ -125,12 +124,40 @@ public synchronized void register(String streamName) { ehClient.init(); eventHubClients.put(streamName, ehClient); - streamPartitionSenders.put(streamName, new HashMap<>()); } @Override public synchronized void start() { LOG.info("Starting system producer."); + + if (PartitioningMethod.PARTITION_KEY_AS_PARTITION.equals(partitioningMethod)) { + // Create all partition senders + eventHubClients.forEach((streamName, samzaEventHubClient) -> { + EventHubClient ehClient = samzaEventHubClient.getEventHubClient(); + + try { + Map partitionSenders = new HashMap<>(); + long timeoutMs = config.getRuntimeInfoWaitTimeMS(systemName); + Integer numPartitions = ehClient.getRuntimeInformation().get(timeoutMs, TimeUnit.MILLISECONDS) + .getPartitionCount(); + + for (int i = 0; i < numPartitions; i++) { // 32 partitions max + String partitionId = String.valueOf(i); + PartitionSender partitionSender = ehClient.createPartitionSenderSync(partitionId); + partitionSenders.put(i, partitionSender); + } + + streamPartitionSenders.put(streamName, partitionSenders); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + String msg = "Failed to fetch number of Event Hub partitions for partition sender creation"; + throw new SamzaException(msg, e); + } catch (ServiceBusException | IllegalArgumentException e) { + String msg = "Creation of partition sender failed with exception"; + throw new SamzaException(msg, e); + } + }); + } + for (String eventHub : eventHubClients.keySet()) { eventWriteRate.put(eventHub, registry.newCounter(eventHub, EVENT_WRITE_RATE)); eventByteWriteRate.put(eventHub, registry.newCounter(eventHub, EVENT_BYTE_WRITE_RATE)); @@ -223,14 +250,17 @@ private CompletableFuture sendToEventHub(String streamName, EventData even throw new SamzaException(msg); } - PartitionSender sender = getOrCreatePartitionSender(streamName, (int) partitionKey, eventHubClient); + Integer numPartition = streamPartitionSenders.get(streamName).size(); + Integer destinationPartition = (Integer) partitionKey % numPartition; + + PartitionSender sender = streamPartitionSenders.get(streamName).get(destinationPartition); return sender.send(eventData); } else { throw new SamzaException("Unknown partitioning method " + partitioningMethod); } } - private Object getEnvelopePartitionId(OutgoingMessageEnvelope envelope) { + protected Object getEnvelopePartitionId(OutgoingMessageEnvelope envelope) { return envelope.getPartitionKey(); } @@ -246,33 +276,6 @@ private String convertPartitionKeyToString(Object partitionKey) { } } - private PartitionSender getOrCreatePartitionSender(String streamName, int partition, EventHubClient eventHubClient) { - Map partitionSenders = streamPartitionSenders.get(streamName); - if (!partitionSenders.containsKey(partition)) { - try { - if (!streamPartitionCounts.containsKey(streamName)) { - long timeoutMs = config.getRuntimeInfoWaitTimeMS(systemName); - Integer numPartitions = eventHubClient.getRuntimeInformation().get(timeoutMs, TimeUnit.MILLISECONDS) - .getPartitionCount(); - - streamPartitionCounts.put(streamName, numPartitions); - } - String partitionId = String.valueOf(partition % streamPartitionCounts.get(streamName)); - - PartitionSender partitionSender = eventHubClient.createPartitionSenderSync(partitionId); - partitionSenders.put(partition, partitionSender); - } catch (ServiceBusException | IllegalArgumentException e) { - String msg = "Creation of partition sender failed with exception"; - throw new SamzaException(msg, e); - } catch (InterruptedException | ExecutionException | TimeoutException e) { - String msg = "Failed to fetch number of Event Hub partitions for partition sender creation"; - throw new SamzaException(msg, e); - } - } - - return partitionSenders.get(partition); - } - private EventData createEventData(String streamName, OutgoingMessageEnvelope envelope) { Optional> serde = Optional.ofNullable(serdes.getOrDefault(streamName, null)); byte[] eventValue = (byte[]) envelope.getMessage(); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockSamzaEventHubClientFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockSamzaEventHubClientFactory.java index 8b36dea923..2df370a862 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockSamzaEventHubClientFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockSamzaEventHubClientFactory.java @@ -19,25 +19,29 @@ package org.apache.samza.system.eventhub; -import com.microsoft.azure.eventhubs.EventData; -import com.microsoft.azure.eventhubs.EventHubClient; -import com.microsoft.azure.eventhubs.PartitionReceiveHandler; -import com.microsoft.azure.eventhubs.PartitionReceiver; +import com.microsoft.azure.eventhubs.*; import org.apache.samza.system.SystemStreamPartition; import org.junit.Assert; import org.mockito.stubbing.Answer; import org.powermock.api.mockito.PowerMockito; import java.time.Instant; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyBoolean; -import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.*; public class MockSamzaEventHubClientFactory extends SamzaEventHubClientFactory { private Map> eventData; + private Map>>> receivedData; + + public MockSamzaEventHubClientFactory() { + this.receivedData = new HashMap<>(); + } public MockSamzaEventHubClientFactory(Map> eventData) { this.eventData = eventData; @@ -45,22 +49,45 @@ public MockSamzaEventHubClientFactory(Map @Override public SamzaEventHubClient getSamzaEventHubClient(String systemName, String streamName, EventHubConfig config) { - return new MockSamzaEventHubClient(); + if (receivedData != null) { + if (!receivedData.containsKey(systemName)) { + receivedData.put(systemName, new HashMap<>()); + } + + if (!receivedData.get(systemName).containsKey(streamName)) { + receivedData.get(systemName).put(streamName, new HashMap<>()); + receivedData.get(systemName).get(streamName).put(0, new ArrayList<>()); + receivedData.get(systemName).get(streamName).put(1, new ArrayList<>()); + } + } + return new MockSamzaEventHubClient(systemName, streamName); } // Emulate EventHub sending data public void sendToHandlers(Map handlers) { + if (eventData == null) return; handlers.forEach((ssp, value) -> value.onReceive(eventData.get(ssp))); } + public List getSentData(String systemName, String streamName, Integer partitionId) { + if (receivedData.containsKey(systemName) && receivedData.get(systemName).containsKey(streamName)) { + return receivedData.get(systemName).get(streamName).get(partitionId); + } + return null; + } + private class MockSamzaEventHubClient implements SamzaEventHubClient { Boolean initiated = false; EventHubClient mockEventHubClient = PowerMockito.mock(EventHubClient.class); + String systemName; + String streamName; - MockSamzaEventHubClient() { - PartitionReceiver mockPartitionReceiver = PowerMockito.mock(PartitionReceiver.class); + MockSamzaEventHubClient(String systemName, String streamName) { + this.systemName = systemName; + this.streamName = streamName; - // Set mocks + // Consumer mocks + PartitionReceiver mockPartitionReceiver = PowerMockito.mock(PartitionReceiver.class); PowerMockito.when(mockPartitionReceiver.setReceiveHandler(any())).then((Answer) invocationOnMock -> { PartitionReceiveHandler handler = invocationOnMock.getArgumentAt(0, PartitionReceiveHandler.class); if (handler == null) { @@ -69,13 +96,49 @@ private class MockSamzaEventHubClient implements SamzaEventHubClient { return null; }); + // Producer mocks + PartitionSender mockPartitionSender0 = PowerMockito.mock(PartitionSender.class); + PartitionSender mockPartitionSender1 = PowerMockito.mock(PartitionSender.class); + PowerMockito.when(mockPartitionSender0.send(any(EventData.class))) + .then((Answer>) invocationOnMock -> { + EventData data = invocationOnMock.getArgumentAt(0, EventData.class); + receivedData.get(systemName).get(streamName).get(0).add(data); + return new CompletableFuture<>(); + }); + PowerMockito.when(mockPartitionSender1.send(any(EventData.class))) + .then((Answer>) invocationOnMock -> { + EventData data = invocationOnMock.getArgumentAt(0, EventData.class); + receivedData.get(systemName).get(streamName).get(1).add(data); + return new CompletableFuture<>(); + }); + + EventHubRuntimeInformation mockRuntimeInfo = PowerMockito.mock(EventHubRuntimeInformation.class); + CompletableFuture future = new MockFuture(mockRuntimeInfo); + PowerMockito.when(mockRuntimeInfo.getPartitionCount()).thenReturn(2); + try { + // Consumer calls PowerMockito.when(mockEventHubClient.createReceiverSync(anyString(), anyString(), any(Instant.class))) .thenReturn(mockPartitionReceiver); PowerMockito.when(mockEventHubClient.createReceiverSync(anyString(), anyString(), anyString(), anyBoolean())) .thenReturn(mockPartitionReceiver); + + // Producer calls + PowerMockito.when(mockEventHubClient.createPartitionSenderSync("0")).thenReturn(mockPartitionSender0); + PowerMockito.when(mockEventHubClient.createPartitionSenderSync("1")).thenReturn(mockPartitionSender1); + + PowerMockito.when(mockEventHubClient.getRuntimeInformation()).thenReturn(future); + + PowerMockito.when(mockEventHubClient.send(any(EventData.class), anyString())) + .then((Answer>) invocationOnMock -> { + EventData data = invocationOnMock.getArgumentAt(0, EventData.class); + String key = invocationOnMock.getArgumentAt(1, String.class); + Integer intKey = Integer.valueOf(key); + receivedData.get(systemName).get(streamName).get(intKey % 2).add(data); + return new CompletableFuture<>(); + }); } catch (Exception e) { - Assert.fail("Cannot create mockReceiverSync"); + Assert.fail("Failed to create create mock methods for EventHubClient"); } } @@ -100,6 +163,19 @@ public void close(long timeoutMS) { initiated = false; } + private class MockFuture extends CompletableFuture { + EventHubRuntimeInformation runtimeInformation; + + MockFuture(EventHubRuntimeInformation runtimeInformation) { + this.runtimeInformation = runtimeInformation; + } + + @Override + public EventHubRuntimeInformation get(long timeout, TimeUnit unit) { + return runtimeInformation; + } + } + } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumerIntegration.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java similarity index 98% rename from samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumerIntegration.java rename to samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java index 630d5e1ead..cd2a1c6c77 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumerIntegration.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java @@ -38,7 +38,7 @@ import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.STREAM_NAME1; import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.SYSTEM_NAME; -public class TestEventHubSystemConsumerIntegration { +public class ITestEventHubSystemConsumer { private Config createEventHubConfig() { return MockEventHubConfigFactory.getEventHubConfig(EventHubSystemProducer.PartitioningMethod.EVENT_HUB_HASHING); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index d2bd063000..5e07ae4ace 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -20,9 +20,7 @@ package org.apache.samza.system.eventhub.consumer; -import com.microsoft.azure.eventhubs.EventData; -import com.microsoft.azure.eventhubs.EventHubClient; -import com.microsoft.azure.eventhubs.PartitionReceiver; +import com.microsoft.azure.eventhubs.*; import org.apache.samza.Partition; import org.apache.samza.metrics.Counter; import org.apache.samza.serializers.ByteSerde; @@ -43,7 +41,7 @@ import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.*; @RunWith(PowerMockRunner.class) -@PrepareForTest({PartitionReceiver.class, EventHubClient.class}) +@PrepareForTest({EventHubRuntimeInformation.class, EventHubClient.class, PartitionReceiver.class, PartitionSender.class}) public class TestEventHubSystemConsumer { private static final String MOCK_ENTITY_1 = "mocktopic1"; private static final String MOCK_ENTITY_2 = "mocktopic2"; diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducerIntegration.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/ITestEventHubSystemProducer.java similarity index 97% rename from samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducerIntegration.java rename to samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/ITestEventHubSystemProducer.java index d511d3aa5b..1a81517541 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducerIntegration.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/ITestEventHubSystemProducer.java @@ -36,8 +36,8 @@ import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.*; -public class TestEventHubSystemProducerIntegration { - private static final Logger LOG = LoggerFactory.getLogger(TestEventHubSystemProducerIntegration.class.getName()); +public class ITestEventHubSystemProducer { + private static final Logger LOG = LoggerFactory.getLogger(ITestEventHubSystemProducer.class.getName()); @Test public void testSystemFactoryCreateAndStartProducer() { diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java new file mode 100644 index 0000000000..5bd17df78e --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -0,0 +1,155 @@ +/* +* 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.samza.system.eventhub.producer; + +import com.microsoft.azure.eventhubs.*; +import org.apache.samza.serializers.ByteSerde; +import org.apache.samza.serializers.Serde; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemStream; +import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.MockSamzaEventHubClientFactory; +import org.apache.samza.system.eventhub.producer.EventHubSystemProducer.PartitioningMethod; +import org.apache.samza.system.eventhub.TestMetricsRegistry; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.*; +import java.util.stream.Collectors; + +import static org.apache.samza.system.eventhub.MockEventHubConfigFactory.*; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({EventHubRuntimeInformation.class, EventHubClient.class, PartitionReceiver.class, PartitionSender.class}) +public class TestEventHubSystemProducer { + + private static List generateMessages(int numMsg) { + Random rand = new Random(System.currentTimeMillis()); + List messages = new ArrayList<>(); + for (int i = 0; i < numMsg; i++) { + messages.add("message payload: " + rand.nextInt()); + } + return messages; + } + + @Test + public void testSendingToSpecificPartitions() throws Exception { + String systemName = "eventhubs"; + String streamName = "testStream"; + int numEvents = 10; + int partitionId0 = 0; + int partitionId1 = 1; + + TestMetricsRegistry testMetrics = new TestMetricsRegistry(); + Map> serdes = new HashMap<>(); + serdes.put(streamName, new ByteSerde()); + + List outgoingMessagesP0 = generateMessages(numEvents); + List outgoingMessagesP1 = generateMessages(numEvents); + + // Set configs + Map configMap = new HashMap<>(); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), streamName); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, systemName, streamName), EVENTHUB_NAMESPACE); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), EVENTHUB_ENTITY1); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "latest"); + configMap.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName), + PartitioningMethod.PARTITION_KEY_AS_PARTITION.toString()); + + MockSamzaEventHubClientFactory factory = new MockSamzaEventHubClientFactory(); + + EventHubSystemProducer producer = + new EventHubSystemProducer(new EventHubConfig(configMap), systemName, factory, serdes, testMetrics); + + SystemStream systemStream = new SystemStream(systemName, streamName); + producer.register(streamName); + producer.start(); + + outgoingMessagesP0.forEach(message -> + producer.send(streamName, new OutgoingMessageEnvelope(systemStream, partitionId0, null, message.getBytes()))); + outgoingMessagesP1.forEach(message -> + producer.send(streamName, new OutgoingMessageEnvelope(systemStream, partitionId1, null, message.getBytes()))); + + // Retrieve sent data + List receivedData0 = factory.getSentData(systemName, streamName, partitionId0) + .stream().map(eventData -> new String(eventData.getBytes())).collect(Collectors.toList()); + List receivedData1 = factory.getSentData(systemName, streamName, partitionId1) + .stream().map(eventData -> new String(eventData.getBytes())).collect(Collectors.toList()); + + Assert.assertTrue(outgoingMessagesP0.equals(receivedData0)); + Assert.assertTrue(outgoingMessagesP1.equals(receivedData1)); + } + + @Test + public void testSendingToEventHubHashing() throws Exception { + String systemName = "eventhubs"; + String streamName = "testStream"; + int numEvents = 10; + String partitionId0 = "124"; + String partitionId1 = "235"; + + TestMetricsRegistry testMetrics = new TestMetricsRegistry(); + Map> serdes = new HashMap<>(); + serdes.put(streamName, new ByteSerde()); + + List outgoingMessagesP0 = generateMessages(numEvents); + List outgoingMessagesP1 = generateMessages(numEvents); + + // Set configs + Map configMap = new HashMap<>(); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), streamName); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, systemName, streamName), EVENTHUB_NAMESPACE); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), EVENTHUB_ENTITY1); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "latest"); + // mod 2 on the partitionid to simulate consistent hashing + configMap.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName), + PartitioningMethod.EVENT_HUB_HASHING.toString()); + + MockSamzaEventHubClientFactory factory = new MockSamzaEventHubClientFactory(); + + EventHubSystemProducer producer = + new EventHubSystemProducer(new EventHubConfig(configMap), systemName, factory, serdes, testMetrics); + + SystemStream systemStream = new SystemStream(systemName, streamName); + producer.register(streamName); + producer.start(); + + outgoingMessagesP0.forEach(message -> + producer.send(streamName, new OutgoingMessageEnvelope(systemStream, partitionId0, null, message.getBytes()))); + outgoingMessagesP1.forEach(message -> + producer.send(streamName, new OutgoingMessageEnvelope(systemStream, partitionId1, null, message.getBytes()))); + + // Retrieve sent data + List receivedData0 = factory.getSentData(systemName, streamName, 0) + .stream().map(eventData -> new String(eventData.getBytes())).collect(Collectors.toList()); + List receivedData1 = factory.getSentData(systemName, streamName, 1) + .stream().map(eventData -> new String(eventData.getBytes())).collect(Collectors.toList()); + + Assert.assertTrue(outgoingMessagesP0.equals(receivedData0)); + Assert.assertTrue(outgoingMessagesP1.equals(receivedData1)); + } +} From 2b2c11c359a087218f9e5405b7caa63208f27b95 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 16 Oct 2017 11:50:04 -0700 Subject: [PATCH 23/34] Removed EventDataWrapper, added EventHubIME --- .../system/eventhub/EventDataWrapper.java | 48 ----------------- .../system/eventhub/consumer/EventHubIME.java | 51 +++++++++++++++++++ .../consumer/EventHubSystemConsumer.java | 6 +-- .../consumer/ITestEventHubSystemConsumer.java | 7 ++- .../consumer/TestEventHubSystemConsumer.java | 2 +- 5 files changed, 57 insertions(+), 57 deletions(-) delete mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubIME.java diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java deleted file mode 100644 index 1c2993673e..0000000000 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventDataWrapper.java +++ /dev/null @@ -1,48 +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. -*/ - -package org.apache.samza.system.eventhub; - -import com.microsoft.azure.eventhubs.EventData; - -/** - * Simpler wrapper of {@link EventData} events with the decrypted payload - */ -public class EventDataWrapper { - private final EventData eventData; - private final byte[] body; - - public EventDataWrapper(EventData eventData, byte[] body) { - this.eventData = eventData; - this.body = body; - } - - public EventData getEventData() { - return eventData; - } - - /** - * @return the body of decrypted body of the message. In case no encryption is setup for this topic, - * just returns the body of the message. - */ - public byte[] getBody() { - return body; - } - -} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubIME.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubIME.java new file mode 100644 index 0000000000..f1ca9b9427 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubIME.java @@ -0,0 +1,51 @@ +/* +* 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.samza.system.eventhub.consumer; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventData.SystemProperties; +import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.system.SystemStreamPartition; + +import java.util.Map; + +/** + * Extension of {@link IncomingMessageEnvelope} which contains {@link EventData} system and user properties metadata + */ +public class EventHubIME extends IncomingMessageEnvelope { + private SystemProperties systemProperties; + private Map userProperties; + + + public EventHubIME(SystemStreamPartition systemStreamPartition, String offset, Object key, Object message, EventData eventData) { + super(systemStreamPartition, offset, key, message); + + this.systemProperties = eventData.getSystemProperties(); + this.userProperties = eventData.getProperties(); + } + + public SystemProperties getSystemProperties() { + return systemProperties; + } + + public Map getUserProperties() { + return userProperties; + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index e5565c2495..3d25e70146 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -29,7 +29,6 @@ import org.apache.samza.serializers.Serde; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.system.eventhub.EventDataWrapper; import org.apache.samza.system.eventhub.SamzaEventHubClientFactory; import org.apache.samza.system.eventhub.SamzaEventHubClient; import org.apache.samza.system.eventhub.EventHubConfig; @@ -273,12 +272,11 @@ public void onReceive(Iterable events) { if (serde != null) { decryptedBody = serde.fromBytes(decryptedBody); } - EventDataWrapper wrappedEvent = new EventDataWrapper(event, decryptedBody); try { updateMetrics(event); // note that the partition key can be null - put(ssp, new IncomingMessageEnvelope(ssp, event.getSystemProperties().getOffset(), - event.getSystemProperties().getPartitionKey(), wrappedEvent)); + put(ssp, new EventHubIME(ssp, event.getSystemProperties().getOffset(), + event.getSystemProperties().getPartitionKey(), decryptedBody, event)); } catch (Exception e) { String msg = String.format("Exception while adding the event from ssp %s to dispatch queue.", ssp); LOG.error(msg, e); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java index cd2a1c6c77..374301d0b3 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java @@ -24,7 +24,6 @@ import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemConsumer; import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.system.eventhub.EventDataWrapper; import org.apache.samza.system.eventhub.EventHubSystemFactory; import org.apache.samza.system.eventhub.MockEventHubConfigFactory; import org.apache.samza.system.eventhub.TestMetricsRegistry; @@ -64,9 +63,9 @@ public void testSinglePartitionConsumptionHappyPath() throws Exception { List result = consumer.poll(Collections.singleton(ssp), 2000).get(ssp); numEvents = result == null ? 0 : result.size(); if (numEvents > 0) { - EventDataWrapper eventData = (EventDataWrapper) result.get(0).getMessage(); - System.out.println("System properties: " + eventData.getEventData().getSystemProperties()); - System.out.println("Message: " + new String(eventData.getEventData().getBody())); + EventHubIME eventData = (EventHubIME) result.get(0); + System.out.println("System properties: " + eventData.getSystemProperties()); + System.out.println("Message: " + new String((byte[]) eventData.getMessage())); break; } System.out.println("Retries left: " + numRetries); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index 5e07ae4ace..611b322a0c 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -52,7 +52,7 @@ private void verifyEvents(List messages, List Date: Wed, 18 Oct 2017 18:52:37 -0700 Subject: [PATCH 24/34] Added admin close, consumer async onerror callback, producer sync --- .../system/eventhub/SamzaEventHubClient.java | 6 +- .../eventhub/SamzaEventHubClientImpl.java | 20 +++++- .../eventhub/admin/EventHubSystemAdmin.java | 58 +++++++++------ .../consumer/EventHubSystemConsumer.java | 71 ++++++++++++++----- .../producer/EventHubSystemProducer.java | 60 +++++++--------- 5 files changed, 138 insertions(+), 77 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClient.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClient.java index 0a9c61863b..b8770fc223 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClient.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClient.java @@ -22,16 +22,16 @@ import com.microsoft.azure.eventhubs.EventHubClient; /** - * Wraps the {@link EventHubClient} with lifestyle hooks for initialization and close + * Wraps the {@link EventHubClient} with lifestyle hooks for initialization and close. */ public interface SamzaEventHubClient { /** - * Initiate the connection to EventHub + * Initiate the connection to EventHub. */ void init(); /** - * Returns the EventHubClient instance of the wrapper so its methods can be invoked directly + * Returns the EventHubClient instance of the wrapper so its methods can be invoked directly. * * @return EventHub client instance of the wrapper */ diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientImpl.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientImpl.java index f4856ce129..f3c4f29654 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientImpl.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientImpl.java @@ -22,18 +22,26 @@ import com.microsoft.azure.eventhubs.EventHubClient; import com.microsoft.azure.servicebus.ClientConstants; import com.microsoft.azure.servicebus.ConnectionStringBuilder; +import com.microsoft.azure.servicebus.RetryExponential; +import com.microsoft.azure.servicebus.RetryPolicy; import com.microsoft.azure.servicebus.ServiceBusException; import org.apache.samza.SamzaException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.time.Duration; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; public class SamzaEventHubClientImpl implements SamzaEventHubClient { private static final Logger LOG = LoggerFactory.getLogger(SamzaEventHubClientImpl.class.getName()); + private static final String EVENTHUB_REMOTE_HOST_FORMAT = "%s.servicebus.windows.net"; + private static final Duration MIN_RETRY_BACKOFF = Duration.ofSeconds(0); + private static final Duration MAX_RETRY_BACKOFF = Duration.ofSeconds(10); + private static final int MAX_RETRY_COUNT = 100; + private static final String SAMZA_EVENTHUB_RETRY = "SAMZA_CONNECTOR_RETRY"; private EventHubClient eventHubClient; @@ -41,12 +49,20 @@ public class SamzaEventHubClientImpl implements SamzaEventHubClient { private final String entityPath; private final String sasKeyName; private final String sasKey; + private final RetryPolicy retryPolicy; public SamzaEventHubClientImpl(String eventHubNamespace, String entityPath, String sasKeyName, String sasKey) { + this(eventHubNamespace, entityPath, sasKeyName, sasKey, + new RetryExponential(MIN_RETRY_BACKOFF, MAX_RETRY_BACKOFF, MAX_RETRY_COUNT, SAMZA_EVENTHUB_RETRY)); + } + + public SamzaEventHubClientImpl(String eventHubNamespace, String entityPath, String sasKeyName, String sasKey, + RetryPolicy retryPolicy) { this.eventHubNamespace = eventHubNamespace; this.entityPath = entityPath; this.sasKeyName = sasKeyName; this.sasKey = sasKey; + this.retryPolicy = retryPolicy; } public void init() { @@ -55,7 +71,7 @@ public void init() { ConnectionStringBuilder connectionStringBuilder = new ConnectionStringBuilder(eventHubNamespace, entityPath, sasKeyName, sasKey); - eventHubClient = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString()); + eventHubClient = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString(), retryPolicy); } catch (IOException | ServiceBusException e) { String msg = String.format("Creation of EventHub client failed for eventHub %s %s %s %s on remote host %s:%d", entityPath, eventHubNamespace, sasKeyName, sasKey, remoteHost, ClientConstants.AMQPS_PORT); @@ -78,7 +94,7 @@ public void close(long timeoutMS) { future.get(timeoutMS, TimeUnit.MILLISECONDS); } } catch (Exception e) { - LOG.warn("Closing the event hub client failed ", e); + LOG.error("Closing the event hub client failed ", e); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index 53b0bfd90e..6a360288e4 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -34,6 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Duration; import java.util.HashMap; import java.util.Map; import java.util.Set; @@ -44,6 +45,8 @@ public class EventHubSystemAdmin implements SystemAdmin { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemAdmin.class); + private static final long DEFAULT_SHUTDOWN_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); + private final SamzaEventHubClientFactory samzaEventHubClientFactory; private String systemName; private EventHubConfig eventHubConfig; @@ -74,8 +77,8 @@ public Map getOffsetsAfter(Map getSystemStreamMetadata(Set streamNames) { - Map requestedMetadata = new HashMap<>(); - Map> ehRuntimeInfos = new HashMap<>(); + final Map requestedMetadata = new HashMap<>(); + final Map> ehRuntimeInfos = new HashMap<>(); streamNames.forEach((streamName) -> { if (!streamPartitions.containsKey(streamName)) { @@ -86,24 +89,39 @@ public Map getSystemStreamMetadata(Set str ehRuntimeInfos.put(streamName, runtimeInfo); } }); - ehRuntimeInfos.forEach((streamName, ehRuntimeInfo) -> { - if (!streamPartitions.containsKey(streamName)) { - try { - long timeoutMs = eventHubConfig.getRuntimeInfoWaitTimeMS(systemName); - EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(timeoutMs, TimeUnit.MILLISECONDS); - - streamPartitions.put(streamName, ehInfo.getPartitionIds()); - } catch (InterruptedException | ExecutionException | TimeoutException e) { - String msg = String.format("Error while fetching EventHubRuntimeInfo for System:%s, Stream:%s", - systemName, streamName); - throw new SamzaException(msg); + + try { + ehRuntimeInfos.forEach((streamName, ehRuntimeInfo) -> { + + if (!streamPartitions.containsKey(streamName)) { + try { + + long timeoutMs = eventHubConfig.getRuntimeInfoWaitTimeMS(systemName); + EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(timeoutMs, TimeUnit.MILLISECONDS); + + streamPartitions.put(streamName, ehInfo.getPartitionIds()); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + + String msg = String.format("Error while fetching EventHubRuntimeInfo for System:%s, Stream:%s", + systemName, streamName); + throw new SamzaException(msg); + } } - } - String[] partitionIds = streamPartitions.get(streamName); - Map sspMetadataMap = getPartitionMetadata(streamName, partitionIds); - SystemStreamMetadata systemStreamMetadata = new SystemStreamMetadata(streamName, sspMetadataMap); - requestedMetadata.put(streamName, systemStreamMetadata); - }); + + String[] partitionIds = streamPartitions.get(streamName); + Map sspMetadataMap = getPartitionMetadata(streamName, partitionIds); + SystemStreamMetadata systemStreamMetadata = new SystemStreamMetadata(streamName, sspMetadataMap); + + requestedMetadata.put(streamName, systemStreamMetadata); + }); + + } finally { + + // Closing clients + eventHubClients.forEach((streamName, client) -> client.close(DEFAULT_SHUTDOWN_TIMEOUT_MILLIS)); + eventHubClients.clear(); + } + return requestedMetadata; } @@ -138,7 +156,7 @@ private Map getPartitionMetadata(Strin String startingOffset = EventHubSystemConsumer.START_OF_STREAM; String newestOffset = ehPartitionInfo.getLastEnqueuedOffset(); - String upcomingOffset = EventHubSystemConsumer.END_OF_STREAM; + String upcomingOffset = getNextOffset(newestOffset); SystemStreamPartitionMetadata sspMetadata = new SystemStreamPartitionMetadata(startingOffset, newestOffset, upcomingOffset); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index 3d25e70146..6d1c3528ed 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -22,6 +22,7 @@ import com.microsoft.azure.eventhubs.EventData; import com.microsoft.azure.eventhubs.PartitionReceiveHandler; import com.microsoft.azure.eventhubs.PartitionReceiver; +import com.microsoft.azure.servicebus.ServiceBusException; import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; import org.apache.samza.metrics.Counter; @@ -40,11 +41,11 @@ import java.time.Duration; import java.time.Instant; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeoutException; @@ -92,6 +93,9 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemConsumer.class); private static final int MAX_EVENT_COUNT_PER_PARTITION_POLL = 50; + + // Overall timeout for EventHubClient exponential backoff policy + private static final Duration DEFAULT_EVENTHUB_RECEIVER_TIMEOUT = Duration.ofMinutes(10L); private static final long DEFAULT_SHUTDOWN_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); public static final String START_OF_STREAM = PartitionReceiver.START_OF_STREAM; // -1 @@ -112,10 +116,10 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { private final Map readLatencies; private final Map readErrors; - final Map streamPartitionHandlers = new HashMap<>(); - private final Map streamPartitionReceivers = new HashMap<>(); - private final Map streamEventHubClients = new HashMap<>(); - private final Map streamPartitionStartingOffsets = new HashMap<>(); + final ConcurrentHashMap streamPartitionHandlers = new ConcurrentHashMap<>(); + private final ConcurrentHashMap streamPartitionReceivers = new ConcurrentHashMap<>(); + private final ConcurrentHashMap streamEventHubClients = new ConcurrentHashMap<>(); + private final ConcurrentHashMap streamPartitionOffsets = new ConcurrentHashMap<>(); private final Map> serdes; private boolean isStarted = false; private final EventHubConfig config; @@ -161,7 +165,7 @@ public EventHubSystemConsumer(EventHubConfig config, String systemName, } @Override - public synchronized void register(SystemStreamPartition systemStreamPartition, String offset) { + public void register(SystemStreamPartition systemStreamPartition, String offset) { super.register(systemStreamPartition, offset); if (isStarted) { @@ -181,14 +185,14 @@ public synchronized void register(SystemStreamPartition systemStreamPartition, S config.getStartPosition(systemName, systemStreamPartition.getStream())); } } - streamPartitionStartingOffsets.put(systemStreamPartition, offset); + streamPartitionOffsets.put(systemStreamPartition, offset); } @Override - public synchronized void start() { + public void start() { isStarted = true; // Create receivers for Event Hubs - for (Map.Entry entry : streamPartitionStartingOffsets.entrySet()) { + for (Map.Entry entry : streamPartitionOffsets.entrySet()) { SystemStreamPartition ssp = entry.getKey(); String streamName = ssp.getStream(); Integer partitionId = ssp.getPartition().getPartitionId(); @@ -196,14 +200,14 @@ public synchronized void start() { String consumerGroup = config.getStreamConsumerGroup(systemName, streamName); String namespace = config.getStreamNamespace(systemName, streamName); String entityPath = config.getStreamEntityPath(systemName, streamName); - SamzaEventHubClient ehClientWrapper = streamEventHubClients.get(streamName); + SamzaEventHubClient samzaEventHubClient = streamEventHubClients.get(streamName); try { PartitionReceiver receiver; if (offset.equals(EventHubSystemConsumer.END_OF_STREAM)) { - receiver = ehClientWrapper.getEventHubClient() + receiver = samzaEventHubClient.getEventHubClient() .createReceiverSync(consumerGroup, partitionId.toString(), Instant.now()); } else { - receiver = ehClientWrapper.getEventHubClient() + receiver = samzaEventHubClient.getEventHubClient() .createReceiverSync(consumerGroup, partitionId.toString(), offset, !offset.equals(EventHubSystemConsumer.START_OF_STREAM)); } @@ -212,6 +216,8 @@ public synchronized void start() { serdes.getOrDefault(streamName, null)); streamPartitionHandlers.put(ssp, handler); + // Timeout for EventHubClient exponential backoff policy + receiver.setReceiveTimeout(DEFAULT_EVENTHUB_RECEIVER_TIMEOUT); receiver.setReceiveHandler(handler); streamPartitionReceivers.put(ssp, receiver); } catch (Exception e) { @@ -219,14 +225,37 @@ public synchronized void start() { String.format("Failed to create receiver for EventHubs: namespace=%s, entity=%s, partitionId=%d", namespace, entityPath, partitionId), e); } - LOG.info(String.format("Connection successfully started for namespace=%s, entity=%s ", namespace, entityPath)); + LOG.debug(String.format("Connection successfully started for namespace=%s, entity=%s ", namespace, entityPath)); } } + private void renewPartitionReceiver(SystemStreamPartition ssp) { + LOG.error(String.format("EventHub receiver timed out encountered an error for SystemStreamPartition=%s", ssp.toString())); + + SamzaEventHubClient samzaEventHubClient = streamEventHubClients.get(ssp.getStream()); + String offset = streamPartitionOffsets.get(ssp); + Integer partitionId = ssp.getPartition().getPartitionId(); + String consumerGroup = config.getStreamConsumerGroup(ssp.getSystem(), ssp.getStream()); + + try { + PartitionReceiver receiver = samzaEventHubClient.getEventHubClient() + .createReceiverSync(consumerGroup, partitionId.toString(), offset, + !offset.equals(EventHubSystemConsumer.START_OF_STREAM)); + + receiver.setReceiveTimeout(DEFAULT_EVENTHUB_RECEIVER_TIMEOUT); + receiver.setReceiveHandler(streamPartitionHandlers.get(ssp)); + streamPartitionReceivers.put(ssp, receiver); + } catch (ServiceBusException e) { + throw new SamzaException( + String.format("Failed to recreate receiver after ReceiverHandlerError for EventHubs for SystemStreamPartition=%s", + ssp), e); + } + } + @Override - public synchronized void stop() { - LOG.info("Stopping event hub system consumer..."); + public void stop() { + LOG.debug("Stopping event hub system consumer..."); List> futures = new ArrayList<>(); streamPartitionReceivers.values().forEach((receiver) -> futures.add(receiver.close())); CompletableFuture future = CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])); @@ -266,22 +295,26 @@ protected class PartitionReceiverHandlerImpl extends PartitionReceiveHandler { @Override public void onReceive(Iterable events) { if (events != null) { - events.forEach(event -> { byte[] decryptedBody = event.getBytes(); if (serde != null) { decryptedBody = serde.fromBytes(decryptedBody); } + String offset = event.getSystemProperties().getOffset(); + Object partitionKey = event.getSystemProperties().getPartitionKey(); try { updateMetrics(event); + // note that the partition key can be null - put(ssp, new EventHubIME(ssp, event.getSystemProperties().getOffset(), - event.getSystemProperties().getPartitionKey(), decryptedBody, event)); + put(ssp, new EventHubIME(ssp, offset, partitionKey, decryptedBody, event)); } catch (Exception e) { String msg = String.format("Exception while adding the event from ssp %s to dispatch queue.", ssp); LOG.error(msg, e); throw new SamzaException(msg, e); } + + // Cache latest checkpoint + streamPartitionOffsets.put(ssp, offset); }); } } @@ -299,6 +332,8 @@ private void updateMetrics(EventData event) { @Override public void onError(Throwable throwable) { + // Retry creating a receiver + renewPartitionReceiver(ssp); errorRate.inc(); aggReadErrors.inc(); LOG.error(String.format("Received error from event hub connection (ssp=%s): ", ssp), throwable); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index ce3165ae01..4a68235aba 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -38,12 +38,13 @@ import java.nio.charset.Charset; import java.time.Duration; -import java.util.HashMap; +import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.List; -import java.util.ArrayList; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; @@ -52,8 +53,8 @@ public class EventHubSystemProducer implements SystemProducer { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemProducer.class.getName()); - private static final long FLUSH_SLEEP_TIME_MILLIS = 1000; private static final long DEFAULT_SHUTDOWN_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); + private static final long DEFAULT_FLUSH_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; @@ -87,23 +88,20 @@ public enum PartitioningMethod { private final PartitioningMethod partitioningMethod; private final String systemName; - private Throwable sendExceptionOnCallback; + private volatile Throwable sendExceptionOnCallback; private boolean isStarted; // Map of the system name to the event hub client. private final Map eventHubClients = new HashMap<>(); private final Map> streamPartitionSenders = new HashMap<>(); - // Running count for the next message Id - private long messageId; private Map> serdes; - private final Map> pendingFutures = new ConcurrentHashMap<>(); + private final Set> pendingFutures = ConcurrentHashMap.newKeySet(); public EventHubSystemProducer(EventHubConfig config, String systemName, SamzaEventHubClientFactory samzaEventHubClientFactory, Map> serdes, MetricsRegistry registry) { - messageId = 0; this.config = config; this.registry = registry; this.systemName = systemName; @@ -114,7 +112,7 @@ public EventHubSystemProducer(EventHubConfig config, String systemName, @Override public synchronized void register(String streamName) { - LOG.info("Trying to register {}.", streamName); + LOG.debug("Trying to register {}.", streamName); if (isStarted) { String msg = "Cannot register once the producer is started."; throw new SamzaException(msg); @@ -128,7 +126,7 @@ public synchronized void register(String streamName) { @Override public synchronized void start() { - LOG.info("Starting system producer."); + LOG.debug("Starting system producer."); if (PartitioningMethod.PARTITION_KEY_AS_PARTITION.equals(partitioningMethod)) { // Create all partition senders @@ -192,9 +190,10 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo } if (sendExceptionOnCallback != null) { + Throwable throwable = sendExceptionOnCallback; sendExceptionOnCallback = null; pendingFutures.clear(); - throw new SamzaException(sendExceptionOnCallback); + throw new SamzaException(throwable); } EventData eventData = createEventData(destination, envelope); @@ -215,14 +214,8 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo sendLatency.get(destination).update(latencyMs); aggSendLatency.update(latencyMs); - long messageId = ++this.messageId; - // Rotate the messageIds - if (messageId == Long.MAX_VALUE) { - this.messageId = 0; - } - - pendingFutures.put(messageId, sendResult); + pendingFutures.add(sendResult); // Auto remove the future from the list when they are complete. sendResult.handle((aVoid, throwable) -> { @@ -235,7 +228,6 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo LOG.error("Send message to event hub: {} failed with exception: ", destination, throwable); sendExceptionOnCallback = throwable; } - pendingFutures.remove(messageId); return aVoid; }); } @@ -299,31 +291,31 @@ private EventData createEventData(String streamName, OutgoingMessageEnvelope env } @Override - public void flush(String source) { - LOG.info("Trying to flush pending {} sends messages: {}", pendingFutures.size(), pendingFutures.keySet()); - // Wait till all the pending sends are complete. - while (!pendingFutures.isEmpty()) { - try { - Thread.sleep(FLUSH_SLEEP_TIME_MILLIS); - } catch (InterruptedException e) { - String msg = "Flush failed with error"; - throw new SamzaException(msg, e); - } + public synchronized void flush(String source) { + LOG.debug("Trying to flush pending {} sends messages.", pendingFutures.size()); + + CompletableFuture future = CompletableFuture + .allOf(pendingFutures.toArray(new CompletableFuture[pendingFutures.size()])); + try { + // Wait until all the pending sends are complete or timeout. + future.get(DEFAULT_FLUSH_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + String msg = "Flush failed with error"; + throw new SamzaException(msg, e); } + // Check for send errors from EventHub side if (sendExceptionOnCallback != null) { String msg = "Sending one of the message failed during flush"; Throwable throwable = sendExceptionOnCallback; sendExceptionOnCallback = null; throw new SamzaException(msg, throwable); } - - LOG.info("Flush succeeded."); + pendingFutures.clear(); } @Override public synchronized void stop() { - LOG.info("Stopping event hub system producer..."); streamPartitionSenders.values().forEach((streamPartitionSender) -> { List> futures = new ArrayList<>(); streamPartitionSender.forEach((key, value) -> futures.add(value.close())); @@ -331,7 +323,7 @@ public synchronized void stop() { try { future.get(DEFAULT_SHUTDOWN_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS); } catch (ExecutionException | InterruptedException | TimeoutException e) { - LOG.warn("Closing the partition sender failed ", e); + LOG.error("Closing the partition sender failed ", e); } }); eventHubClients.values().forEach(ehClient -> ehClient.close(DEFAULT_SHUTDOWN_TIMEOUT_MILLIS)); @@ -339,6 +331,6 @@ public synchronized void stop() { } Collection> getPendingFutures() { - return pendingFutures.values(); + return pendingFutures; } } From c7e69dd940ac16c62e7f1f3e4468425e9dee76bb Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Thu, 19 Oct 2017 18:45:54 -0700 Subject: [PATCH 25/34] Renaming SamzaEventHubClient, added Consumer retry on transient error --- build.gradle | 1 + ...Client.java => EventHubClientManager.java} | 2 +- ...java => EventHubClientManagerFactory.java} | 6 +- .../eventhub/EventHubSystemFactory.java | 6 +- ...l.java => SamzaEventHubClientManager.java} | 14 +-- .../eventhub/admin/EventHubSystemAdmin.java | 40 +++++---- .../consumer/EventHubSystemConsumer.java | 89 +++++++++++++------ .../producer/EventHubSystemProducer.java | 30 ++++--- ... => MockEventHubClientManagerFactory.java} | 14 +-- .../admin/TestEventHubSystemAdmin.java | 3 +- .../consumer/TestEventHubSystemConsumer.java | 6 +- .../producer/ITestEventHubSystemProducer.java | 6 +- .../producer/TestEventHubSystemProducer.java | 6 +- 13 files changed, 135 insertions(+), 88 deletions(-) rename samza-azure/src/main/java/org/apache/samza/system/eventhub/{SamzaEventHubClient.java => EventHubClientManager.java} (97%) rename samza-azure/src/main/java/org/apache/samza/system/eventhub/{SamzaEventHubClientFactory.java => EventHubClientManagerFactory.java} (81%) rename samza-azure/src/main/java/org/apache/samza/system/eventhub/{SamzaEventHubClientImpl.java => SamzaEventHubClientManager.java} (85%) rename samza-azure/src/test/java/org/apache/samza/system/eventhub/{MockSamzaEventHubClientFactory.java => MockEventHubClientManagerFactory.java} (92%) diff --git a/build.gradle b/build.gradle index 90192ce007..d9025c48d1 100644 --- a/build.gradle +++ b/build.gradle @@ -209,6 +209,7 @@ project(':samza-azure') { toolVersion = "$checkstyleVersion" } test { + // Exclude integration tests that require connection to EventHub exclude 'org/apache/samza/system/eventhub/producer/*ITest*' exclude 'org/apache/samza/system/eventhub/consumer/*ITest*' } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClient.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientManager.java similarity index 97% rename from samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClient.java rename to samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientManager.java index b8770fc223..1324af57f4 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClient.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientManager.java @@ -24,7 +24,7 @@ /** * Wraps the {@link EventHubClient} with lifestyle hooks for initialization and close. */ -public interface SamzaEventHubClient { +public interface EventHubClientManager { /** * Initiate the connection to EventHub. */ diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientManagerFactory.java similarity index 81% rename from samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientFactory.java rename to samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientManagerFactory.java index ece49d8ff8..0578a50cda 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientManagerFactory.java @@ -19,14 +19,14 @@ package org.apache.samza.system.eventhub; -public class SamzaEventHubClientFactory { - public SamzaEventHubClient getSamzaEventHubClient(String systemName, String streamName, EventHubConfig config) { +public class EventHubClientManagerFactory { + public EventHubClientManager getEventHubClientManager(String systemName, String streamName, EventHubConfig config) { String eventHubNamespace = config.getStreamNamespace(systemName, streamName); String entityPath = config.getStreamEntityPath(systemName, streamName); String sasKeyName = config.getStreamSasKeyName(systemName, streamName); String sasToken = config.getStreamSasToken(systemName, streamName); - return new SamzaEventHubClientImpl(eventHubNamespace, entityPath, sasKeyName, sasToken); + return new SamzaEventHubClientManager(eventHubNamespace, entityPath, sasKeyName, sasToken); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java index 79d31c4cc8..5a8512d3a3 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java @@ -47,20 +47,20 @@ private Map> getSerdesMap(EventHubConfig config, String sy @Override public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { EventHubConfig eventHubConfig = new EventHubConfig(config); - return new EventHubSystemConsumer(eventHubConfig, systemName, new SamzaEventHubClientFactory(), + return new EventHubSystemConsumer(eventHubConfig, systemName, new EventHubClientManagerFactory(), getSerdesMap(eventHubConfig, systemName), registry); } @Override public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { EventHubConfig eventHubConfig = new EventHubConfig(config); - return new EventHubSystemProducer(eventHubConfig, systemName, new SamzaEventHubClientFactory(), + return new EventHubSystemProducer(eventHubConfig, systemName, new EventHubClientManagerFactory(), getSerdesMap(eventHubConfig, systemName), registry); } @Override public SystemAdmin getAdmin(String systemName, Config config) { - return new EventHubSystemAdmin(systemName, new EventHubConfig(config), new SamzaEventHubClientFactory()); + return new EventHubSystemAdmin(systemName, new EventHubConfig(config), new EventHubClientManagerFactory()); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientImpl.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientManager.java similarity index 85% rename from samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientImpl.java rename to samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientManager.java index f3c4f29654..a836da75b1 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientImpl.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientManager.java @@ -34,12 +34,12 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -public class SamzaEventHubClientImpl implements SamzaEventHubClient { - private static final Logger LOG = LoggerFactory.getLogger(SamzaEventHubClientImpl.class.getName()); +public class SamzaEventHubClientManager implements EventHubClientManager { + private static final Logger LOG = LoggerFactory.getLogger(SamzaEventHubClientManager.class.getName()); private static final String EVENTHUB_REMOTE_HOST_FORMAT = "%s.servicebus.windows.net"; - private static final Duration MIN_RETRY_BACKOFF = Duration.ofSeconds(0); - private static final Duration MAX_RETRY_BACKOFF = Duration.ofSeconds(10); + private static final Duration MIN_RETRY_BACKOFF = Duration.ofMillis(100); + private static final Duration MAX_RETRY_BACKOFF = Duration.ofMillis(11000); private static final int MAX_RETRY_COUNT = 100; private static final String SAMZA_EVENTHUB_RETRY = "SAMZA_CONNECTOR_RETRY"; @@ -51,13 +51,13 @@ public class SamzaEventHubClientImpl implements SamzaEventHubClient { private final String sasKey; private final RetryPolicy retryPolicy; - public SamzaEventHubClientImpl(String eventHubNamespace, String entityPath, String sasKeyName, String sasKey) { + public SamzaEventHubClientManager(String eventHubNamespace, String entityPath, String sasKeyName, String sasKey) { this(eventHubNamespace, entityPath, sasKeyName, sasKey, new RetryExponential(MIN_RETRY_BACKOFF, MAX_RETRY_BACKOFF, MAX_RETRY_COUNT, SAMZA_EVENTHUB_RETRY)); } - public SamzaEventHubClientImpl(String eventHubNamespace, String entityPath, String sasKeyName, String sasKey, - RetryPolicy retryPolicy) { + public SamzaEventHubClientManager(String eventHubNamespace, String entityPath, String sasKeyName, String sasKey, + RetryPolicy retryPolicy) { this.eventHubNamespace = eventHubNamespace; this.entityPath = entityPath; this.sasKeyName = sasKeyName; diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index 6a360288e4..87f7fad2ac 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -27,10 +27,10 @@ import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata; import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.system.eventhub.SamzaEventHubClient; +import org.apache.samza.system.eventhub.EventHubClientManager; import org.apache.samza.system.eventhub.EventHubConfig; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; -import org.apache.samza.system.eventhub.SamzaEventHubClientFactory; +import org.apache.samza.system.eventhub.EventHubClientManagerFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,17 +47,17 @@ public class EventHubSystemAdmin implements SystemAdmin { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemAdmin.class); private static final long DEFAULT_SHUTDOWN_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); - private final SamzaEventHubClientFactory samzaEventHubClientFactory; - private String systemName; - private EventHubConfig eventHubConfig; - private Map eventHubClients = new HashMap<>(); - private Map streamPartitions = new HashMap<>(); + private final EventHubClientManagerFactory eventHubClientManagerFactory; + private final String systemName; + private final EventHubConfig eventHubConfig; + private final Map eventHubClients = new HashMap<>(); + private final Map streamPartitions = new HashMap<>(); public EventHubSystemAdmin(String systemName, EventHubConfig eventHubConfig, - SamzaEventHubClientFactory samzaEventHubClientFactory) { + EventHubClientManagerFactory eventHubClientManagerFactory) { this.systemName = systemName; this.eventHubConfig = eventHubConfig; - this.samzaEventHubClientFactory = samzaEventHubClientFactory; + this.eventHubClientManagerFactory = eventHubClientManagerFactory; } private static String getNextOffset(String currentOffset) { @@ -82,8 +82,8 @@ public Map getSystemStreamMetadata(Set str streamNames.forEach((streamName) -> { if (!streamPartitions.containsKey(streamName)) { - SamzaEventHubClient samzaEventHubClient = getOrCreateStreamEventHubClient(streamName); - CompletableFuture runtimeInfo = samzaEventHubClient.getEventHubClient() + EventHubClientManager eventHubClientManager = getOrCreateStreamEventHubClient(streamName); + CompletableFuture runtimeInfo = eventHubClientManager.getEventHubClient() .getRuntimeInformation(); ehRuntimeInfos.put(streamName, runtimeInfo); @@ -94,11 +94,13 @@ public Map getSystemStreamMetadata(Set str ehRuntimeInfos.forEach((streamName, ehRuntimeInfo) -> { if (!streamPartitions.containsKey(streamName)) { + LOG.debug(String.format("Partition ids for Stream=%s not found", streamName)); try { long timeoutMs = eventHubConfig.getRuntimeInfoWaitTimeMS(systemName); EventHubRuntimeInformation ehInfo = ehRuntimeInfo.get(timeoutMs, TimeUnit.MILLISECONDS); + LOG.debug(String.format("Adding partition ids for Stream=%s", streamName)); streamPartitions.put(streamName, ehInfo.getPartitionIds()); } catch (InterruptedException | ExecutionException | TimeoutException e) { @@ -125,24 +127,26 @@ public Map getSystemStreamMetadata(Set str return requestedMetadata; } - private SamzaEventHubClient getOrCreateStreamEventHubClient(String streamName) { + private EventHubClientManager getOrCreateStreamEventHubClient(String streamName) { if (!eventHubClients.containsKey(streamName)) { - SamzaEventHubClient samzaEventHubClient = samzaEventHubClientFactory - .getSamzaEventHubClient(systemName, streamName, eventHubConfig); + LOG.debug(String.format("Creating EventHubClient for Stream=%s", streamName)); - samzaEventHubClient.init(); - eventHubClients.put(streamName, samzaEventHubClient); + EventHubClientManager eventHubClientManager = eventHubClientManagerFactory + .getEventHubClientManager(systemName, streamName, eventHubConfig); + + eventHubClientManager.init(); + eventHubClients.put(streamName, eventHubClientManager); } return eventHubClients.get(streamName); } private Map getPartitionMetadata(String streamName, String[] partitionIds) { - SamzaEventHubClient samzaEventHubClient = getOrCreateStreamEventHubClient(streamName); + EventHubClientManager eventHubClientManager = getOrCreateStreamEventHubClient(streamName); Map sspMetadataMap = new HashMap<>(); Map> ehRuntimeInfos = new HashMap<>(); for (String partition : partitionIds) { - CompletableFuture partitionRuntimeInfo = samzaEventHubClient + CompletableFuture partitionRuntimeInfo = eventHubClientManager .getEventHubClient() .getPartitionRuntimeInformation(partition); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index 6d1c3528ed..69646fe0df 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -30,8 +30,8 @@ import org.apache.samza.serializers.Serde; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.system.eventhub.SamzaEventHubClientFactory; -import org.apache.samza.system.eventhub.SamzaEventHubClient; +import org.apache.samza.system.eventhub.EventHubClientManagerFactory; +import org.apache.samza.system.eventhub.EventHubClientManager; import org.apache.samza.system.eventhub.EventHubConfig; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; import org.apache.samza.util.BlockingEnvelopeMap; @@ -43,6 +43,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -100,11 +101,14 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { public static final String START_OF_STREAM = PartitionReceiver.START_OF_STREAM; // -1 public static final String END_OF_STREAM = "-2"; - public static final String AGGREGATE = "aggregate"; public static final String EVENT_READ_RATE = "eventReadRate"; public static final String EVENT_BYTE_READ_RATE = "eventByteReadRate"; public static final String READ_LATENCY = "readLatency"; public static final String READ_ERRORS = "readErrors"; + public static final String AGGREGATE = "aggregate"; + + private static final Object AGGREGATE_METRICS_LOCK = new Object(); + private final Object receiverErrorLock = new Object(); private static Counter aggEventReadRate = null; private static Counter aggEventByteReadRate = null; @@ -118,16 +122,20 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { final ConcurrentHashMap streamPartitionHandlers = new ConcurrentHashMap<>(); private final ConcurrentHashMap streamPartitionReceivers = new ConcurrentHashMap<>(); - private final ConcurrentHashMap streamEventHubClients = new ConcurrentHashMap<>(); + private final ConcurrentHashMap streamEventHubManagers = new ConcurrentHashMap<>(); private final ConcurrentHashMap streamPartitionOffsets = new ConcurrentHashMap<>(); private final Map> serdes; private boolean isStarted = false; private final EventHubConfig config; private final String systemName; + // Partition receiver error propagation + private volatile Throwable perminentEventHubError = null; + private volatile SystemStreamPartition failedSSPReceiver = null; + public EventHubSystemConsumer(EventHubConfig config, String systemName, - SamzaEventHubClientFactory samzaEventHubClientFactory, + EventHubClientManagerFactory eventHubClientManagerFactory, Map> serdes, MetricsRegistry registry) { super(registry, System::currentTimeMillis); @@ -137,10 +145,10 @@ public EventHubSystemConsumer(EventHubConfig config, String systemName, List streamNames = config.getStreams(systemName); // Create and initiate connections to Event Hubs for (String streamName : streamNames) { - SamzaEventHubClient ehClientWrapper = samzaEventHubClientFactory - .getSamzaEventHubClient(systemName, streamName, config); - streamEventHubClients.put(streamName, ehClientWrapper); - ehClientWrapper.init(); + EventHubClientManager eventHubClientManager = eventHubClientManagerFactory + .getEventHubClientManager(systemName, streamName, config); + streamEventHubManagers.put(streamName, eventHubClientManager); + eventHubClientManager.init(); } // Initiate metrics @@ -154,7 +162,7 @@ public EventHubSystemConsumer(EventHubConfig config, String systemName, .collect(Collectors.toMap(Function.identity(), x -> registry.newCounter(x, READ_ERRORS))); // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. - synchronized (AGGREGATE) { + synchronized (AGGREGATE_METRICS_LOCK) { if (aggEventReadRate == null) { aggEventReadRate = registry.newCounter(AGGREGATE, EVENT_READ_RATE); aggEventByteReadRate = registry.newCounter(AGGREGATE, EVENT_BYTE_READ_RATE); @@ -193,6 +201,7 @@ public void start() { isStarted = true; // Create receivers for Event Hubs for (Map.Entry entry : streamPartitionOffsets.entrySet()) { + SystemStreamPartition ssp = entry.getKey(); String streamName = ssp.getStream(); Integer partitionId = ssp.getPartition().getPartitionId(); @@ -200,25 +209,29 @@ public void start() { String consumerGroup = config.getStreamConsumerGroup(systemName, streamName); String namespace = config.getStreamNamespace(systemName, streamName); String entityPath = config.getStreamEntityPath(systemName, streamName); - SamzaEventHubClient samzaEventHubClient = streamEventHubClients.get(streamName); + EventHubClientManager eventHubClientManager = streamEventHubManagers.get(streamName); + try { PartitionReceiver receiver; if (offset.equals(EventHubSystemConsumer.END_OF_STREAM)) { - receiver = samzaEventHubClient.getEventHubClient() + receiver = eventHubClientManager.getEventHubClient() .createReceiverSync(consumerGroup, partitionId.toString(), Instant.now()); } else { - receiver = samzaEventHubClient.getEventHubClient() + receiver = eventHubClientManager.getEventHubClient() .createReceiverSync(consumerGroup, partitionId.toString(), offset, !offset.equals(EventHubSystemConsumer.START_OF_STREAM)); } + PartitionReceiveHandler handler = new PartitionReceiverHandlerImpl(ssp, eventReadRates.get(streamName), eventByteReadRates.get(streamName), readLatencies.get(streamName), readErrors.get(streamName), serdes.getOrDefault(streamName, null)); - streamPartitionHandlers.put(ssp, handler); - // Timeout for EventHubClient exponential backoff policy + + // Timeout for EventHubClient receive receiver.setReceiveTimeout(DEFAULT_EVENTHUB_RECEIVER_TIMEOUT); receiver.setReceiveHandler(handler); + + streamPartitionHandlers.put(ssp, handler); streamPartitionReceivers.put(ssp, receiver); } catch (Exception e) { throw new SamzaException( @@ -230,22 +243,33 @@ public void start() { } } - private void renewPartitionReceiver(SystemStreamPartition ssp) { - LOG.error(String.format("EventHub receiver timed out encountered an error for SystemStreamPartition=%s", ssp.toString())); + @Override + public Map> poll(Set systemStreamPartitions, long timeout) throws InterruptedException { + if (perminentEventHubError != null) { + String msg = String.format("Received a non transient error from event hub partition receiver (ssp=%s)", failedSSPReceiver); + throw new SamzaException(msg, perminentEventHubError); + } + return super.poll(systemStreamPartitions, timeout); + } + + private void renewPartitionReceiver(SystemStreamPartition ssp, Throwable throwable) { + LOG.warn(String.format("Received a transient error from event hub partition receiver (ssp=%s), restarting receiver: ", ssp), throwable); - SamzaEventHubClient samzaEventHubClient = streamEventHubClients.get(ssp.getStream()); + EventHubClientManager eventHubClientManager = streamEventHubManagers.get(ssp.getStream()); String offset = streamPartitionOffsets.get(ssp); Integer partitionId = ssp.getPartition().getPartitionId(); String consumerGroup = config.getStreamConsumerGroup(ssp.getSystem(), ssp.getStream()); try { - PartitionReceiver receiver = samzaEventHubClient.getEventHubClient() + PartitionReceiver receiver = eventHubClientManager.getEventHubClient() .createReceiverSync(consumerGroup, partitionId.toString(), offset, !offset.equals(EventHubSystemConsumer.START_OF_STREAM)); + // Timeout for EventHubClient receive receiver.setReceiveTimeout(DEFAULT_EVENTHUB_RECEIVER_TIMEOUT); receiver.setReceiveHandler(streamPartitionHandlers.get(ssp)); streamPartitionReceivers.put(ssp, receiver); + } catch (ServiceBusException e) { throw new SamzaException( String.format("Failed to recreate receiver after ReceiverHandlerError for EventHubs for SystemStreamPartition=%s", @@ -264,7 +288,7 @@ public void stop() { } catch (ExecutionException | InterruptedException | TimeoutException e) { LOG.warn("Failed to close receivers", e); } - streamEventHubClients.values().forEach(ehClient -> ehClient.close(DEFAULT_SHUTDOWN_TIMEOUT_MILLIS)); + streamEventHubManagers.values().forEach(ehClientManager -> ehClientManager.close(DEFAULT_SHUTDOWN_TIMEOUT_MILLIS)); } @Override @@ -307,8 +331,8 @@ public void onReceive(Iterable events) { // note that the partition key can be null put(ssp, new EventHubIME(ssp, offset, partitionKey, decryptedBody, event)); - } catch (Exception e) { - String msg = String.format("Exception while adding the event from ssp %s to dispatch queue.", ssp); + } catch (InterruptedException e) { + String msg = String.format("Interrupted while adding the event from ssp %s to dispatch queue.", ssp); LOG.error(msg, e); throw new SamzaException(msg, e); } @@ -325,6 +349,7 @@ private void updateMetrics(EventData event) { aggEventReadRate.inc(); eventByteReadRate.inc(eventDataLength); aggEventByteReadRate.inc(eventDataLength); + long latencyMs = Duration.between(Instant.now(), event.getSystemProperties().getEnqueuedTime()).toMillis(); readLatency.update(latencyMs); aggReadLatency.update(latencyMs); @@ -332,11 +357,25 @@ private void updateMetrics(EventData event) { @Override public void onError(Throwable throwable) { - // Retry creating a receiver - renewPartitionReceiver(ssp); errorRate.inc(); aggReadErrors.inc(); - LOG.error(String.format("Received error from event hub connection (ssp=%s): ", ssp), throwable); + + if (throwable instanceof ServiceBusException) { + ServiceBusException busException = (ServiceBusException) throwable; + + if (busException.getIsTransient()) { + // Retry creating a receiver since error likely due to timeout + + renewPartitionReceiver(ssp, throwable); + return; + } + } + + // Non transient Error propagated to user + synchronized (receiverErrorLock) { + perminentEventHubError = throwable; + failedSSPReceiver = ssp; + } } } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 4a68235aba..ccc17e5fd4 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -29,9 +29,9 @@ import org.apache.samza.serializers.Serde; import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemProducer; -import org.apache.samza.system.eventhub.SamzaEventHubClient; +import org.apache.samza.system.eventhub.EventHubClientManager; import org.apache.samza.system.eventhub.EventHubConfig; -import org.apache.samza.system.eventhub.SamzaEventHubClientFactory; +import org.apache.samza.system.eventhub.EventHubClientManagerFactory; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,7 +59,6 @@ public class EventHubSystemProducer implements SystemProducer { public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; // Metrics recording - public static final String AGGREGATE = "aggregate"; private static final String EVENT_WRITE_RATE = "eventWriteRate"; private static final String EVENT_BYTE_WRITE_RATE = "eventByteWriteRate"; private static final String SEND_ERRORS = "sendErrors"; @@ -70,6 +69,9 @@ public class EventHubSystemProducer implements SystemProducer { private static Counter aggSendErrors = null; private static SamzaHistogram aggSendLatency = null; private static SamzaHistogram aggSendCallbackLatency = null; + private static final String AGGREGATE = "aggregate"; + + private static final Object AGGREGATE_METRICS_LOCK = new Object(); public enum PartitioningMethod { EVENT_HUB_HASHING, @@ -82,31 +84,31 @@ public enum PartitioningMethod { private final HashMap sendCallbackLatency = new HashMap<>(); private final HashMap sendErrors = new HashMap<>(); - private final SamzaEventHubClientFactory samzaEventHubClientFactory; + private final EventHubClientManagerFactory eventHubClientManagerFactory; private final EventHubConfig config; private final MetricsRegistry registry; private final PartitioningMethod partitioningMethod; private final String systemName; - private volatile Throwable sendExceptionOnCallback; - private boolean isStarted; + private volatile Throwable sendExceptionOnCallback = null; + private volatile boolean isStarted = false; // Map of the system name to the event hub client. - private final Map eventHubClients = new HashMap<>(); + private final Map eventHubClients = new HashMap<>(); private final Map> streamPartitionSenders = new HashMap<>(); - private Map> serdes; + private final Map> serdes; private final Set> pendingFutures = ConcurrentHashMap.newKeySet(); public EventHubSystemProducer(EventHubConfig config, String systemName, - SamzaEventHubClientFactory samzaEventHubClientFactory, + EventHubClientManagerFactory eventHubClientManagerFactory, Map> serdes, MetricsRegistry registry) { this.config = config; this.registry = registry; this.systemName = systemName; this.partitioningMethod = config.getPartitioningMethod(systemName); - this.samzaEventHubClientFactory = samzaEventHubClientFactory; + this.eventHubClientManagerFactory = eventHubClientManagerFactory; this.serdes = serdes; } @@ -118,7 +120,7 @@ public synchronized void register(String streamName) { throw new SamzaException(msg); } - SamzaEventHubClient ehClient = samzaEventHubClientFactory.getSamzaEventHubClient(systemName, streamName, config); + EventHubClientManager ehClient = eventHubClientManagerFactory.getEventHubClientManager(systemName, streamName, config); ehClient.init(); eventHubClients.put(streamName, ehClient); @@ -165,7 +167,7 @@ public synchronized void start() { } // Locking to ensure that these aggregated metrics will be created only once across multiple system consumers. - synchronized (AGGREGATE) { + synchronized (AGGREGATE_METRICS_LOCK) { if (aggEventWriteRate == null) { aggEventWriteRate = registry.newCounter(AGGREGATE, EVENT_WRITE_RATE); aggEventByteWriteRate = registry.newCounter(AGGREGATE, EVENT_BYTE_WRITE_RATE); @@ -202,7 +204,7 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo aggEventWriteRate.inc(); eventByteWriteRate.get(destination).inc(eventDataLength); aggEventByteWriteRate.inc(eventDataLength); - SamzaEventHubClient ehClient = eventHubClients.get(destination); + EventHubClientManager ehClient = eventHubClients.get(destination); long beforeSendTimeMs = System.currentTimeMillis(); @@ -217,7 +219,7 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo pendingFutures.add(sendResult); - // Auto remove the future from the list when they are complete. + // Auto update the metrics and possible throwable when futures are complete. sendResult.handle((aVoid, throwable) -> { long callbackLatencyMs = System.currentTimeMillis() - afterSendTimeMs; sendCallbackLatency.get(destination).update(callbackLatencyMs); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockSamzaEventHubClientFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java similarity index 92% rename from samza-azure/src/test/java/org/apache/samza/system/eventhub/MockSamzaEventHubClientFactory.java rename to samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java index 2df370a862..e9c1da8f1c 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockSamzaEventHubClientFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java @@ -35,20 +35,20 @@ import static org.mockito.Matchers.*; -public class MockSamzaEventHubClientFactory extends SamzaEventHubClientFactory { +public class MockEventHubClientManagerFactory extends EventHubClientManagerFactory { private Map> eventData; private Map>>> receivedData; - public MockSamzaEventHubClientFactory() { + public MockEventHubClientManagerFactory() { this.receivedData = new HashMap<>(); } - public MockSamzaEventHubClientFactory(Map> eventData) { + public MockEventHubClientManagerFactory(Map> eventData) { this.eventData = eventData; } @Override - public SamzaEventHubClient getSamzaEventHubClient(String systemName, String streamName, EventHubConfig config) { + public EventHubClientManager getEventHubClientManager(String systemName, String streamName, EventHubConfig config) { if (receivedData != null) { if (!receivedData.containsKey(systemName)) { receivedData.put(systemName, new HashMap<>()); @@ -60,7 +60,7 @@ public SamzaEventHubClient getSamzaEventHubClient(String systemName, String stre receivedData.get(systemName).get(streamName).put(1, new ArrayList<>()); } } - return new MockSamzaEventHubClient(systemName, streamName); + return new MockEventHubClientManager(systemName, streamName); } // Emulate EventHub sending data @@ -76,13 +76,13 @@ public List getSentData(String systemName, String streamName, Integer return null; } - private class MockSamzaEventHubClient implements SamzaEventHubClient { + private class MockEventHubClientManager implements EventHubClientManager { Boolean initiated = false; EventHubClient mockEventHubClient = PowerMockito.mock(EventHubClient.class); String systemName; String streamName; - MockSamzaEventHubClient(String systemName, String streamName) { + MockEventHubClientManager(String systemName, String streamName) { this.systemName = systemName; this.streamName = streamName; diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java index bc34ea7c08..c49e68eb70 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/admin/TestEventHubSystemAdmin.java @@ -104,7 +104,8 @@ public void testGetStreamMetadata() { Assert.assertNotSame(EventHubSystemConsumer.END_OF_STREAM, metadata.getNewestOffset()); Assert.assertTrue(Long.parseLong(EventHubSystemConsumer.END_OF_STREAM) <= Long.parseLong(metadata.getNewestOffset())); - Assert.assertEquals(EventHubSystemConsumer.END_OF_STREAM, metadata.getUpcomingOffset()); + String expectedUpcomingOffset = String.valueOf(Long.parseLong(metadata.getNewestOffset()) + 1); + Assert.assertEquals(expectedUpcomingOffset, metadata.getUpcomingOffset()); }); } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index 611b322a0c..2b87542234 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -83,7 +83,7 @@ public void testSinglePartitionConsumptionHappyPath() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "latest"); - MockSamzaEventHubClientFactory eventHubClientWrapperFactory = new MockSamzaEventHubClientFactory(eventData); + MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, serdes, @@ -134,7 +134,7 @@ public void testMultiPartitionConsumptionHappyPath() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); - MockSamzaEventHubClientFactory eventHubClientWrapperFactory = new MockSamzaEventHubClientFactory(eventData); + MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, serdes, @@ -194,7 +194,7 @@ public void testMultiStreamsConsumptionHappyPath() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName2), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName2), EVENTHUB_KEY); - MockSamzaEventHubClientFactory eventHubClientWrapperFactory = new MockSamzaEventHubClientFactory(eventData); + MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, serdes, diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/ITestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/ITestEventHubSystemProducer.java index 1a81517541..cc401984ce 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/ITestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/ITestEventHubSystemProducer.java @@ -91,9 +91,9 @@ public void testSend() { @Test public void testReceive() throws ServiceBusException { - SamzaEventHubClientFactory clientFactory = new SamzaEventHubClientFactory(); - SamzaEventHubClient wrapper = clientFactory - .getSamzaEventHubClient(SYSTEM_NAME, STREAM_NAME1, new EventHubConfig(createEventHubConfig())); + EventHubClientManagerFactory clientFactory = new EventHubClientManagerFactory(); + EventHubClientManager wrapper = clientFactory + .getEventHubClientManager(SYSTEM_NAME, STREAM_NAME1, new EventHubConfig(createEventHubConfig())); wrapper.init(); EventHubClient client = wrapper.getEventHubClient(); PartitionReceiver receiver = diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java index 5bd17df78e..e1845870eb 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -25,7 +25,7 @@ import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemStream; import org.apache.samza.system.eventhub.EventHubConfig; -import org.apache.samza.system.eventhub.MockSamzaEventHubClientFactory; +import org.apache.samza.system.eventhub.MockEventHubClientManagerFactory; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer.PartitioningMethod; import org.apache.samza.system.eventhub.TestMetricsRegistry; import org.junit.Assert; @@ -78,7 +78,7 @@ public void testSendingToSpecificPartitions() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName), PartitioningMethod.PARTITION_KEY_AS_PARTITION.toString()); - MockSamzaEventHubClientFactory factory = new MockSamzaEventHubClientFactory(); + MockEventHubClientManagerFactory factory = new MockEventHubClientManagerFactory(); EventHubSystemProducer producer = new EventHubSystemProducer(new EventHubConfig(configMap), systemName, factory, serdes, testMetrics); @@ -129,7 +129,7 @@ public void testSendingToEventHubHashing() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName), PartitioningMethod.EVENT_HUB_HASHING.toString()); - MockSamzaEventHubClientFactory factory = new MockSamzaEventHubClientFactory(); + MockEventHubClientManagerFactory factory = new MockEventHubClientManagerFactory(); EventHubSystemProducer producer = new EventHubSystemProducer(new EventHubConfig(configMap), systemName, factory, serdes, testMetrics); From e958073f04717cdec02366d1394d3e96b8de7573 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 23 Oct 2017 13:40:18 -0700 Subject: [PATCH 26/34] Support multiple register consumer calls to same stream --- .../eventhub/admin/EventHubSystemAdmin.java | 8 +++- .../consumer/EventHubSystemConsumer.java | 9 +++++ .../MockEventHubClientManagerFactory.java | 19 ++++++++- .../consumer/TestEventHubSystemConsumer.java | 40 +++++++++++++++++++ 4 files changed, 72 insertions(+), 4 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index 87f7fad2ac..ed482933f1 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -177,8 +177,7 @@ private Map getPartitionMetadata(Strin return sspMetadataMap; } - @Override - public Integer offsetComparator(String offset1, String offset2) { + public static Integer compareOffsets(String offset1, String offset2) { if (offset1 == null || offset2 == null) { return null; } @@ -192,4 +191,9 @@ public Integer offsetComparator(String offset1, String offset2) { return null; } } + + @Override + public Integer offsetComparator(String offset1, String offset2) { + return compareOffsets(offset1, offset2); + } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index 69646fe0df..b0f4517e67 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -33,6 +33,7 @@ import org.apache.samza.system.eventhub.EventHubClientManagerFactory; import org.apache.samza.system.eventhub.EventHubClientManager; import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.admin.EventHubSystemAdmin; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; import org.apache.samza.util.BlockingEnvelopeMap; import org.slf4j.Logger; @@ -193,6 +194,14 @@ public void register(SystemStreamPartition systemStreamPartition, String offset) config.getStartPosition(systemName, systemStreamPartition.getStream())); } } + + if (streamPartitionOffsets.containsKey(systemStreamPartition)) { + String prevOffset = streamPartitionOffsets.get(systemStreamPartition); + if (EventHubSystemAdmin.compareOffsets(offset, prevOffset) > -1) { + // Only update if new offset is lower than previous offset + return; + } + } streamPartitionOffsets.put(systemStreamPartition, offset); } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java index e9c1da8f1c..1f06f7d310 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubClientManagerFactory.java @@ -21,6 +21,7 @@ import com.microsoft.azure.eventhubs.*; import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.junit.Assert; import org.mockito.stubbing.Answer; import org.powermock.api.mockito.PowerMockito; @@ -38,6 +39,7 @@ public class MockEventHubClientManagerFactory extends EventHubClientManagerFactory { private Map> eventData; private Map>>> receivedData; + private Map startingOffsets = new HashMap<>(); public MockEventHubClientManagerFactory() { this.receivedData = new HashMap<>(); @@ -69,6 +71,10 @@ public void sendToHandlers(Map h handlers.forEach((ssp, value) -> value.onReceive(eventData.get(ssp))); } + public String getPartitionOffset(String partitionId) { + return startingOffsets.getOrDefault(partitionId, null); + } + public List getSentData(String systemName, String streamName, Integer partitionId) { if (receivedData.containsKey(systemName) && receivedData.get(systemName).containsKey(streamName)) { return receivedData.get(systemName).get(streamName).get(partitionId); @@ -119,9 +125,18 @@ private class MockEventHubClientManager implements EventHubClientManager { try { // Consumer calls PowerMockito.when(mockEventHubClient.createReceiverSync(anyString(), anyString(), any(Instant.class))) - .thenReturn(mockPartitionReceiver); + .then((Answer) invocationOnMock -> { + String partitionId = invocationOnMock.getArgumentAt(1, String.class); + startingOffsets.put(partitionId, EventHubSystemConsumer.END_OF_STREAM); + return mockPartitionReceiver; + }); PowerMockito.when(mockEventHubClient.createReceiverSync(anyString(), anyString(), anyString(), anyBoolean())) - .thenReturn(mockPartitionReceiver); + .then((Answer) invocationOnMock -> { + String partitionId = invocationOnMock.getArgumentAt(1, String.class); + String offset = invocationOnMock.getArgumentAt(2, String.class); + startingOffsets.put(partitionId, offset); + return mockPartitionReceiver; + }); // Producer calls PowerMockito.when(mockEventHubClient.createPartitionSenderSync("0")).thenReturn(mockPartitionSender0); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index 2b87542234..218b42eb8d 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -57,6 +57,46 @@ private void verifyEvents(List messages, List> eventData = new HashMap<>(); + SystemStreamPartition ssp = new SystemStreamPartition(systemName, streamName, new Partition(partitionId)); + Map> serdes = new HashMap<>(); + serdes.put(streamName, new ByteSerde()); + + // create EventData + List singlePartitionEventData = MockEventData.generateEventData(numEvents); + eventData.put(ssp, singlePartitionEventData); + + // Set configs + Map configMap = new HashMap<>(); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_LIST, systemName), streamName); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, systemName, streamName), EVENTHUB_NAMESPACE); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); + configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "earliest"); + + MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); + + EventHubSystemConsumer consumer = + new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, serdes, + testMetrics); + consumer.register(ssp, "1"); + consumer.register(ssp, EventHubSystemConsumer.END_OF_STREAM); + consumer.register(ssp, null); + consumer.start(); + + Assert.assertEquals(EventHubSystemConsumer.START_OF_STREAM, + eventHubClientWrapperFactory.getPartitionOffset(String.valueOf(partitionId))); + } + @Test public void testSinglePartitionConsumptionHappyPath() throws Exception { String systemName = "eventhubs"; From a452a8ec8d4d1f4ecb987ea4ccd194868d5cdf03 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Mon, 23 Oct 2017 13:46:17 -0700 Subject: [PATCH 27/34] Support multiple register producer calls to same stream --- .../system/eventhub/producer/EventHubSystemProducer.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index ccc17e5fd4..9c2df6ad37 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -120,6 +120,11 @@ public synchronized void register(String streamName) { throw new SamzaException(msg); } + if (eventHubClients.containsKey(streamName)) { + LOG.warn("Already registered stream {}.", streamName); + return; + } + EventHubClientManager ehClient = eventHubClientManagerFactory.getEventHubClientManager(systemName, streamName, config); ehClient.init(); From 6c0aa47482c7d31a37806bb887bfd4b59a2ae138 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Tue, 24 Oct 2017 14:17:43 -0700 Subject: [PATCH 28/34] Save first throwable --- .../producer/EventHubSystemProducer.java | 35 ++++++++++--------- 1 file changed, 19 insertions(+), 16 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 9c2df6ad37..68634d6f42 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -50,6 +50,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; public class EventHubSystemProducer implements SystemProducer { private static final Logger LOG = LoggerFactory.getLogger(EventHubSystemProducer.class.getName()); @@ -57,6 +58,7 @@ public class EventHubSystemProducer implements SystemProducer { private static final long DEFAULT_FLUSH_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; + public static final String FLUSH_ALL = "all"; // Metrics recording private static final String EVENT_WRITE_RATE = "eventWriteRate"; @@ -90,7 +92,7 @@ public enum PartitioningMethod { private final PartitioningMethod partitioningMethod; private final String systemName; - private volatile Throwable sendExceptionOnCallback = null; + private final AtomicReference sendExceptionOnCallback = new AtomicReference<>(null); private volatile boolean isStarted = false; // Map of the system name to the event hub client. @@ -196,12 +198,7 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo throw new SamzaException(msg); } - if (sendExceptionOnCallback != null) { - Throwable throwable = sendExceptionOnCallback; - sendExceptionOnCallback = null; - pendingFutures.clear(); - throw new SamzaException(throwable); - } + checkCallbackThrowable("Received exception on message send"); EventData eventData = createEventData(destination, envelope); int eventDataLength = eventData.getBytes() == null ? 0 : eventData.getBytes().length; @@ -213,6 +210,7 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo long beforeSendTimeMs = System.currentTimeMillis(); + // Async send call CompletableFuture sendResult = sendToEventHub(destination, eventData, getEnvelopePartitionId(envelope), ehClient.getEventHubClient()); @@ -221,7 +219,6 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo sendLatency.get(destination).update(latencyMs); aggSendLatency.update(latencyMs); - pendingFutures.add(sendResult); // Auto update the metrics and possible throwable when futures are complete. @@ -233,7 +230,7 @@ public synchronized void send(String destination, OutgoingMessageEnvelope envelo sendErrors.get(destination).inc(); aggSendErrors.inc(); LOG.error("Send message to event hub: {} failed with exception: ", destination, throwable); - sendExceptionOnCallback = throwable; + sendExceptionOnCallback.compareAndSet(null, throwable); } return aVoid; }); @@ -300,29 +297,35 @@ private EventData createEventData(String streamName, OutgoingMessageEnvelope env @Override public synchronized void flush(String source) { LOG.debug("Trying to flush pending {} sends messages.", pendingFutures.size()); + checkCallbackThrowable("Received exception on message send"); CompletableFuture future = CompletableFuture - .allOf(pendingFutures.toArray(new CompletableFuture[pendingFutures.size()])); + .allOf(pendingFutures.toArray(new CompletableFuture[pendingFutures.size()])); + try { - // Wait until all the pending sends are complete or timeout. + // Block until all the pending sends are complete or timeout. future.get(DEFAULT_FLUSH_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { String msg = "Flush failed with error"; throw new SamzaException(msg, e); } + checkCallbackThrowable("Sending one or more of the messages failed during flush"); + } + + private void checkCallbackThrowable(String msg) { // Check for send errors from EventHub side - if (sendExceptionOnCallback != null) { - String msg = "Sending one of the message failed during flush"; - Throwable throwable = sendExceptionOnCallback; - sendExceptionOnCallback = null; - throw new SamzaException(msg, throwable); + Throwable sendThrowable = sendExceptionOnCallback.get(); + if (sendThrowable != null) { + throw new SamzaException(msg, sendThrowable); } pendingFutures.clear(); } @Override public synchronized void stop() { + LOG.debug("Stopping producer.", pendingFutures.size()); + streamPartitionSenders.values().forEach((streamPartitionSender) -> { List> futures = new ArrayList<>(); streamPartitionSender.forEach((key, value) -> futures.add(value.close())); From ac62987043ad0592070e6055fef3a4854a52ab29 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Tue, 24 Oct 2017 16:59:29 -0700 Subject: [PATCH 29/34] Removed starting offset default in EH consumer, use system level config --- .../samza/system/eventhub/EventHubConfig.java | 21 ------------------- .../consumer/EventHubSystemConsumer.java | 15 ------------- .../eventhub/MockEventHubConfigFactory.java | 2 -- .../consumer/ITestEventHubSystemConsumer.java | 2 +- .../consumer/TestEventHubSystemConsumer.java | 6 ++---- .../producer/TestEventHubSystemProducer.java | 2 -- 6 files changed, 3 insertions(+), 45 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java index b3b0d9c296..3bc04f885b 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubConfig.java @@ -29,11 +29,6 @@ public class EventHubConfig extends MapConfig { - public enum StartPosition { - EARLIEST, - LATEST - } - public static final String CONFIG_STREAM_LIST = "systems.%s.stream.list"; public static final String CONFIG_STREAM_NAMESPACE = "systems.%s.streams.%s.eventhubs.namespace"; @@ -47,9 +42,6 @@ public enum StartPosition { public static final String CONFIG_STREAM_CONSUMER_GROUP = "systems.%s.streams.%s.eventhubs.consumer.group"; public static final String DEFAULT_CONFIG_STREAM_CONSUMER_GROUP = EventHubClient.DEFAULT_CONSUMER_GROUP_NAME; - public static final String CONFIG_STREAM_CONSUMER_START_POSITION = "systems.%s.streams.%s.eventhubs.start.position"; - public static final String DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION = StartPosition.LATEST.name(); - public static final String CONFIG_PRODUCER_PARTITION_METHOD = "systems.%s.eventhubs.partition.method"; public static final String DEFAULT_CONFIG_PRODUCER_PARTITION_METHOD = EventHubSystemProducer .PartitioningMethod.EVENT_HUB_HASHING.name(); @@ -134,19 +126,6 @@ public String getStreamConsumerGroup(String systemName, String streamName) { return get(String.format(CONFIG_STREAM_CONSUMER_GROUP, systemName, streamName), DEFAULT_CONFIG_STREAM_CONSUMER_GROUP); } - /** - * Get the start position when there is no checkpoints. By default the consumer starts from latest (end of stream) - * - * @param systemName name of the system - * @param streamName name of the stream - * @return Starting position when no checkpoints - */ - public StartPosition getStartPosition(String systemName, String streamName) { - String startPositionStr = get(String.format(CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), - DEFAULT_CONFIG_STREAM_CONSUMER_START_POSITION); - return StartPosition.valueOf(startPositionStr.toUpperCase()); - } - /** * Get the partition method of the systemName. By default partitioning is handed by EventHub. * diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index b0f4517e67..34dae3a6f5 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -23,7 +23,6 @@ import com.microsoft.azure.eventhubs.PartitionReceiveHandler; import com.microsoft.azure.eventhubs.PartitionReceiver; import com.microsoft.azure.servicebus.ServiceBusException; -import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.MetricsRegistry; @@ -181,20 +180,6 @@ public void register(SystemStreamPartition systemStreamPartition, String offset) throw new SamzaException("Trying to add partition when the connection has already started."); } - if (StringUtils.isBlank(offset)) { - switch (config.getStartPosition(systemName, systemStreamPartition.getStream())) { - case EARLIEST: - offset = START_OF_STREAM; - break; - case LATEST: - offset = END_OF_STREAM; - break; - default: - throw new SamzaException("Unknown starting position config " + - config.getStartPosition(systemName, systemStreamPartition.getStream())); - } - } - if (streamPartitionOffsets.containsKey(systemStreamPartition)) { String prevOffset = streamPartitionOffsets.get(systemStreamPartition); if (EventHubSystemAdmin.compareOffsets(offset, prevOffset) > -1) { diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java index bf944b3f52..1d8e0ce88b 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/MockEventHubConfigFactory.java @@ -50,13 +50,11 @@ public static Config getEventHubConfig(EventHubSystemProducer.PartitioningMethod mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_ENTITY1); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY_NAME); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME1), EVENTHUB_KEY); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME1), "earliest"); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_NAMESPACE, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_NAMESPACE); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_ENTITY2); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY_NAME); mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, SYSTEM_NAME, STREAM_NAME2), EVENTHUB_KEY); - mapConfig.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, SYSTEM_NAME, STREAM_NAME2), "earliest"); return new MapConfig(mapConfig); } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java index 374301d0b3..79ce0837dd 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java @@ -54,7 +54,7 @@ public void testSinglePartitionConsumptionHappyPath() throws Exception { EventHubSystemFactory factory = new EventHubSystemFactory(); SystemConsumer consumer = factory.getConsumer(SYSTEM_NAME, eventHubConfig, testMetrics); - consumer.register(ssp, null); + consumer.register(ssp, EventHubSystemConsumer.START_OF_STREAM); consumer.start(); int numEvents = 0; diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index 218b42eb8d..42aea3688c 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -81,7 +81,6 @@ public void testMultipleRegistersToSameSSP() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "earliest"); MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); @@ -90,7 +89,7 @@ public void testMultipleRegistersToSameSSP() throws Exception { testMetrics); consumer.register(ssp, "1"); consumer.register(ssp, EventHubSystemConsumer.END_OF_STREAM); - consumer.register(ssp, null); + consumer.register(ssp, EventHubSystemConsumer.START_OF_STREAM); consumer.start(); Assert.assertEquals(EventHubSystemConsumer.START_OF_STREAM, @@ -121,14 +120,13 @@ public void testSinglePartitionConsumptionHappyPath() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), MOCK_ENTITY_1); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "latest"); MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, serdes, testMetrics); - consumer.register(ssp, null); + consumer.register(ssp, EventHubSystemConsumer.END_OF_STREAM); consumer.start(); // Mock received data from EventHub diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java index e1845870eb..b5afffeef7 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -74,7 +74,6 @@ public void testSendingToSpecificPartitions() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), EVENTHUB_ENTITY1); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "latest"); configMap.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName), PartitioningMethod.PARTITION_KEY_AS_PARTITION.toString()); @@ -124,7 +123,6 @@ public void testSendingToEventHubHashing() throws Exception { configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_KEY_NAME, systemName, streamName), EVENTHUB_KEY_NAME); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_SAS_TOKEN, systemName, streamName), EVENTHUB_KEY); configMap.put(String.format(EventHubConfig.CONFIG_STREAM_ENTITYPATH, systemName, streamName), EVENTHUB_ENTITY1); - configMap.put(String.format(EventHubConfig.CONFIG_STREAM_CONSUMER_START_POSITION, systemName, streamName), "latest"); // mod 2 on the partitionid to simulate consistent hashing configMap.put(String.format(EventHubConfig.CONFIG_PRODUCER_PARTITION_METHOD, systemName), PartitioningMethod.EVENT_HUB_HASHING.toString()); From 503df5701f602e94dcc7dcde892bdab2df2e2de4 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Wed, 25 Oct 2017 13:04:41 -0700 Subject: [PATCH 30/34] minor pr comments and docs --- .../eventhub/EventHubClientManager.java | 37 +++++++++++++++---- .../eventhub/SamzaEventHubClientManager.java | 9 +++-- .../eventhub/admin/EventHubSystemAdmin.java | 2 +- .../consumer/EventHubSystemConsumer.java | 10 +++-- 4 files changed, 42 insertions(+), 16 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientManager.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientManager.java index 1324af57f4..0b4f18fa23 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientManager.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubClientManager.java @@ -22,27 +22,48 @@ import com.microsoft.azure.eventhubs.EventHubClient; /** - * Wraps the {@link EventHubClient} with lifestyle hooks for initialization and close. + *

+ * EventHubClient manager is the interface that must be implemented to wrap the + * {@link EventHubClient} with lifecycle hooks for initialization and close. + *

+ * + *

+ * {@link #init()} should be invoked once during the startup and provides a + * hook to perform some initialization before the creation of the underlying + * {@link EventHubClient}. {@link #close(long)} is invoked once during shut-down + * and can be used to perform clean-ups. + *

*/ public interface EventHubClientManager { /** - * Initiate the connection to EventHub. + * A constant that can be used in the close method's timeout parameter to + * denote that the close invocation should block until all the teardown + * operations for the {@link EventHubClient} are completed + */ + public static int BLOCK_UNTIL_CLOSE = -1; + + /** + * Lifecycle hook to perform initializations for the creation of + * the underlying {@link EventHubClient}. */ void init(); /** - * Returns the EventHubClient instance of the wrapper so its methods can be invoked directly. + * Returns the underlying {@link EventHubClient} instance. Multiple invocations + * of this method should return the same instance instead of + * creating new ones. * * @return EventHub client instance of the wrapper */ EventHubClient getEventHubClient(); /** - * Timed synchronous connection close to the EventHub. + * Tries to close the {@link EventHubClient} instance within the provided + * timeout. Use this method to perform clean-ups after the execution of the + * {@link EventHubClient}. Set timeout the {@link #BLOCK_UNTIL_CLOSE} to + * block until the client is closed. * - * @param timeoutMs - * Time in Milliseconds to wait for individual components to - * shutdown before moving to the next stage. + * @param timeoutMs Close timeout in Milliseconds */ void close(long timeoutMs); -} \ No newline at end of file +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientManager.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientManager.java index a836da75b1..ada5694123 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientManager.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/SamzaEventHubClientManager.java @@ -38,6 +38,7 @@ public class SamzaEventHubClientManager implements EventHubClientManager { private static final Logger LOG = LoggerFactory.getLogger(SamzaEventHubClientManager.class.getName()); private static final String EVENTHUB_REMOTE_HOST_FORMAT = "%s.servicebus.windows.net"; + private static final Duration MIN_RETRY_BACKOFF = Duration.ofMillis(100); private static final Duration MAX_RETRY_BACKOFF = Duration.ofMillis(11000); private static final int MAX_RETRY_COUNT = 100; @@ -65,6 +66,7 @@ public SamzaEventHubClientManager(String eventHubNamespace, String entityPath, S this.retryPolicy = retryPolicy; } + @Override public void init() { String remoteHost = String.format(EVENTHUB_REMOTE_HOST_FORMAT, eventHubNamespace); try { @@ -80,21 +82,22 @@ public void init() { } } + @Override public EventHubClient getEventHubClient() { return eventHubClient; } - + @Override public void close(long timeoutMS) { try { - if (timeoutMS <= 0) { + if (timeoutMS == EventHubClientManager.BLOCK_UNTIL_CLOSE) { eventHubClient.closeSync(); } else { CompletableFuture future = eventHubClient.close(); future.get(timeoutMS, TimeUnit.MILLISECONDS); } } catch (Exception e) { - LOG.error("Closing the event hub client failed ", e); + LOG.error("Closing the EventHub client failed", e); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java index ed482933f1..11998a4734 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/EventHubSystemAdmin.java @@ -60,7 +60,7 @@ public EventHubSystemAdmin(String systemName, EventHubConfig eventHubConfig, this.eventHubClientManagerFactory = eventHubClientManagerFactory; } - private static String getNextOffset(String currentOffset) { + private String getNextOffset(String currentOffset) { // EventHub will return the first message AFTER the offset // that was specified in the fetch request. return currentOffset.equals(EventHubSystemConsumer.END_OF_STREAM) ? currentOffset : diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index 34dae3a6f5..66d101d181 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -130,7 +130,7 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { private final String systemName; // Partition receiver error propagation - private volatile Throwable perminentEventHubError = null; + private volatile Throwable permanentEventHubError = null; private volatile SystemStreamPartition failedSSPReceiver = null; @@ -223,6 +223,8 @@ public void start() { // Timeout for EventHubClient receive receiver.setReceiveTimeout(DEFAULT_EVENTHUB_RECEIVER_TIMEOUT); + + // Start the receiver thread receiver.setReceiveHandler(handler); streamPartitionHandlers.put(ssp, handler); @@ -239,9 +241,9 @@ public void start() { @Override public Map> poll(Set systemStreamPartitions, long timeout) throws InterruptedException { - if (perminentEventHubError != null) { + if (permanentEventHubError != null) { String msg = String.format("Received a non transient error from event hub partition receiver (ssp=%s)", failedSSPReceiver); - throw new SamzaException(msg, perminentEventHubError); + throw new SamzaException(msg, permanentEventHubError); } return super.poll(systemStreamPartitions, timeout); } @@ -367,7 +369,7 @@ public void onError(Throwable throwable) { // Non transient Error propagated to user synchronized (receiverErrorLock) { - perminentEventHubError = throwable; + permanentEventHubError = throwable; failedSSPReceiver = ssp; } } From 811b90f8ab1a63fe0e17dace5e47335aec6b2a1a Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Wed, 25 Oct 2017 15:05:17 -0700 Subject: [PATCH 31/34] Log warning on poll when a non transient error occured --- .../consumer/EventHubSystemConsumer.java | 55 +++++++++++++------ 1 file changed, 37 insertions(+), 18 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index 66d101d181..6f4343fb37 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -51,6 +51,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; @@ -108,7 +109,6 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { public static final String AGGREGATE = "aggregate"; private static final Object AGGREGATE_METRICS_LOCK = new Object(); - private final Object receiverErrorLock = new Object(); private static Counter aggEventReadRate = null; private static Counter aggEventByteReadRate = null; @@ -130,9 +130,7 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { private final String systemName; // Partition receiver error propagation - private volatile Throwable permanentEventHubError = null; - private volatile SystemStreamPartition failedSSPReceiver = null; - + private final AtomicReference eventHubHandlerError = new AtomicReference<>(null); public EventHubSystemConsumer(EventHubConfig config, String systemName, EventHubClientManagerFactory eventHubClientManagerFactory, @@ -241,15 +239,28 @@ public void start() { @Override public Map> poll(Set systemStreamPartitions, long timeout) throws InterruptedException { - if (permanentEventHubError != null) { - String msg = String.format("Received a non transient error from event hub partition receiver (ssp=%s)", failedSSPReceiver); - throw new SamzaException(msg, permanentEventHubError); + Throwable handlerError = eventHubHandlerError.get(); + + if (handlerError != null) { + if (isErrorTransient(handlerError)) { + // Log a warning if the error is transient + // Partition receiver handler OnError should have handled it by recreating the receiver + LOG.warn("Received a transient error from event hub partition receiver, restarted receiver", handlerError); + } else { + // Propagate the error to user if the throwable is either + // 1. permanent ServiceBusException error from client + // 2. SamzaException thrown bu the EventHubConsumer + // 2a. Interrupted during put operation to BEM + // 2b. Failure in renewing the Partititon Receiver + String msg = "Received a non transient error from event hub partition receiver"; + throw new SamzaException(msg, handlerError); + } } + return super.poll(systemStreamPartitions, timeout); } - private void renewPartitionReceiver(SystemStreamPartition ssp, Throwable throwable) { - LOG.warn(String.format("Received a transient error from event hub partition receiver (ssp=%s), restarting receiver: ", ssp), throwable); + private void renewPartitionReceiver(SystemStreamPartition ssp) { EventHubClientManager eventHubClientManager = streamEventHubManagers.get(ssp.getStream()); String offset = streamPartitionOffsets.get(ssp); @@ -267,9 +278,9 @@ private void renewPartitionReceiver(SystemStreamPartition ssp, Throwable throwab streamPartitionReceivers.put(ssp, receiver); } catch (ServiceBusException e) { - throw new SamzaException( + eventHubHandlerError.set(new SamzaException( String.format("Failed to recreate receiver after ReceiverHandlerError for EventHubs for SystemStreamPartition=%s", - ssp), e); + ssp), e)); } } @@ -287,6 +298,14 @@ public void stop() { streamEventHubManagers.values().forEach(ehClientManager -> ehClientManager.close(DEFAULT_SHUTDOWN_TIMEOUT_MILLIS)); } + private boolean isErrorTransient(Throwable throwable) { + if (throwable instanceof ServiceBusException) { + ServiceBusException serviceBusException = (ServiceBusException) throwable; + return serviceBusException.getIsTransient(); + } + return false; + } + @Override protected BlockingQueue newBlockingQueue() { return new LinkedBlockingQueue<>(config.getConsumerBufferCapacity(systemName)); @@ -360,18 +379,18 @@ public void onError(Throwable throwable) { ServiceBusException busException = (ServiceBusException) throwable; if (busException.getIsTransient()) { - // Retry creating a receiver since error likely due to timeout - renewPartitionReceiver(ssp, throwable); + // Only set to transient throwable if there has been no previous errors + eventHubHandlerError.compareAndSet(null, throwable); + + // Retry creating a receiver since error likely due to timeout + renewPartitionReceiver(ssp); return; } } - // Non transient Error propagated to user - synchronized (receiverErrorLock) { - permanentEventHubError = throwable; - failedSSPReceiver = ssp; - } + // Propagate non transient or unknown errors + eventHubHandlerError.set(throwable); } } } From 9a4290a36c8650a310db88dd91d61b833b688a45 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Wed, 25 Oct 2017 17:27:03 -0700 Subject: [PATCH 32/34] Added interceptor, renamed EventHubIME --- .../eventhub/EventHubSystemFactory.java | 15 ++++---- .../samza/system/eventhub/Interceptor.java | 34 +++++++++++++++++++ .../admin/PassThroughInterceptor.java | 33 ++++++++++++++++++ ...a => EventHubIncomingMessageEnvelope.java} | 23 ++++--------- .../consumer/EventHubSystemConsumer.java | 26 +++++++------- .../producer/EventHubSystemProducer.java | 14 ++++---- .../consumer/ITestEventHubSystemConsumer.java | 4 +-- .../consumer/TestEventHubSystemConsumer.java | 29 ++++++++-------- .../producer/TestEventHubSystemProducer.java | 16 ++++----- 9 files changed, 125 insertions(+), 69 deletions(-) create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/Interceptor.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/PassThroughInterceptor.java rename samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/{EventHubIME.java => EventHubIncomingMessageEnvelope.java} (64%) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java index 5a8512d3a3..fd7a99ce29 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/EventHubSystemFactory.java @@ -21,13 +21,12 @@ import org.apache.samza.config.Config; import org.apache.samza.metrics.MetricsRegistry; -import org.apache.samza.serializers.ByteSerde; -import org.apache.samza.serializers.Serde; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemConsumer; import org.apache.samza.system.SystemFactory; import org.apache.samza.system.SystemProducer; import org.apache.samza.system.eventhub.admin.EventHubSystemAdmin; +import org.apache.samza.system.eventhub.admin.PassThroughInterceptor; import org.apache.samza.system.eventhub.consumer.EventHubSystemConsumer; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer; @@ -37,25 +36,25 @@ public class EventHubSystemFactory implements SystemFactory { - private Map> getSerdesMap(EventHubConfig config, String systemName) { - Map> serdes = new HashMap<>(); + private Map getInterceptorsMap(EventHubConfig config, String systemName) { + Map interceptors = new HashMap<>(); List streamList = config.getStreams(systemName); - streamList.forEach((streamName) -> serdes.put(streamName, new ByteSerde())); - return serdes; + streamList.forEach((streamName) -> interceptors.put(streamName, new PassThroughInterceptor())); + return interceptors; } @Override public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { EventHubConfig eventHubConfig = new EventHubConfig(config); return new EventHubSystemConsumer(eventHubConfig, systemName, new EventHubClientManagerFactory(), - getSerdesMap(eventHubConfig, systemName), registry); + getInterceptorsMap(eventHubConfig, systemName), registry); } @Override public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { EventHubConfig eventHubConfig = new EventHubConfig(config); return new EventHubSystemProducer(eventHubConfig, systemName, new EventHubClientManagerFactory(), - getSerdesMap(eventHubConfig, systemName), + getInterceptorsMap(eventHubConfig, systemName), registry); } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/Interceptor.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/Interceptor.java new file mode 100644 index 0000000000..2a6edfb54c --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/Interceptor.java @@ -0,0 +1,34 @@ +/* + * 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.samza.system.eventhub; + +/** + * Interface that is used for converting bytes to bytes. + */ +public interface Interceptor { + + /** + * Intercepts and converts bytes. + * + * @param bytes bytes to be converted + * @return Converted array of bytes converted + */ + byte[] intercept(byte[] bytes); +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/PassThroughInterceptor.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/PassThroughInterceptor.java new file mode 100644 index 0000000000..79dfb337e9 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/admin/PassThroughInterceptor.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.samza.system.eventhub.admin; + +import org.apache.samza.system.eventhub.Interceptor; + +/** + * An default {@link Interceptor} that is a pass-through. + */ +public class PassThroughInterceptor implements Interceptor { + + @Override + public byte[] intercept(byte[] bytes) { + return bytes; + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubIME.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubIncomingMessageEnvelope.java similarity index 64% rename from samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubIME.java rename to samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubIncomingMessageEnvelope.java index f1ca9b9427..8aa7480376 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubIME.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubIncomingMessageEnvelope.java @@ -20,32 +20,23 @@ package org.apache.samza.system.eventhub.consumer; import com.microsoft.azure.eventhubs.EventData; -import com.microsoft.azure.eventhubs.EventData.SystemProperties; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; -import java.util.Map; - /** * Extension of {@link IncomingMessageEnvelope} which contains {@link EventData} system and user properties metadata */ -public class EventHubIME extends IncomingMessageEnvelope { - private SystemProperties systemProperties; - private Map userProperties; - +public class EventHubIncomingMessageEnvelope extends IncomingMessageEnvelope { + private EventData eventData; - public EventHubIME(SystemStreamPartition systemStreamPartition, String offset, Object key, Object message, EventData eventData) { + public EventHubIncomingMessageEnvelope(SystemStreamPartition systemStreamPartition, String offset, Object key, + Object message, EventData eventData) { super(systemStreamPartition, offset, key, message); - this.systemProperties = eventData.getSystemProperties(); - this.userProperties = eventData.getProperties(); - } - - public SystemProperties getSystemProperties() { - return systemProperties; + this.eventData = eventData; } - public Map getUserProperties() { - return userProperties; + public EventData getEventData() { + return eventData; } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index 6f4343fb37..a437002c59 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -26,12 +26,12 @@ import org.apache.samza.SamzaException; import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.MetricsRegistry; -import org.apache.samza.serializers.Serde; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.eventhub.EventHubClientManagerFactory; import org.apache.samza.system.eventhub.EventHubClientManager; import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.Interceptor; import org.apache.samza.system.eventhub.admin.EventHubSystemAdmin; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; import org.apache.samza.util.BlockingEnvelopeMap; @@ -124,7 +124,7 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { private final ConcurrentHashMap streamPartitionReceivers = new ConcurrentHashMap<>(); private final ConcurrentHashMap streamEventHubManagers = new ConcurrentHashMap<>(); private final ConcurrentHashMap streamPartitionOffsets = new ConcurrentHashMap<>(); - private final Map> serdes; + private final Map interceptors; private boolean isStarted = false; private final EventHubConfig config; private final String systemName; @@ -134,12 +134,12 @@ public class EventHubSystemConsumer extends BlockingEnvelopeMap { public EventHubSystemConsumer(EventHubConfig config, String systemName, EventHubClientManagerFactory eventHubClientManagerFactory, - Map> serdes, MetricsRegistry registry) { + Map interceptors, MetricsRegistry registry) { super(registry, System::currentTimeMillis); this.config = config; this.systemName = systemName; - this.serdes = serdes; + this.interceptors = interceptors; List streamNames = config.getStreams(systemName); // Create and initiate connections to Event Hubs for (String streamName : streamNames) { @@ -216,7 +216,7 @@ public void start() { PartitionReceiveHandler handler = new PartitionReceiverHandlerImpl(ssp, eventReadRates.get(streamName), eventByteReadRates.get(streamName), readLatencies.get(streamName), readErrors.get(streamName), - serdes.getOrDefault(streamName, null)); + interceptors.getOrDefault(streamName, null)); // Timeout for EventHubClient receive @@ -317,27 +317,27 @@ protected class PartitionReceiverHandlerImpl extends PartitionReceiveHandler { private final Counter eventByteReadRate; private final SamzaHistogram readLatency; private final Counter errorRate; - private final Serde serde; + private final Interceptor interceptor; SystemStreamPartition ssp; PartitionReceiverHandlerImpl(SystemStreamPartition ssp, Counter eventReadRate, Counter eventByteReadRate, - SamzaHistogram readLatency, Counter readErrors, Serde serde) { + SamzaHistogram readLatency, Counter readErrors, Interceptor interceptor) { super(MAX_EVENT_COUNT_PER_PARTITION_POLL); this.ssp = ssp; this.eventReadRate = eventReadRate; this.eventByteReadRate = eventByteReadRate; this.readLatency = readLatency; - errorRate = readErrors; - this.serde = serde; + this.errorRate = readErrors; + this.interceptor = interceptor; } @Override public void onReceive(Iterable events) { if (events != null) { events.forEach(event -> { - byte[] decryptedBody = event.getBytes(); - if (serde != null) { - decryptedBody = serde.fromBytes(decryptedBody); + byte[] eventDataBody = event.getBytes(); + if (interceptor != null) { + eventDataBody = interceptor.intercept(eventDataBody); } String offset = event.getSystemProperties().getOffset(); Object partitionKey = event.getSystemProperties().getPartitionKey(); @@ -345,7 +345,7 @@ public void onReceive(Iterable events) { updateMetrics(event); // note that the partition key can be null - put(ssp, new EventHubIME(ssp, offset, partitionKey, decryptedBody, event)); + put(ssp, new EventHubIncomingMessageEnvelope(ssp, offset, partitionKey, eventDataBody, event)); } catch (InterruptedException e) { String msg = String.format("Interrupted while adding the event from ssp %s to dispatch queue.", ssp); LOG.error(msg, e); diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 68634d6f42..2e6141a352 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -26,12 +26,12 @@ import org.apache.samza.SamzaException; import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.MetricsRegistry; -import org.apache.samza.serializers.Serde; import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemProducer; import org.apache.samza.system.eventhub.EventHubClientManager; import org.apache.samza.system.eventhub.EventHubConfig; import org.apache.samza.system.eventhub.EventHubClientManagerFactory; +import org.apache.samza.system.eventhub.Interceptor; import org.apache.samza.system.eventhub.metrics.SamzaHistogram; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,19 +99,19 @@ public enum PartitioningMethod { private final Map eventHubClients = new HashMap<>(); private final Map> streamPartitionSenders = new HashMap<>(); - private final Map> serdes; + private final Map interceptors; private final Set> pendingFutures = ConcurrentHashMap.newKeySet(); public EventHubSystemProducer(EventHubConfig config, String systemName, EventHubClientManagerFactory eventHubClientManagerFactory, - Map> serdes, MetricsRegistry registry) { + Map interceptors, MetricsRegistry registry) { this.config = config; this.registry = registry; this.systemName = systemName; this.partitioningMethod = config.getPartitioningMethod(systemName); this.eventHubClientManagerFactory = eventHubClientManagerFactory; - this.serdes = serdes; + this.interceptors = interceptors; } @Override @@ -273,10 +273,10 @@ private String convertPartitionKeyToString(Object partitionKey) { } private EventData createEventData(String streamName, OutgoingMessageEnvelope envelope) { - Optional> serde = Optional.ofNullable(serdes.getOrDefault(streamName, null)); + Optional interceptor = Optional.ofNullable(interceptors.getOrDefault(streamName, null)); byte[] eventValue = (byte[]) envelope.getMessage(); - if (serde.isPresent()) { - eventValue = serde.get().toBytes(eventValue); + if (interceptor.isPresent()) { + eventValue = interceptor.get().intercept(eventValue); } EventData eventData = new EventData(eventValue); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java index 79ce0837dd..dbf8807f86 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/ITestEventHubSystemConsumer.java @@ -63,8 +63,8 @@ public void testSinglePartitionConsumptionHappyPath() throws Exception { List result = consumer.poll(Collections.singleton(ssp), 2000).get(ssp); numEvents = result == null ? 0 : result.size(); if (numEvents > 0) { - EventHubIME eventData = (EventHubIME) result.get(0); - System.out.println("System properties: " + eventData.getSystemProperties()); + EventHubIncomingMessageEnvelope eventData = (EventHubIncomingMessageEnvelope) result.get(0); + System.out.println("System properties: " + eventData.getEventData().getSystemProperties()); System.out.println("Message: " + new String((byte[]) eventData.getMessage())); break; } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java index 42aea3688c..b89c805904 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/consumer/TestEventHubSystemConsumer.java @@ -23,11 +23,10 @@ import com.microsoft.azure.eventhubs.*; import org.apache.samza.Partition; import org.apache.samza.metrics.Counter; -import org.apache.samza.serializers.ByteSerde; -import org.apache.samza.serializers.Serde; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.eventhub.*; +import org.apache.samza.system.eventhub.admin.PassThroughInterceptor; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -67,8 +66,8 @@ public void testMultipleRegistersToSameSSP() throws Exception { TestMetricsRegistry testMetrics = new TestMetricsRegistry(); Map> eventData = new HashMap<>(); SystemStreamPartition ssp = new SystemStreamPartition(systemName, streamName, new Partition(partitionId)); - Map> serdes = new HashMap<>(); - serdes.put(streamName, new ByteSerde()); + Map interceptors = new HashMap<>(); + interceptors.put(streamName, new PassThroughInterceptor()); // create EventData List singlePartitionEventData = MockEventData.generateEventData(numEvents); @@ -85,7 +84,7 @@ public void testMultipleRegistersToSameSSP() throws Exception { MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, serdes, + new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, interceptors, testMetrics); consumer.register(ssp, "1"); consumer.register(ssp, EventHubSystemConsumer.END_OF_STREAM); @@ -106,8 +105,8 @@ public void testSinglePartitionConsumptionHappyPath() throws Exception { TestMetricsRegistry testMetrics = new TestMetricsRegistry(); Map> eventData = new HashMap<>(); SystemStreamPartition ssp = new SystemStreamPartition(systemName, streamName, new Partition(partitionId)); - Map> serdes = new HashMap<>(); - serdes.put(streamName, new ByteSerde()); + Map interceptors = new HashMap<>(); + interceptors.put(streamName, new PassThroughInterceptor()); // create EventData List singlePartitionEventData = MockEventData.generateEventData(numEvents); @@ -124,7 +123,7 @@ public void testSinglePartitionConsumptionHappyPath() throws Exception { MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, serdes, + new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, interceptors, testMetrics); consumer.register(ssp, EventHubSystemConsumer.END_OF_STREAM); consumer.start(); @@ -155,8 +154,8 @@ public void testMultiPartitionConsumptionHappyPath() throws Exception { Map> eventData = new HashMap<>(); SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId1)); SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName, new Partition(partitionId2)); - Map> serdes = new HashMap<>(); - serdes.put(streamName, new ByteSerde()); + Map interceptor = new HashMap<>(); + interceptor.put(streamName, new PassThroughInterceptor()); // create EventData List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); @@ -175,7 +174,7 @@ public void testMultiPartitionConsumptionHappyPath() throws Exception { MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, serdes, + new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, interceptor, testMetrics); consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); consumer.register(ssp2, EventHubSystemConsumer.START_OF_STREAM); @@ -211,9 +210,9 @@ public void testMultiStreamsConsumptionHappyPath() throws Exception { Map> eventData = new HashMap<>(); SystemStreamPartition ssp1 = new SystemStreamPartition(systemName, streamName1, new Partition(partitionId)); SystemStreamPartition ssp2 = new SystemStreamPartition(systemName, streamName2, new Partition(partitionId)); - Map> serdes = new HashMap<>(); - serdes.put(streamName1, new ByteSerde()); - serdes.put(streamName2, new ByteSerde()); + Map interceptor = new HashMap<>(); + interceptor.put(streamName1, new PassThroughInterceptor()); + interceptor.put(streamName2, new PassThroughInterceptor()); List singlePartitionEventData1 = MockEventData.generateEventData(numEvents); List singlePartitionEventData2 = MockEventData.generateEventData(numEvents); @@ -235,7 +234,7 @@ public void testMultiStreamsConsumptionHappyPath() throws Exception { MockEventHubClientManagerFactory eventHubClientWrapperFactory = new MockEventHubClientManagerFactory(eventData); EventHubSystemConsumer consumer = - new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, serdes, + new EventHubSystemConsumer(new EventHubConfig(configMap), systemName, eventHubClientWrapperFactory, interceptor, testMetrics); consumer.register(ssp1, EventHubSystemConsumer.START_OF_STREAM); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java index b5afffeef7..bf62e92594 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/producer/TestEventHubSystemProducer.java @@ -20,12 +20,12 @@ package org.apache.samza.system.eventhub.producer; import com.microsoft.azure.eventhubs.*; -import org.apache.samza.serializers.ByteSerde; -import org.apache.samza.serializers.Serde; import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemStream; import org.apache.samza.system.eventhub.EventHubConfig; +import org.apache.samza.system.eventhub.Interceptor; import org.apache.samza.system.eventhub.MockEventHubClientManagerFactory; +import org.apache.samza.system.eventhub.admin.PassThroughInterceptor; import org.apache.samza.system.eventhub.producer.EventHubSystemProducer.PartitioningMethod; import org.apache.samza.system.eventhub.TestMetricsRegistry; import org.junit.Assert; @@ -61,8 +61,8 @@ public void testSendingToSpecificPartitions() throws Exception { int partitionId1 = 1; TestMetricsRegistry testMetrics = new TestMetricsRegistry(); - Map> serdes = new HashMap<>(); - serdes.put(streamName, new ByteSerde()); + Map interceptor = new HashMap<>(); + interceptor.put(streamName, new PassThroughInterceptor()); List outgoingMessagesP0 = generateMessages(numEvents); List outgoingMessagesP1 = generateMessages(numEvents); @@ -80,7 +80,7 @@ public void testSendingToSpecificPartitions() throws Exception { MockEventHubClientManagerFactory factory = new MockEventHubClientManagerFactory(); EventHubSystemProducer producer = - new EventHubSystemProducer(new EventHubConfig(configMap), systemName, factory, serdes, testMetrics); + new EventHubSystemProducer(new EventHubConfig(configMap), systemName, factory, interceptor, testMetrics); SystemStream systemStream = new SystemStream(systemName, streamName); producer.register(streamName); @@ -110,8 +110,8 @@ public void testSendingToEventHubHashing() throws Exception { String partitionId1 = "235"; TestMetricsRegistry testMetrics = new TestMetricsRegistry(); - Map> serdes = new HashMap<>(); - serdes.put(streamName, new ByteSerde()); + Map interceptor = new HashMap<>(); + interceptor.put(streamName, new PassThroughInterceptor()); List outgoingMessagesP0 = generateMessages(numEvents); List outgoingMessagesP1 = generateMessages(numEvents); @@ -130,7 +130,7 @@ public void testSendingToEventHubHashing() throws Exception { MockEventHubClientManagerFactory factory = new MockEventHubClientManagerFactory(); EventHubSystemProducer producer = - new EventHubSystemProducer(new EventHubConfig(configMap), systemName, factory, serdes, testMetrics); + new EventHubSystemProducer(new EventHubConfig(configMap), systemName, factory, interceptor, testMetrics); SystemStream systemStream = new SystemStream(systemName, streamName); producer.register(streamName); From 705f0f3db9872d5fc61a64fb8be914a64329a524 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Wed, 25 Oct 2017 18:14:32 -0700 Subject: [PATCH 33/34] Close PartitionReceiver before creating a new one --- .../system/eventhub/consumer/EventHubSystemConsumer.java | 9 ++++++--- .../system/eventhub/producer/EventHubSystemProducer.java | 1 - 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index a437002c59..bfb859a9be 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -268,6 +268,10 @@ private void renewPartitionReceiver(SystemStreamPartition ssp) { String consumerGroup = config.getStreamConsumerGroup(ssp.getSystem(), ssp.getStream()); try { + // Close current receiver + streamPartitionReceivers.get(ssp).close().get(DEFAULT_SHUTDOWN_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS); + + // Recreate receiver PartitionReceiver receiver = eventHubClientManager.getEventHubClient() .createReceiverSync(consumerGroup, partitionId.toString(), offset, !offset.equals(EventHubSystemConsumer.START_OF_STREAM)); @@ -277,10 +281,9 @@ private void renewPartitionReceiver(SystemStreamPartition ssp) { receiver.setReceiveHandler(streamPartitionHandlers.get(ssp)); streamPartitionReceivers.put(ssp, receiver); - } catch (ServiceBusException e) { + } catch (Exception e) { eventHubHandlerError.set(new SamzaException( - String.format("Failed to recreate receiver after ReceiverHandlerError for EventHubs for SystemStreamPartition=%s", - ssp), e)); + String.format("Failed to recreate receiver for EventHubs after ReceiverHandlerError (ssp=%s)", ssp), e)); } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java index 2e6141a352..c8c5538b0c 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/producer/EventHubSystemProducer.java @@ -58,7 +58,6 @@ public class EventHubSystemProducer implements SystemProducer { private static final long DEFAULT_FLUSH_TIMEOUT_MILLIS = Duration.ofMinutes(1L).toMillis(); public static final String PRODUCE_TIMESTAMP = "produce-timestamp"; - public static final String FLUSH_ALL = "all"; // Metrics recording private static final String EVENT_WRITE_RATE = "eventWriteRate"; From ea1bafeca4ec6c99a792c408de0b6b519cdc1858 Mon Sep 17 00:00:00 2001 From: Daniel Chen <29577458+dxichen@users.noreply.github.com> Date: Thu, 26 Oct 2017 09:51:34 -0700 Subject: [PATCH 34/34] Added extra comment for receicer start clarity --- .../samza/system/eventhub/consumer/EventHubSystemConsumer.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java index bfb859a9be..4de34de7d8 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/eventhub/consumer/EventHubSystemConsumer.java @@ -278,6 +278,8 @@ private void renewPartitionReceiver(SystemStreamPartition ssp) { // Timeout for EventHubClient receive receiver.setReceiveTimeout(DEFAULT_EVENTHUB_RECEIVER_TIMEOUT); + + // Create and start receiver thread with handler receiver.setReceiveHandler(streamPartitionHandlers.get(ssp)); streamPartitionReceivers.put(ssp, receiver);