hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Chris Nauroth (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HADOOP-13403) AzureNativeFileSystem rename/delete performance improvements
Date Fri, 29 Jul 2016 17:16:20 GMT

    [ https://issues.apache.org/jira/browse/HADOOP-13403?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15399707#comment-15399707
] 

Chris Nauroth commented on HADOOP-13403:
----------------------------------------

[~pattipaka], in addition to the earlier code review feedback, I have a higher-level question
about the implementation of the {{executeParallel}} method.

Typical usage of a {{ThreadPoolExecutor}} is to configure it with a certain thread count and
then submit multiple tasks, with each task performing a single isolated action.  When tasks
are submitted, they get added to an internal queue (the {{LinkedBlockingQueue}} in your code).
 Then, the number of threads all independently pull and execute tasks from the queue.

The {{executeParallel}} method in your current patch does not follow this typical usage of
{{ThreadPoolExecutor}}.  Instead, it submits a number of tasks identical to the thread count,
and the code in each task iterates through an array of work items ({{FileMetadata[]}}) tracked
manually.  The array is shared state across all of those tasks/threads, so safe access requires
additional synchronization code.

It appears the current {{executeParallel}} reimplements functionality already built into the
{{ThreadPoolExecutor}}.  Switching this code to more idiomatic usage of {{ThreadPoolExecutor}}
would make it easier for maintainers to understand, and it would remove the need for extra
cross-thread synchronization, which is always tricky to get right.  For example, assuming
10 threads configured for {{fs.azure.delete.threads}} and a deletion of 100 blobs, I recommend
using a {{ThreadPoolExecutor}} configured with 10 threads and submitting 100 task instances
to it, each task deleting a single blob, with no shared state across the tasks/threads.

I also see the current logic carefully tracks the last exception that happened on a thread.
 Maybe that's why you implemented {{executeParallel}} this way?  However, there is also a
more idiomatic way to look for exceptions: call {{ThreadPoolExecutor#submit}} for all tasks,
track every returned {{Future}} in a list, and then iterate through that list calling {{Future#get}}
on each one.  If any task encountered an exception, it would propagate as an {{ExecutionException}},
and then you can unwrap it to get the underlying exception that was thrown on the thread.

I also see logic for handling {{RejectedExecutionException}}.  In practice, I expect this
won't happen, because the {{LinkedBlockingQueue}} will cause the caller submitting the task
to block instead of failing if it runs out of capacity.  If you're concerned about this, then
another way to approach this is to look into [{{ThreadPoolExecutor.CallerRunsPolicy}}|http://docs.oracle.com/javase/7/docs/api/java/util/concurrent/ThreadPoolExecutor.CallerRunsPolicy.html],
which would fall back to running the task in the main submitting thread.

If there is some other reason I'm missing that requires the current unusual implementation
of {{executeParallel}}, please let me know.  Thank you.

> AzureNativeFileSystem rename/delete performance improvements
> ------------------------------------------------------------
>
>                 Key: HADOOP-13403
>                 URL: https://issues.apache.org/jira/browse/HADOOP-13403
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: azure
>    Affects Versions: 2.7.2
>            Reporter: Subramanyam Pattipaka
>            Assignee: Subramanyam Pattipaka
>             Fix For: 2.9.0
>
>         Attachments: HADOOP-13403-001.patch, HADOOP-13403-002.patch
>
>
> WASB Performance Improvements
> Problem
> -----------
> Azure Native File system operations like rename/delete which has large number of directories
and/or files in the source directory are experiencing performance issues. Here are possible
reasons
> a)	We first list all files under source directory hierarchically. This is a serial operation.

> b)	After collecting the entire list of files under a folder, we delete or rename files
one by one serially.
> c)	There is no logging information available for these costly operations even in DEBUG
mode leading to difficulty in understanding wasb performance issues.
> Proposal
> -------------
> Step 1: Rename and delete operations will generate a list all files under the source
folder. We need to use azure flat listing option to get list with single request to azure
store. We have introduced config fs.azure.flatlist.enable to enable this option. The default
value is 'false' which means flat listing is disabled.
> Step 2: Create thread pool and threads dynamically based on user configuration. These
thread pools will be deleted after operation is over.  We are introducing introducing two
new configs
> 	a)	fs.azure.rename.threads : Config to set number of rename threads. Default value is
0 which means no threading.
> 	b)	fs.azure.delete.threads: Config to set number of delete threads. Default value is
0 which means no threading.
> 	We have provided debug log information on number of threads not used for the operation
which can be useful .
> 	Failure Scenarios:
> 	If we fail to create thread pool due to ANY reason (for example trying create with thread
count with large value such as 1000000), we fall back to serialization operation. 
> Step 3: Bob operations can be done in parallel using multiple threads executing following
snippet
> 	while ((currentIndex = fileIndex.getAndIncrement()) < files.length) {
> 		FileMetadata file = files[currentIndex];
> 		Rename/delete(file);
> 	}
> 	The above strategy depends on the fact that all files are stored in a final array and
each thread has to determine synchronized next index to do the job. The advantage of this
strategy is that even if user configures large number of unusable threads, we always ensure
that work doesn’t get serialized due to lagging threads. 
> 	We are logging following information which can be useful for tuning number of threads
> 	a) Number of unusable threads
> 	b) Time taken by each thread
> 	c) Number of files processed by each thread
> 	d) Total time taken for the operation
> 	Failure Scenarios:
> 	Failure to queue a thread execute request shouldn’t be an issue if we can ensure at
least one thread has completed execution successfully. If we couldn't schedule one thread
then we should take serialization path. Exceptions raised while executing threads are still
considered regular exceptions and returned to client as operation failed. Exceptions raised
while stopping threads and deleting thread pool shouldn't can be ignored if operation all
files are done with out any issue.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


Mime
View raw message