[
https://issues.apache.org/jira/browse/APEXMALHAR-1963?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15121292#comment-15121292
]
ASF GitHub Bot commented on APEXMALHAR-1963:
--------------------------------------------
Github user chinmaykolhatkar commented on a diff in the pull request:
https://github.com/apache/incubator-apex-malhar/pull/157#discussion_r51114181
--- Diff:
library/src/main/java/com/datatorrent/lib/async/AbstractAsyncProcessor.java ---
@@ -0,0 +1,341 @@
+/**
+ * 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 com.datatorrent.lib.async;
+
+import java.util.Iterator;
+import java.util.Queue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import javax.validation.constraints.Max;
+import javax.validation.constraints.Min;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+import com.datatorrent.api.Context;
+import com.datatorrent.api.Operator.IdleTimeHandler;
+import com.datatorrent.common.util.BaseOperator;
+
+
+/**
+ * This base operator can be used to enqueue tuples for asynchronous and
parallel processing by separate worker threads.
+ *
+ * User can configure number of threads required for async processing. By
default 32 threads will be created.
+ * These worker threads, spawned by operator thread, can also be
configured to processing timeout.
+ *
+ * <b>Workflow is as follows:</b>
+ * <ol>
+ * <li>Implementing class can enqueue the tuples of type QUEUETUPLE
using <i>enqueueTupleForProcessing</i>
+ * method.</li>
+ * <li>When a tuple gets its turn for execution,
<i>processTupleAsync</i> gets called
+ * by worker thread. This is asynchronous call</li>
+ * <li>Based on maintainTupleOrder parameter, the results are
consolidated at the end of every window and respective
+ * call to <i>handleProcessedTuple</i> will be made. This callback
is in operator thread.</li>
+ * </ol>
+ *
+ *
+ * <b>Use case examples:</b>
+ * <ul>
+ * <li>Parallel reads from external datastore/database system.</li>
+ * <li>Operations which are long running tasks per tuple but DAG i/o
should not be blocked.</li>
+ * </ul>
+ *
+ * @param <QUEUETUPLE> input type of tuple
+ * @param <RESULTTUPLE> Result of async processing of tuple.
+ */
+public abstract class AbstractAsyncProcessor<QUEUETUPLE, RESULTTUPLE>
extends BaseOperator implements IdleTimeHandler
+{
+ private static final Logger logger =
LoggerFactory.getLogger(AbstractAsyncProcessor.class);
+
+ /**
+ * This is an executor service which holds thread pool and enqueues task
for processing.
+ */
+ private transient ExecutorService executor;
+
+ /**
+ * Number of worker threads for processing
+ * Minimum value is 1 and max allowed number of threads are 32.
+ *
+ * Default number of threads is 32.
+ */
+ @Min(1)
+ @Max(32)
+ private int numProcessors = 32;
+
+ /**
+ * Tells whether the <i>handleProcessedTuple</i> should be called for
tuples in the same order in which
+ * they get enqueued using <i>enqueueTupleForProcessing</i>.
+ */
+ private boolean maintainTupleOrder = false;
+
+ /**
+ * Timeout in milliseconds after which processing of tuple may be
terminated.
+ */
+ private long processTimeoutMs = 0;
+
+ /**
+ * The queue holds following types of tuples:
+ * <ul>
+ * <li>Yet to be processed</li>
+ * <li>In Process</li>
+ * <li>Completed processing but yet to be notified for process
completion</li>
+ * </ul>
+ */
+ private Queue<WorkItem> waitingTuples = Lists.newLinkedList();
+
+ /**
+ * Setup method of this operator will initialize various types of
executors possible.
+ *
+ * @param context OperatorContext
+ */
+ @Override public void setup(Context.OperatorContext context)
+ {
+ this.executor = Executors.newFixedThreadPool(numProcessors);
+ replayTuplesIfRequired();
+ }
+
+ /**
+ * This method is called from setup of the operator. If
<i>waitingTuples</i> contains anything, the method enqueues
+ * all those tuples for processing to worker threads.
+ *
+ * Dependening on condition the method will behave as follows:
+ * <ul>
+ * <li>During fresh start of operator, this method will not have any
effect as there are no tuples in
+ * <i>waitingTuples</i>.</li>
+ * <li>During restore of operator to previous checkpointed state, if
there are any tuples already present
+ * in <i>waitingTuples</i> that means they're yet to be notified,
hence enqueued again.</li>
+ * </ul>
+ */
+ private void replayTuplesIfRequired()
+ {
+ // Enqueue everything that is still left.
+ // Its left because its either not completed processing OR its not
emitted because of ordering restrictions.
+ for (WorkItem item : waitingTuples) {
+ item.taskHandle = executor.submit(new Processor(item));
+ item.processState = State.SUBMITTED;
+ item.processStartTime = 0;
+ item.outTuple = null;
+ }
+ }
+
+ /**
+ * Teardown of operator which stops executor service.
+ */
+ @Override public void teardown()
+ {
+ this.executor.shutdownNow();
+ }
+
+ /**
+ * Enqueues tuple of type QUEUETUPLE for processing to worker threads.
+ * This method should be called only from operator thread.
+ *
+ * @param QUEUETUPLE Input tuple of type QUEUETUPLE
+ */
+ protected void enqueueTupleForProcessing(QUEUETUPLE QUEUETUPLE)
+ {
+ WorkItem item = new WorkItem(QUEUETUPLE);
+ waitingTuples.add(item);
+ item.taskHandle = executor.submit(new Processor(item));
+ item.processState = State.SUBMITTED;
--- End diff --
calling executor.submit after variables are set. This should take care of
overrides.
> Add abstract operator for Async Processing
> ------------------------------------------
>
> Key: APEXMALHAR-1963
> URL: https://issues.apache.org/jira/browse/APEXMALHAR-1963
> Project: Apache Apex Malhar
> Issue Type: New Feature
> Affects Versions: 3.3.0
> Reporter: Chinmay Kolhatkar
> Assignee: Chinmay Kolhatkar
>
> Create an abstract operator which does following:
> 1) Asynchronously processes the tuples
> 2) Have parallel executions
> 3) Make sure the outbound tuples are ordered similar to inbound tuples if
> configured.
> 4) Have processing timeout.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)