-
Notifications
You must be signed in to change notification settings - Fork 9.1k
HADOOP-17998. Allow get command to run with multi threads. #3645
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
💔 -1 overall
This message was automatically generated. |
This seems like a good change. I will try to review in more detail in the next few days. Could you add the new -t parameter to the docs in |
@sodonnel thanks for your comment, I have added -t and -q parameters to the docs, and fixed some mistakes in docs meanwhile. |
💔 -1 overall
This message was automatically generated. |
assertEquals(0, cmd.run(args)); | ||
} | ||
|
||
@org.junit.Test(timeout = 10000) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: There is a mixture of @org.junit.Test
and @Test
annotations in this class - can you change them all to just @Test
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed
public void testCopy() throws Exception { | ||
Path dir = new Path("dir" + RandomStringUtils.randomNumeric(4)); | ||
initialize(dir); | ||
MultiThreadedCopy copy = new MultiThreadedCopy(1, DEFAULT_QUEUE_SIZE, 0); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Every test starts with these two lines:
Path dir = new Path("dir" + RandomStringUtils.randomNumeric(4));
initialize(dir);
Do you think it would be better to create a @Before
method to run before each test?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed by created @Before
method
|
||
public static final int DEFAULT_QUEUE_SIZE = 1024; | ||
public static final int MAX_THREAD_COUNT = | ||
Runtime.getRuntime().availableProcessors() * 2; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder if we should limit the number of threads like this. Its hard to say if the copy will be CPU bound, or disk / IO bound overall. I guess it is best to avoid having no limit, but I wonder if having 2 * cores would be enough for a small VM trying to put a large dir into the cluster. What do you think? Maybe setting the limit to 4 or 8 * cores would be more flexible for users and they can experiment with their own hardware to find the best setting?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I guess it is best to avoid having no limit
I agree with you, no limit and decided by users is the best way. Set the limit just for keep same with the original. I will cancel it.
protected void setThreadCount(String optValue) { | ||
if (optValue != null) { | ||
int count = Integer.parseInt(optValue); | ||
threadCount = count < 1 ? 1 : Math.min(count, MAX_THREAD_COUNT); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we warn here if the thread count is being reduced due to the MAX_THREAD_COUNT in a similar way to setThreadPoolQueueSize
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed By add docs to these methods.
Thanks for working on this @smarthanwang. The change looks mostly good to me. I have just a few minor comments. There is also a failing test in |
@sodonnel thanks for your detailed review, I have removed the limit of thread count, and fixed the problems in unit tests. Please help review again. |
The failure was caused by the modification of command's usage and description, fixed now. |
💔 -1 overall
This message was automatically generated. |
There are 3 errors coming up in the Java Docs build job related to the changes here:
I guess it does not like the Could you try to fix those by changing the Java Doc comments and then I think this change is good to commit. |
@sodonnel thanks for review, fixed the java docs errors. |
🎊 +1 overall
This message was automatically generated. |
(cherry picked from commit 63018dc) Conflicts: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
Description
JIRA https://issues.apache.org/jira/browse/HADOOP-17998
CopyFromLocal/Put is enabled to run with multi-thread with HDFS-11786 and HADOOP-14698, and make put dirs or multiple files faster.So, It's necessary to enable get and copyToLocal command run with multi-thread.
Tests
Test case: 1 dir 240 files 13G
1. Get with single thread.
2. Get with 10 threads.