Return-Path: X-Original-To: apmail-streams-dev-archive@minotaur.apache.org Delivered-To: apmail-streams-dev-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id DFD92177F6 for ; Wed, 8 Oct 2014 19:27:06 +0000 (UTC) Received: (qmail 1546 invoked by uid 500); 8 Oct 2014 19:27:06 -0000 Delivered-To: apmail-streams-dev-archive@streams.apache.org Received: (qmail 1504 invoked by uid 500); 8 Oct 2014 19:27:06 -0000 Mailing-List: contact dev-help@streams.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@streams.incubator.apache.org Delivered-To: mailing list dev@streams.incubator.apache.org Received: (qmail 1490 invoked by uid 99); 8 Oct 2014 19:27:06 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 08 Oct 2014 19:27:06 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Wed, 08 Oct 2014 19:26:43 +0000 Received: (qmail 99470 invoked by uid 99); 8 Oct 2014 19:26:40 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 08 Oct 2014 19:26:40 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 0B64392C9AF; Wed, 8 Oct 2014 19:26:40 +0000 (UTC) From: rbnks To: dev@streams.incubator.apache.org Reply-To: dev@streams.incubator.apache.org References: In-Reply-To: Subject: [GitHub] incubator-streams pull request: Streams 190 Content-Type: text/plain Message-Id: <20141008192640.0B64392C9AF@tyr.zones.apache.org> Date: Wed, 8 Oct 2014 19:26:40 +0000 (UTC) X-Virus-Checked: Checked by ClamAV on apache.org Github user rbnks commented on a diff in the pull request: https://github.com/apache/incubator-streams/pull/99#discussion_r18606180 --- Diff: streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/queues/ThroughputQueue.java --- @@ -0,0 +1,382 @@ +package org.apache.streams.local.queues; + +import net.jcip.annotations.GuardedBy; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import sun.reflect.generics.reflectiveObjects.NotImplementedException; + +import javax.management.*; +import java.lang.management.ManagementFactory; +import java.util.Collection; +import java.util.Iterator; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * A {@link java.util.concurrent.BlockingQueue} implementation that allows the measure measurement of how + * data flows through the queue. Is also a {@code MBean} so the flow statistics can be viewed through + * JMX. Registration of the bean happens whenever a constructor receives a non-null id. + * + * !!! Warning !!! + * Only the necessary methods for the local streams runtime are implemented. All other methods throw a + * {@link sun.reflect.generics.reflectiveObjects.NotImplementedException}. + */ +public class ThroughputQueue implements BlockingQueue, ThroughputQueueMXBean { + + public static final String NAME_TEMPLATE = "org.apache.streams.local:type=ThroughputQueue,name=%s"; + + private static final Logger LOGGER = LoggerFactory.getLogger(ThroughputQueue.class); + + private BlockingQueue> underlyingQueue; + private ReadWriteLock putCountsLock; + private ReadWriteLock takeCountsLock; + @GuardedBy("putCountsLock") + private long elementsAdded; + @GuardedBy("takeCountsLock") + private long elementsRemoved; + @GuardedBy("this") + private long startTime; + @GuardedBy("takeCountsLock") + private long totalQueueTime; + @GuardedBy("takeCountsLock") + private long maxQueuedTime; + private volatile boolean active; + + /** + * Creates an unbounded, unregistered {@code ThroughputQueue} + */ + public ThroughputQueue() { + this(-1, null); + } + + /** + * Creates a bounded, unregistered {@code ThroughputQueue} + * @param maxSize maximum capacity of queue, if maxSize < 1 then unbounded + */ + public ThroughputQueue(int maxSize) { + this(maxSize, null); + } + + /** + * Creates an unbounded, registered {@code ThroughputQueue} + * @param id unique id for this queue to be registered with. if id == NULL then not registered + */ + public ThroughputQueue(String id) { + this(-1, id); + } + + /** + * Creates a bounded, registered {@code ThroughputQueue} + * @param maxSize maximum capacity of queue, if maxSize < 1 then unbounded + * @param id unique id for this queue to be registered with. if id == NULL then not registered + */ + public ThroughputQueue(int maxSize, String id) { + if(maxSize < 1) { + this.underlyingQueue = new LinkedBlockingQueue<>(); + } else { + this.underlyingQueue = new LinkedBlockingQueue<>(maxSize); + } + this.elementsAdded = 0; + this.elementsRemoved = 0; + this.startTime = -1; + this.putCountsLock = new ReentrantReadWriteLock(); + this.takeCountsLock = new ReentrantReadWriteLock(); + this.active = false; + this.maxQueuedTime = -1; + if(id != null) { + try { + ObjectName name = new ObjectName(String.format(NAME_TEMPLATE, id)); + MBeanServer mbs = ManagementFactory.getPlatformMBeanServer(); + mbs.registerMBean(this, name); + } catch (MalformedObjectNameException|InstanceAlreadyExistsException|MBeanRegistrationException|NotCompliantMBeanException e) { + LOGGER.error("Failed to register MXBean : {}", e); + throw new RuntimeException(e); + } + } + } + + @Override + public boolean add(E e) { + throw new NotImplementedException(); + } + + @Override + public boolean offer(E e) { + throw new NotImplementedException(); + } + + @Override + public void put(E e) throws InterruptedException { + this.underlyingQueue.put(new ThroughputElement(e)); + try { + this.putCountsLock.writeLock().lockInterruptibly(); + ++this.elementsAdded; + } finally { + this.putCountsLock.writeLock().unlock(); + } + synchronized (this) { + if (!this.active) { + this.startTime = System.currentTimeMillis(); + this.active = true; + } + } + + } + + @Override + public boolean offer(E e, long timeout, TimeUnit unit) throws InterruptedException { + if(this.underlyingQueue.offer(new ThroughputElement(e), timeout, unit)) { + try { + this.putCountsLock.writeLock().lockInterruptibly(); + ++this.elementsAdded; + } finally { + this.putCountsLock.writeLock().unlock(); + } + synchronized (this) { + if (!this.active) { + this.startTime = System.currentTimeMillis(); + this.active = true; + } + } + return true; + } + return false; + } + + @Override + public E take() throws InterruptedException { + ThroughputElement e = this.underlyingQueue.take(); + try { + this.takeCountsLock.writeLock().lockInterruptibly(); + ++this.elementsRemoved; + Long queueTime = e.getWaited(); + this.totalQueueTime += queueTime; + if(this.maxQueuedTime < queueTime) { + this.maxQueuedTime = queueTime; + } + } finally { + this.takeCountsLock.writeLock().unlock(); + } + return e.getElement(); + } + + @Override + public E poll(long timeout, TimeUnit unit) throws InterruptedException { + throw new NotImplementedException(); + } + + @Override + public int remainingCapacity() { + throw new NotImplementedException(); + } + + @Override + public boolean remove(Object o) { + throw new NotImplementedException(); + } + + @Override + public boolean contains(Object o) { + throw new NotImplementedException(); + } + + @Override + public int drainTo(Collection c) { + throw new NotImplementedException(); + } + + @Override + public int drainTo(Collection c, int maxElements) { + throw new NotImplementedException(); + } + + @Override + public E remove() { + throw new NotImplementedException(); + } + + @Override + public E poll() { + throw new NotImplementedException(); + } + + @Override + public E element() { + throw new NotImplementedException(); + } + + @Override + public E peek() { + throw new NotImplementedException(); + } + + @Override + public int size() { + return this.underlyingQueue.size(); + } + + @Override + public boolean isEmpty() { + return this.underlyingQueue.isEmpty(); + } + + @Override + public Iterator iterator() { + throw new NotImplementedException(); + } + + @Override + public Object[] toArray() { + throw new NotImplementedException(); + } + + @Override + public T[] toArray(T[] a) { + throw new NotImplementedException(); + } + + @Override + public boolean containsAll(Collection c) { + throw new NotImplementedException(); + } + + @Override + public boolean addAll(Collection c) { + throw new NotImplementedException(); + } + + @Override + public boolean removeAll(Collection c) { + throw new NotImplementedException(); + } + + @Override + public boolean retainAll(Collection c) { + throw new NotImplementedException(); + } + + @Override + public void clear() { + throw new NotImplementedException(); + } + + @Override + public long getCurrentSize() { + long size = -1; + try { + this.putCountsLock.readLock().lock(); + try { + this.takeCountsLock.readLock().lock(); + size = this.elementsAdded - this.elementsRemoved; + } finally { --- End diff -- Yes, an exception would be thrown. Try/Finally blocks just says that if an exception is thrown in that block, before throwing the exception run the code that is in the finally block first. The finally block attempts to release the lock it may have obtained in the try block. This is to avoid having a situation where this thread obtains a lock, gets an exception, and dies while still holding a lock. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastructure@apache.org or file a JIRA ticket with INFRA. ---