[ 
https://issues.apache.org/jira/browse/SAMZA-2781?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Bharath Kumarasubramanian updated SAMZA-2781:
---------------------------------------------
    Description: 
{*}Summary{*}: Use framework thread pool to execute operator hand-offs and 
synchronous sub-DAG executions.

{*}Description{*}: Currently, the operator implementation chains the future 
using synchronous APIs (thenCompose, thenApply) which results in execution of 
these method calls on the future completion thread which happens to be the user 
thread in case of asynchronous operators in application DAG.

{*}Changes{*}:
 * Use thread pool inject through `job.container.task.executor.factory`
 * Extend the task executor factory to return operator executor 
 * Wire in the task executor in `TaskContext`
 * Default implementation which uses `#getTaskExecutor` if enabled and 
`job.container.thread.pool.size` > 1 or fallback to `ForkJoinPool` if disabled.
 * Unit tests for the factory
 * Test updates

{*}API Changes{*}:
 * Add `getOperatorExecutor` to `TaskExecutorFactory`
 * Provide a default implementation that reuses `getTaskExecutor`

  was:
{*}Summary{*}: Use framework thread pool to execute operator hand-offs and 
synchronous sub-DAG executions.

{*}Description{*}: Currently, the operator implementation chains the future 
using synchronous APIs (thenCompose, thenApply) which results in execution of 
these method calls on the future completion thread which happens to be the user 
thread in case of asynchronous operators in application DAG.

{*}Changes{*}:
 * Use thread pool inject through `job.container.task.executor.factory`
 * Extend the task executor factory to return operator executor 
 * Wire in the task executor in `TaskContext`
 * Default implementation which uses `#getTaskExecutor` if enabled and 
`job.container.thread.pool.size` > 1 or fallback to `ForkJoinPool` if disabled.
 * Unit tests for the factory
 * Test updates

{*}API Changes{*}:
 * Add `getOperatorExecutor` to `TaskExecutorFactory`
 * Provide a default implementation that reuses `getTaskExecutor`

{*}Usage Instructions{*}:

{*}Upgrade Instructions{*}: 
 * The operator hand off will use `ForkJoinPool` if no changes are made to 
application. Users may have to tune JVM `ForkJoinPool` to match their user 
thread pool size if they notice performance degradation.


> Use framework thread to execute hand-offs and sub-DAG execution
> ---------------------------------------------------------------
>
>                 Key: SAMZA-2781
>                 URL: https://issues.apache.org/jira/browse/SAMZA-2781
>             Project: Samza
>          Issue Type: Improvement
>            Reporter: Bharath Kumarasubramanian
>            Assignee: Bharath Kumarasubramanian
>            Priority: Major
>
> {*}Summary{*}: Use framework thread pool to execute operator hand-offs and 
> synchronous sub-DAG executions.
> {*}Description{*}: Currently, the operator implementation chains the future 
> using synchronous APIs (thenCompose, thenApply) which results in execution of 
> these method calls on the future completion thread which happens to be the 
> user thread in case of asynchronous operators in application DAG.
> {*}Changes{*}:
>  * Use thread pool inject through `job.container.task.executor.factory`
>  * Extend the task executor factory to return operator executor 
>  * Wire in the task executor in `TaskContext`
>  * Default implementation which uses `#getTaskExecutor` if enabled and 
> `job.container.thread.pool.size` > 1 or fallback to `ForkJoinPool` if 
> disabled.
>  * Unit tests for the factory
>  * Test updates
> {*}API Changes{*}:
>  * Add `getOperatorExecutor` to `TaskExecutorFactory`
>  * Provide a default implementation that reuses `getTaskExecutor`



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to