apex-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Ganelin, Ilya" <Ilya.Gane...@capitalone.com>
Subject Re: Why does emit require current thread to be the operator thread?
Date Tue, 11 Apr 2017 03:00:25 GMT
Neat – idle would be when not in process, begin window, or end window? Is that its own event
loop or is it a periodic callback?

- Ilya Ganelin


On 4/10/17, 1:58 PM, "Pramod Immaneni" <pramod@datatorrent.com> wrote:

    You can also emit when the system is idling by implementing the
    IdleTimeHandler interface in your operator.
    
    On Mon, Apr 10, 2017 at 1:06 PM, Amol Kekre <amol@datatorrent.com> wrote:
    
    > Not yet, But we could leverage internal structures of Apex as they do same
    > thing. For example in container local streams. There is a catch though -
    > the queue read by main thread will only happen when another data tuple
    > arrives in process call, or control tuple arrives for start or end window.
    >
    > Thks
    > Amol
    >
    >
    >
    > E:amol@datatorrent.com | M: 510-449-2606 | Twitter: @*amolhkekre*
    >
    > www.datatorrent.com
    >
    >
    > On Mon, Apr 10, 2017 at 1:01 PM, Ganelin, Ilya <
    > Ilya.Ganelin@capitalone.com>
    > wrote:
    >
    > > Thanks, Amol – that makes sense and was the solution I’d arrived at. I
    > > just was trying to avoid the delay between the data being ready and
    > > emitting it. Has anyone built a solution where it emits from the parent
    > as
    > > soon as it’s ready in the child (assuming I don’t care about order).
    > >
    > > - Ilya Ganelin
    > >
    > >
    > > On 4/10/17, 12:45 PM, "Amol Kekre" <amol@datatorrent.com> wrote:
    > >
    > >     Ilya,
    > >     This constraint was introduced as allowing two threads to emit data
    > > creates
    > >     lots of bad situations
    > >     1. The emit is triggered between end_window and begin_window. This
    > was
    > > a
    > >     critical blocker
    > >     2. Order no longer guaranteed, upon replay getting wrong order of
    > > events
    > >     within a window. This was something to worry about, but not a blocker
    > >
    > >     We had users report this problem.
    > >
    > >     The solution is to pass the data to main thread and have the main
    > > thread
    > >     emit this data during one of start-window, process, end-window calls.
    > >     Ideally during start-window or end-window so as to guarantee order.
    > > Keeping
    > >     this code in start or end window also ensures that process call
    > remains
    > >     optimal.
    > >
    > >     Thks
    > >     Amol
    > >
    > >
    > >
    > >     E:amol@datatorrent.com | M: 510-449-2606 | Twitter: @*amolhkekre*
    > >
    > >     www.datatorrent.com
    > >
    > >
    > >     On Mon, Apr 10, 2017 at 12:39 PM, Ganelin, Ilya <
    > > Ilya.Ganelin@capitalone.com
    > >     > wrote:
    > >
    > >     > Hello – I’ve got an operator that runs a cleanup thread (separate
    > > from the
    > >     > main event loop) and triggers a callback when an item is removed
    > > from an
    > >     > internal data structure. I would like for this callback to emit
    > data
    > > from
    > >     > one of the operator’s ports, but I run into the following
    > Exception:
    > >     >
    > >     >
    > >     >
    > >     > (From DefaultOutputPort.java, line 58)
    > >     >
    > >     > if (operatorThread != null && Thread.*currentThread*() !=
    > > operatorThread)
    > >     > {
    > >     >   // only under certain modes: enforce this
    > >     >   throw new IllegalStateException("Current thread " + Thread.
    > >     > *currentThread*().getName() +
    > >     >       " is different from the operator thread " +
    > >     > operatorThread.getName());
    > >     > }
    > >     >
    > >     >
    > >     >
    > >     > I could obviously extend DefaultOperatorPort to bypass this but I’d
    > > like
    > >     > to understand why that constraint is there and if there’s a good
    > way
    > > to
    > >     > work around it.
    > >     >
    > >     >
    > >     >
    > >     > Would love to hear the community’s thoughts. Thanks!
    > >     >
    > >     >
    > >     >
    > >     > - Ilya Ganelin
    > >     >
    > >     > [image: id:image001.png@01D1F7A4.F3D42980]
    > >     >
    > >     > ------------------------------
    > >     >
    > >     > The information contained in this e-mail is confidential and/or
    > >     > proprietary to Capital One and/or its affiliates and may only be
    > used
    > >     > solely in performance of work or services for Capital One. The
    > > information
    > >     > transmitted herewith is intended only for use by the individual or
    > > entity
    > >     > to which it is addressed. If the reader of this message is not the
    > > intended
    > >     > recipient, you are hereby notified that any review, retransmission,
    > >     > dissemination, distribution, copying or other use of, or taking of
    > > any
    > >     > action in reliance upon this information is strictly prohibited. If
    > > you
    > >     > have received this communication in error, please contact the
    > sender
    > > and
    > >     > delete the material from your computer.
    > >     >
    > >
    > >
    > > ________________________________________________________
    > >
    > > The information contained in this e-mail is confidential and/or
    > > proprietary to Capital One and/or its affiliates and may only be used
    > > solely in performance of work or services for Capital One. The
    > information
    > > transmitted herewith is intended only for use by the individual or entity
    > > to which it is addressed. If the reader of this message is not the
    > intended
    > > recipient, you are hereby notified that any review, retransmission,
    > > dissemination, distribution, copying or other use of, or taking of any
    > > action in reliance upon this information is strictly prohibited. If you
    > > have received this communication in error, please contact the sender and
    > > delete the material from your computer.
    > >
    >
    

________________________________________________________

The information contained in this e-mail is confidential and/or proprietary to Capital One
and/or its affiliates and may only be used solely in performance of work or services for Capital
One. The information transmitted herewith is intended only for use by the individual or entity
to which it is addressed. If the reader of this message is not the intended recipient, you
are hereby notified that any review, retransmission, dissemination, distribution, copying
or other use of, or taking of any action in reliance upon this information is strictly prohibited.
If you have received this communication in error, please contact the sender and delete the
material from your computer.
Mime
View raw message