[jira] [Commented] (HADOOP-16461) Regression: FileSystem cache lock parses XML within the lock
[ https://issues.apache.org/jira/browse/HADOOP-16461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16892278#comment-16892278 ] Gopal V commented on HADOOP-16461: -- Linked the lines up and opened a PR. > Regression: FileSystem cache lock parses XML within the lock > > > Key: HADOOP-16461 > URL: https://issues.apache.org/jira/browse/HADOOP-16461 > Project: Hadoop Common > Issue Type: Bug > Components: filecache >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > > https://github.com/apache/hadoop/blob/2546e6ece240924af2188bb39b3954a4896e4a4f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java#L3388 > {code} > fs = createFileSystem(uri, conf); > synchronized (this) { // refetch the lock again > FileSystem oldfs = map.get(key); > if (oldfs != null) { // a file system is created while lock is > releasing > fs.close(); // close the new file system > return oldfs; // return the old file system > } > // now insert the new file system into the map > if (map.isEmpty() > && !ShutdownHookManager.get().isShutdownInProgress()) { > ShutdownHookManager.get().addShutdownHook(clientFinalizer, > SHUTDOWN_HOOK_PRIORITY); > } > fs.key = key; > map.put(key, fs); > if (conf.getBoolean( > FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) { > toAutoClose.add(key); > } > return fs; > } > {code} > The lock now has a ShutdownHook creation, which ends up doing > https://github.com/apache/hadoop/blob/2546e6ece240924af2188bb39b3954a4896e4a4f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java#L205 > {code} > HookEntry(Runnable hook, int priority) { > this(hook, priority, > getShutdownTimeout(new Configuration()), > TIME_UNIT_DEFAULT); > } > {code} > which ends up doing a "new Configuration()" within the locked section. > This indirectly hurts the cache hit scenarios as well, since if the lock on > this is held, then the other section cannot be entered either. > https://github.com/apache/tez/blob/master/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java#L65 > {code} > I/O Setup 0 State: BLOCKED CPU usage on sample: 6ms > org.apache.hadoop.fs.FileSystem$Cache.getInternal(URI, Configuration, > FileSystem$Cache$Key) FileSystem.java:3345 > org.apache.hadoop.fs.FileSystem$Cache.get(URI, Configuration) > FileSystem.java:3320 > org.apache.hadoop.fs.FileSystem.get(URI, Configuration) FileSystem.java:479 > org.apache.hadoop.fs.FileSystem.getLocal(Configuration) FileSystem.java:435 > {code} > slowing down the RawLocalFileSystem when there are other threads creating > HDFS FileSystem objects at the same time. -- This message was sent by Atlassian JIRA (v7.6.14#76016) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Updated] (HADOOP-16461) Regression: FileSystem cache lock parses XML within the lock
[ https://issues.apache.org/jira/browse/HADOOP-16461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-16461: - Description: https://github.com/apache/hadoop/blob/2546e6ece240924af2188bb39b3954a4896e4a4f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java#L3388 {code} fs = createFileSystem(uri, conf); synchronized (this) { // refetch the lock again FileSystem oldfs = map.get(key); if (oldfs != null) { // a file system is created while lock is releasing fs.close(); // close the new file system return oldfs; // return the old file system } // now insert the new file system into the map if (map.isEmpty() && !ShutdownHookManager.get().isShutdownInProgress()) { ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY); } fs.key = key; map.put(key, fs); if (conf.getBoolean( FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) { toAutoClose.add(key); } return fs; } {code} The lock now has a ShutdownHook creation, which ends up doing https://github.com/apache/hadoop/blob/2546e6ece240924af2188bb39b3954a4896e4a4f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java#L205 {code} HookEntry(Runnable hook, int priority) { this(hook, priority, getShutdownTimeout(new Configuration()), TIME_UNIT_DEFAULT); } {code} which ends up doing a "new Configuration()" within the locked section. This indirectly hurts the cache hit scenarios as well, since if the lock on this is held, then the other section cannot be entered either. https://github.com/apache/tez/blob/master/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java#L65 {code} I/O Setup 0 State: BLOCKED CPU usage on sample: 6ms org.apache.hadoop.fs.FileSystem$Cache.getInternal(URI, Configuration, FileSystem$Cache$Key) FileSystem.java:3345 org.apache.hadoop.fs.FileSystem$Cache.get(URI, Configuration) FileSystem.java:3320 org.apache.hadoop.fs.FileSystem.get(URI, Configuration) FileSystem.java:479 org.apache.hadoop.fs.FileSystem.getLocal(Configuration) FileSystem.java:435 {code} slowing down the RawLocalFileSystem when there are other threads creating HDFS FileSystem objects at the same time. was: {code} fs = createFileSystem(uri, conf); synchronized (this) { // refetch the lock again FileSystem oldfs = map.get(key); if (oldfs != null) { // a file system is created while lock is releasing fs.close(); // close the new file system return oldfs; // return the old file system } // now insert the new file system into the map if (map.isEmpty() && !ShutdownHookManager.get().isShutdownInProgress()) { ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY); } fs.key = key; map.put(key, fs); if (conf.getBoolean( FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) { toAutoClose.add(key); } return fs; } {code} The lock now has a ShutdownHook creation, which ends up doing {code} HookEntry(Runnable hook, int priority) { this(hook, priority, getShutdownTimeout(new Configuration()), TIME_UNIT_DEFAULT); } {code} which ends up doing a "new Configuration()" within the locked section. This indirectly hurts the cache hit scenarios as well, since if the lock on this is held, then the other section cannot be entered either. {code} I/O Setup 0 State: BLOCKED CPU usage on sample: 6ms org.apache.hadoop.fs.FileSystem$Cache.getInternal(URI, Configuration, FileSystem$Cache$Key) FileSystem.java:3345 org.apache.hadoop.fs.FileSystem$Cache.get(URI, Configuration) FileSystem.java:3320 org.apache.hadoop.fs.FileSystem.get(URI, Configuration) FileSystem.java:479 org.apache.hadoop.fs.FileSystem.getLocal(Configuration) FileSystem.java:435 {code} slowing down the RawLocalFileSystem when there are other threads creating HDFS FileSystem objects at the same time. > Regression: FileSystem cache lock parses XML within the lock > > > Key: HADOOP-16461 > URL: https://issues.apache.org/jira/browse/HADOOP-16461 > Project: Hadoop Common > Issue Type: Bug > Components: filecache >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > > https://github.com/apache/hadoop/blob/2546e6ece240924af2188bb39b3954a4896e4a4f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java#L3388 > {code} > fs = createFileSystem(uri, conf); >
[jira] [Assigned] (HADOOP-16461) Regression: FileSystem cache lock parses XML within the lock
[ https://issues.apache.org/jira/browse/HADOOP-16461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V reassigned HADOOP-16461: Assignee: Gopal V > Regression: FileSystem cache lock parses XML within the lock > > > Key: HADOOP-16461 > URL: https://issues.apache.org/jira/browse/HADOOP-16461 > Project: Hadoop Common > Issue Type: Bug > Components: filecache >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > > {code} > fs = createFileSystem(uri, conf); > synchronized (this) { // refetch the lock again > FileSystem oldfs = map.get(key); > if (oldfs != null) { // a file system is created while lock is > releasing > fs.close(); // close the new file system > return oldfs; // return the old file system > } > // now insert the new file system into the map > if (map.isEmpty() > && !ShutdownHookManager.get().isShutdownInProgress()) { > ShutdownHookManager.get().addShutdownHook(clientFinalizer, > SHUTDOWN_HOOK_PRIORITY); > } > fs.key = key; > map.put(key, fs); > if (conf.getBoolean( > FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) { > toAutoClose.add(key); > } > return fs; > } > {code} > The lock now has a ShutdownHook creation, which ends up doing > {code} > HookEntry(Runnable hook, int priority) { > this(hook, priority, > getShutdownTimeout(new Configuration()), > TIME_UNIT_DEFAULT); > } > {code} > which ends up doing a "new Configuration()" within the locked section. > This indirectly hurts the cache hit scenarios as well, since if the lock on > this is held, then the other section cannot be entered either. > {code} > I/O Setup 0 State: BLOCKED CPU usage on sample: 6ms > org.apache.hadoop.fs.FileSystem$Cache.getInternal(URI, Configuration, > FileSystem$Cache$Key) FileSystem.java:3345 > org.apache.hadoop.fs.FileSystem$Cache.get(URI, Configuration) > FileSystem.java:3320 > org.apache.hadoop.fs.FileSystem.get(URI, Configuration) FileSystem.java:479 > org.apache.hadoop.fs.FileSystem.getLocal(Configuration) FileSystem.java:435 > {code} > slowing down the RawLocalFileSystem when there are other threads creating > HDFS FileSystem objects at the same time. -- This message was sent by Atlassian JIRA (v7.6.14#76016) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Updated] (HADOOP-16461) Regression: FileSystem cache lock parses XML within the lock
[ https://issues.apache.org/jira/browse/HADOOP-16461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-16461: - Description: {code} fs = createFileSystem(uri, conf); synchronized (this) { // refetch the lock again FileSystem oldfs = map.get(key); if (oldfs != null) { // a file system is created while lock is releasing fs.close(); // close the new file system return oldfs; // return the old file system } // now insert the new file system into the map if (map.isEmpty() && !ShutdownHookManager.get().isShutdownInProgress()) { ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY); } fs.key = key; map.put(key, fs); if (conf.getBoolean( FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) { toAutoClose.add(key); } return fs; } {code} The lock now has a ShutdownHook creation, which ends up doing {code} HookEntry(Runnable hook, int priority) { this(hook, priority, getShutdownTimeout(new Configuration()), TIME_UNIT_DEFAULT); } {code} which ends up doing a "new Configuration()" within the locked section. This indirectly hurts the cache hit scenarios as well, since if the lock on this is held, then the other section cannot be entered either. {code} I/O Setup 0 State: BLOCKED CPU usage on sample: 6ms org.apache.hadoop.fs.FileSystem$Cache.getInternal(URI, Configuration, FileSystem$Cache$Key) FileSystem.java:3345 org.apache.hadoop.fs.FileSystem$Cache.get(URI, Configuration) FileSystem.java:3320 org.apache.hadoop.fs.FileSystem.get(URI, Configuration) FileSystem.java:479 org.apache.hadoop.fs.FileSystem.getLocal(Configuration) FileSystem.java:435 {code} slowing down the RawLocalFileSystem when there are other threads creating HDFS FileSystem objects at the same time. was: {code} fs = createFileSystem(uri, conf); synchronized (this) { // refetch the lock again FileSystem oldfs = map.get(key); if (oldfs != null) { // a file system is created while lock is releasing fs.close(); // close the new file system return oldfs; // return the old file system } // now insert the new file system into the map if (map.isEmpty() && !ShutdownHookManager.get().isShutdownInProgress()) { ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY); } fs.key = key; map.put(key, fs); if (conf.getBoolean( FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) { toAutoClose.add(key); } return fs; } {code} The lock now has a ShutdownHook creation, which ends up doing {code} HookEntry(Runnable hook, int priority) { this(hook, priority, getShutdownTimeout(new Configuration()), TIME_UNIT_DEFAULT); } {code} which ends up doing a "new Configuration()" within the locked section. > Regression: FileSystem cache lock parses XML within the lock > > > Key: HADOOP-16461 > URL: https://issues.apache.org/jira/browse/HADOOP-16461 > Project: Hadoop Common > Issue Type: Bug > Components: filecache >Reporter: Gopal V >Priority: Major > > {code} > fs = createFileSystem(uri, conf); > synchronized (this) { // refetch the lock again > FileSystem oldfs = map.get(key); > if (oldfs != null) { // a file system is created while lock is > releasing > fs.close(); // close the new file system > return oldfs; // return the old file system > } > // now insert the new file system into the map > if (map.isEmpty() > && !ShutdownHookManager.get().isShutdownInProgress()) { > ShutdownHookManager.get().addShutdownHook(clientFinalizer, > SHUTDOWN_HOOK_PRIORITY); > } > fs.key = key; > map.put(key, fs); > if (conf.getBoolean( > FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) { > toAutoClose.add(key); > } > return fs; > } > {code} > The lock now has a ShutdownHook creation, which ends up doing > {code} > HookEntry(Runnable hook, int priority) { > this(hook, priority, > getShutdownTimeout(new Configuration()), > TIME_UNIT_DEFAULT); > } > {code} > which ends up doing a "new Configuration()" within the locked section. > This indirectly hurts the cache hit scenarios as well, since if the lock on > this is held, then the other section cannot be entered either. > {code} > I/O Setup 0 State: BLOCKED CPU usage on sample: 6ms >
[jira] [Created] (HADOOP-16461) Regression: FileSystem cache lock parses XML within the lock
Gopal V created HADOOP-16461: Summary: Regression: FileSystem cache lock parses XML within the lock Key: HADOOP-16461 URL: https://issues.apache.org/jira/browse/HADOOP-16461 Project: Hadoop Common Issue Type: Bug Components: filecache Reporter: Gopal V {code} fs = createFileSystem(uri, conf); synchronized (this) { // refetch the lock again FileSystem oldfs = map.get(key); if (oldfs != null) { // a file system is created while lock is releasing fs.close(); // close the new file system return oldfs; // return the old file system } // now insert the new file system into the map if (map.isEmpty() && !ShutdownHookManager.get().isShutdownInProgress()) { ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY); } fs.key = key; map.put(key, fs); if (conf.getBoolean( FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) { toAutoClose.add(key); } return fs; } {code} The lock now has a ShutdownHook creation, which ends up doing {code} HookEntry(Runnable hook, int priority) { this(hook, priority, getShutdownTimeout(new Configuration()), TIME_UNIT_DEFAULT); } {code} which ends up doing a "new Configuration()" within the locked section. -- This message was sent by Atlassian JIRA (v7.6.14#76016) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-16341) ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679
[ https://issues.apache.org/jira/browse/HADOOP-16341?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16854936#comment-16854936 ] Gopal V commented on HADOOP-16341: -- Thanks, opened - https://github.com/apache/hadoop/pull/896 > ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679 > -- > > Key: HADOOP-16341 > URL: https://issues.apache.org/jira/browse/HADOOP-16341 > Project: Hadoop Common > Issue Type: Bug > Components: common >Affects Versions: 3.1.2 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > Fix For: 3.1.3 > > Attachments: HADOOP-16341.branch-3.1.002.patch, > HADOOP-16341.branch-3.1.1.patch, shutdown-hook-removal.png > > > !shutdown-hook-removal.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Updated] (HADOOP-16341) ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679
[ https://issues.apache.org/jira/browse/HADOOP-16341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-16341: - Attachment: HADOOP-16341.branch-3.1.002.patch > ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679 > -- > > Key: HADOOP-16341 > URL: https://issues.apache.org/jira/browse/HADOOP-16341 > Project: Hadoop Common > Issue Type: Bug > Components: common >Affects Versions: 3.1.2 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > Fix For: 3.1.3 > > Attachments: HADOOP-16341.branch-3.1.002.patch, > HADOOP-16341.branch-3.1.1.patch, shutdown-hook-removal.png > > > !shutdown-hook-removal.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-16341) ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679
[ https://issues.apache.org/jira/browse/HADOOP-16341?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16853594#comment-16853594 ] Gopal V commented on HADOOP-16341: -- Build failed for {code} [ERROR] bower qunit#1.19.0 ECMDERR Failed to execute "git clone https://github.com/jquery/qunit.git -b 1.19.0 --progress . --depth 1", exit code of #128 Cloning into '.'... fatal: unable to access 'https://github.com/jquery/qunit.git/': GnuTLS recv error (-54): Error in the pull function. [ERROR] [ERROR] Additional error details: [ERROR] Cloning into '.'... [ERROR] fatal: unable to access 'https://github.com/jquery/qunit.git/': GnuTLS recv error (-54): Error in the pull function. [INFO] {code} > ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679 > -- > > Key: HADOOP-16341 > URL: https://issues.apache.org/jira/browse/HADOOP-16341 > Project: Hadoop Common > Issue Type: Bug > Components: common >Affects Versions: 3.1.2 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > Fix For: 3.1.3 > > Attachments: HADOOP-16341.branch-3.1.1.patch, > shutdown-hook-removal.png > > > !shutdown-hook-removal.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Updated] (HADOOP-16341) ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679
[ https://issues.apache.org/jira/browse/HADOOP-16341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-16341: - Attachment: HADOOP-16341.branch-3.1.1.patch > ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679 > -- > > Key: HADOOP-16341 > URL: https://issues.apache.org/jira/browse/HADOOP-16341 > Project: Hadoop Common > Issue Type: Bug > Components: common >Affects Versions: 3.1.2 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > Attachments: HADOOP-16341.branch-3.1.1.patch, > shutdown-hook-removal.png > > > !shutdown-hook-removal.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Updated] (HADOOP-16341) ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679
[ https://issues.apache.org/jira/browse/HADOOP-16341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-16341: - Fix Version/s: 3.1.3 Status: Patch Available (was: Open) > ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679 > -- > > Key: HADOOP-16341 > URL: https://issues.apache.org/jira/browse/HADOOP-16341 > Project: Hadoop Common > Issue Type: Bug > Components: common >Affects Versions: 3.1.2 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > Fix For: 3.1.3 > > Attachments: HADOOP-16341.branch-3.1.1.patch, > shutdown-hook-removal.png > > > !shutdown-hook-removal.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Updated] (HADOOP-16341) ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679
[ https://issues.apache.org/jira/browse/HADOOP-16341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-16341: - Attachment: HADOOP-16341.1.patch > ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679 > -- > > Key: HADOOP-16341 > URL: https://issues.apache.org/jira/browse/HADOOP-16341 > Project: Hadoop Common > Issue Type: Bug > Components: common >Affects Versions: 3.1.2 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > Attachments: shutdown-hook-removal.png > > > !shutdown-hook-removal.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Updated] (HADOOP-16341) ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679
[ https://issues.apache.org/jira/browse/HADOOP-16341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-16341: - Attachment: (was: HADOOP-16341.1.patch) > ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679 > -- > > Key: HADOOP-16341 > URL: https://issues.apache.org/jira/browse/HADOOP-16341 > Project: Hadoop Common > Issue Type: Bug > Components: common >Affects Versions: 3.1.2 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > Attachments: shutdown-hook-removal.png > > > !shutdown-hook-removal.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Assigned] (HADOOP-16341) ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679
[ https://issues.apache.org/jira/browse/HADOOP-16341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V reassigned HADOOP-16341: Assignee: Gopal V > ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679 > -- > > Key: HADOOP-16341 > URL: https://issues.apache.org/jira/browse/HADOOP-16341 > Project: Hadoop Common > Issue Type: Bug > Components: common >Affects Versions: 3.1.2 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > Attachments: shutdown-hook-removal.png > > > !shutdown-hook-removal.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Created] (HADOOP-16341) ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679
Gopal V created HADOOP-16341: Summary: ShutDownHookManager: Regressed performance on Hook removals after HADOOP-15679 Key: HADOOP-16341 URL: https://issues.apache.org/jira/browse/HADOOP-16341 Project: Hadoop Common Issue Type: Bug Components: common Affects Versions: 3.1.2 Reporter: Gopal V Attachments: shutdown-hook-removal.png !shutdown-hook-removal.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-11223) Offer a read-only conf alternative to new Configuration()
[ https://issues.apache.org/jira/browse/HADOOP-11223?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16769741#comment-16769741 ] Gopal V commented on HADOOP-11223: -- bq. Mainly for performance and concurrency reasons we don't want the Hadoop Configuration to change or be re-read. This is not guaranteed by this patch, unfortunately - the "Unmodifiable" one is not immutable (see earlier discussion about addDefaultResource). > Offer a read-only conf alternative to new Configuration() > - > > Key: HADOOP-11223 > URL: https://issues.apache.org/jira/browse/HADOOP-11223 > Project: Hadoop Common > Issue Type: Bug > Components: conf >Reporter: Gopal V >Assignee: Michael Miller >Priority: Major > Labels: Performance > Attachments: HADOOP-11223.001.patch, HADOOP-11223.002.patch, > HADOOP-11223.003.patch > > > new Configuration() is called from several static blocks across Hadoop. > This is incredibly inefficient, since each one of those involves primarily > XML parsing at a point where the JIT won't be triggered & interpreter mode is > essentially forced on the JVM. > The alternate solution would be to offer a {{Configuration::getDefault()}} > alternative which disallows any modifications. > At the very least, such a method would need to be called from > # org.apache.hadoop.io.nativeio.NativeIO::() > # org.apache.hadoop.security.SecurityUtil::() > # org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider:: -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Updated] (HADOOP-11867) FS API: Add a high-performance vectored Read to FSDataInputStream API
[ https://issues.apache.org/jira/browse/HADOOP-11867?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11867: - Labels: performance (was: ) > FS API: Add a high-performance vectored Read to FSDataInputStream API > - > > Key: HADOOP-11867 > URL: https://issues.apache.org/jira/browse/HADOOP-11867 > Project: Hadoop Common > Issue Type: New Feature > Components: hdfs-client >Affects Versions: 3.0.0 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > Labels: performance > > The most significant way to read from a filesystem in an efficient way is to > let the FileSystem implementation handle the seek behaviour underneath the > API to be the most efficient as possible. > A better approach to the seek problem is to provide a sequence of read > locations as part of a single call, while letting the system schedule/plan > the reads ahead of time. > This is exceedingly useful for seek-heavy readers on HDFS, since this allows > for potentially optimizing away the seek-gaps within the FSDataInputStream > implementation. > For seek+read systems with even more latency than locally-attached disks, > something like a {{readFully(long[] offsets, ByteBuffer[] chunks)}} would > take of the seeks internally while reading chunk.remaining() bytes into each > chunk (which may be {{slice()}}ed off a bigger buffer). > The base implementation can stub in this as a sequence of seeks + read() into > ByteBuffers, without forcing each FS implementation to override this in any > way. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Updated] (HADOOP-11867) FS API: Add a high-performance vectored Read to FSDataInputStream API
[ https://issues.apache.org/jira/browse/HADOOP-11867?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11867: - Component/s: hdfs-client > FS API: Add a high-performance vectored Read to FSDataInputStream API > - > > Key: HADOOP-11867 > URL: https://issues.apache.org/jira/browse/HADOOP-11867 > Project: Hadoop Common > Issue Type: New Feature > Components: hdfs-client >Affects Versions: 3.0.0 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > Labels: performance > > The most significant way to read from a filesystem in an efficient way is to > let the FileSystem implementation handle the seek behaviour underneath the > API to be the most efficient as possible. > A better approach to the seek problem is to provide a sequence of read > locations as part of a single call, while letting the system schedule/plan > the reads ahead of time. > This is exceedingly useful for seek-heavy readers on HDFS, since this allows > for potentially optimizing away the seek-gaps within the FSDataInputStream > implementation. > For seek+read systems with even more latency than locally-attached disks, > something like a {{readFully(long[] offsets, ByteBuffer[] chunks)}} would > take of the seeks internally while reading chunk.remaining() bytes into each > chunk (which may be {{slice()}}ed off a bigger buffer). > The base implementation can stub in this as a sequence of seeks + read() into > ByteBuffers, without forcing each FS implementation to override this in any > way. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Updated] (HADOOP-11867) FS API: Add a high-performance vectored Read to FSDataInputStream API
[ https://issues.apache.org/jira/browse/HADOOP-11867?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11867: - Affects Version/s: (was: 2.8.0) 3.0.0 > FS API: Add a high-performance vectored Read to FSDataInputStream API > - > > Key: HADOOP-11867 > URL: https://issues.apache.org/jira/browse/HADOOP-11867 > Project: Hadoop Common > Issue Type: New Feature > Components: hdfs-client >Affects Versions: 3.0.0 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > Labels: performance > > The most significant way to read from a filesystem in an efficient way is to > let the FileSystem implementation handle the seek behaviour underneath the > API to be the most efficient as possible. > A better approach to the seek problem is to provide a sequence of read > locations as part of a single call, while letting the system schedule/plan > the reads ahead of time. > This is exceedingly useful for seek-heavy readers on HDFS, since this allows > for potentially optimizing away the seek-gaps within the FSDataInputStream > implementation. > For seek+read systems with even more latency than locally-attached disks, > something like a {{readFully(long[] offsets, ByteBuffer[] chunks)}} would > take of the seeks internally while reading chunk.remaining() bytes into each > chunk (which may be {{slice()}}ed off a bigger buffer). > The base implementation can stub in this as a sequence of seeks + read() into > ByteBuffers, without forcing each FS implementation to override this in any > way. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-11867) FS API: Add a high-performance vectored Read to FSDataInputStream API
[ https://issues.apache.org/jira/browse/HADOOP-11867?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16349507#comment-16349507 ] Gopal V commented on HADOOP-11867: -- No, this is not about zero-copy readers or locality (i.e no Located blocks, no MMAP), but about the API common between all the disaggregated storage engines, to reduce total API round-trips to fetch ranges (delegating internal block layouts to the storage layer). HDFS-3051 looks like it is specific optimization to the HDFS implementation, which speeds up the IO when you have locality along with an API like this. > FS API: Add a high-performance vectored Read to FSDataInputStream API > - > > Key: HADOOP-11867 > URL: https://issues.apache.org/jira/browse/HADOOP-11867 > Project: Hadoop Common > Issue Type: New Feature >Affects Versions: 2.8.0 >Reporter: Gopal V >Assignee: Gopal V >Priority: Major > > The most significant way to read from a filesystem in an efficient way is to > let the FileSystem implementation handle the seek behaviour underneath the > API to be the most efficient as possible. > A better approach to the seek problem is to provide a sequence of read > locations as part of a single call, while letting the system schedule/plan > the reads ahead of time. > This is exceedingly useful for seek-heavy readers on HDFS, since this allows > for potentially optimizing away the seek-gaps within the FSDataInputStream > implementation. > For seek+read systems with even more latency than locally-attached disks, > something like a {{readFully(long[] offsets, ByteBuffer[] chunks)}} would > take of the seeks internally while reading chunk.remaining() bytes into each > chunk (which may be {{slice()}}ed off a bigger buffer). > The base implementation can stub in this as a sequence of seeks + read() into > ByteBuffers, without forcing each FS implementation to override this in any > way. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-15171) Hadoop native ZLIB decompressor produces 0 bytes for some input
[ https://issues.apache.org/jira/browse/HADOOP-15171?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16344204#comment-16344204 ] Gopal V commented on HADOOP-15171: -- bq. this is becoming a pain This is a huge perf hit right now, the workaround is much slower than the original codepath. > Hadoop native ZLIB decompressor produces 0 bytes for some input > --- > > Key: HADOOP-15171 > URL: https://issues.apache.org/jira/browse/HADOOP-15171 > Project: Hadoop Common > Issue Type: Bug >Affects Versions: 3.1.0 >Reporter: Sergey Shelukhin >Priority: Blocker > Fix For: 3.1.0, 3.0.1 > > > While reading some ORC file via direct buffers, Hive gets a 0-sized buffer > for a particular compressed segment of the file. We narrowed it down to > Hadoop native ZLIB codec; when the data is copied to heap-based buffer and > the JDK Inflater is used, it produces correct output. Input is only 127 bytes > so I can paste it here. > All the other (many) blocks of the file are decompressed without problems by > the same code. > {noformat} > 2018-01-13T02:47:40,815 TRACE [IO-Elevator-Thread-0 > (1515637158315_0079_1_00_00_0)] encoded.EncodedReaderImpl: Decompressing > 127 bytes to dest buffer pos 524288, limit 786432 > 2018-01-13T02:47:40,816 WARN [IO-Elevator-Thread-0 > (1515637158315_0079_1_00_00_0)] encoded.EncodedReaderImpl: The codec has > produced 0 bytes for 127 bytes at pos 0, data hash 1719565039: [e3 92 e1 62 > 66 60 60 10 12 e5 98 e0 27 c4 c7 f1 e8 12 8f 40 c3 7b 5e 89 09 7f 6e 74 73 04 > 30 70 c9 72 b1 30 14 4d 60 82 49 37 bd e7 15 58 d0 cd 2f 31 a1 a1 e3 35 4c fa > 15 a3 02 4c 7a 51 37 bf c0 81 e5 02 12 13 5a b6 9f e2 04 ea 96 e3 62 65 b8 c3 > b4 01 ae fd d0 72 01 81 07 87 05 25 26 74 3c 5b c9 05 35 fd 0a b3 03 50 7b 83 > 11 c8 f2 c3 82 02 0f 96 0b 49 34 7c fa ff 9f 2d 80 01 00 > 2018-01-13T02:47:40,816 WARN [IO-Elevator-Thread-0 > (1515637158315_0079_1_00_00_0)] encoded.EncodedReaderImpl: Fell back to > JDK decompressor with memcopy; got 155 bytes > {noformat} > Hadoop version is based on 3.1 snapshot. > The size of libhadoop.so is 824403 bytes, and libgplcompression is 78273 > FWIW. Not sure how to extract versions from those. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Comment Edited] (HADOOP-14683) FileStatus.compareTo binary compat issue between 2.7 and 2.8
[ https://issues.apache.org/jira/browse/HADOOP-14683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16100637#comment-16100637 ] Gopal V edited comment on HADOOP-14683 at 7/25/17 7:50 PM: --- bq. does this retain the binary search functionality? Yes - that's why I suggested it. Also for ABI compat, it means that any existing code compiled with call-site compareTo(FileStatus) will continue to work, without breaking ABI from 2.8.0 -> 2.8.2. (edit: just realized for some calls, result will now be an Object, not FileStatus - needs a cast, to assign) As you said it used to handle non-FileStatus objects previously as not-equal. was (Author: gopalv): bq. does this retain the binary search functionality? Yes - that's why I suggested it. Also for ABI compat, it means that any existing code compiled with call-site compareTo(FileStatus) will continue to work, without breaking ABI from 2.8.0 -> 2.8.2. (edit: just realized binarySearch() result will now be an Object, not FileStatus - needs a cast, to assign) As you said it used to handle non-FileStatus objects previously as not-equal. > FileStatus.compareTo binary compat issue between 2.7 and 2.8 > > > Key: HADOOP-14683 > URL: https://issues.apache.org/jira/browse/HADOOP-14683 > Project: Hadoop Common > Issue Type: Bug >Affects Versions: 2.8.0, 2.8.1 >Reporter: Sergey Shelukhin >Assignee: Akira Ajisaka >Priority: Blocker > Attachments: HADOOP-14683-branch-2-01.patch > > > See HIVE-17133. Looks like the signature change is causing issues; according > to [~jnp] this is a public API. > Is it possible to add the old overload back (keeping the new one presumably) > in a point release on 2.8? That way we can avoid creating yet another shim > for this in Hive. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Comment Edited] (HADOOP-14683) FileStatus.compareTo binary compat issue between 2.7 and 2.8
[ https://issues.apache.org/jira/browse/HADOOP-14683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16100637#comment-16100637 ] Gopal V edited comment on HADOOP-14683 at 7/25/17 7:47 PM: --- bq. does this retain the binary search functionality? Yes - that's why I suggested it. Also for ABI compat, it means that any existing code compiled with call-site compareTo(FileStatus) will continue to work, without breaking ABI from 2.8.0 -> 2.8.2. (edit: just realized binarySearch() result will now be an Object, not FileStatus - needs a cast, to assign) As you said it used to handle non-FileStatus objects previously as not-equal. was (Author: gopalv): bq. does this retain the binary search functionality? Yes - that's why I suggested it. Also for ABI compat, it means that any existing code compiled with call-site compareTo(FileStatus) will continue to work, without breaking ABI from 2.8.0 -> 2.8.2. The type assignments could break, if someone has code to assign FileStatus to a Comparable object (but why would someone do that?) - but all Comparable accepting functions commonly used should erase the type to go to Comparable. As you said it used to handle non-FileStatus objects previously as not-equal. > FileStatus.compareTo binary compat issue between 2.7 and 2.8 > > > Key: HADOOP-14683 > URL: https://issues.apache.org/jira/browse/HADOOP-14683 > Project: Hadoop Common > Issue Type: Bug >Affects Versions: 2.8.0, 2.8.1 >Reporter: Sergey Shelukhin >Assignee: Akira Ajisaka >Priority: Blocker > Attachments: HADOOP-14683-branch-2-01.patch > > > See HIVE-17133. Looks like the signature change is causing issues; according > to [~jnp] this is a public API. > Is it possible to add the old overload back (keeping the new one presumably) > in a point release on 2.8? That way we can avoid creating yet another shim > for this in Hive. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Comment Edited] (HADOOP-14683) FileStatus.compareTo binary compat issue between 2.7 and 2.8
[ https://issues.apache.org/jira/browse/HADOOP-14683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16100637#comment-16100637 ] Gopal V edited comment on HADOOP-14683 at 7/25/17 7:45 PM: --- bq. does this retain the binary search functionality? Yes - that's why I suggested it. Also for ABI compat, it means that any existing code compiled with call-site compareTo(FileStatus) will continue to work, without breaking ABI from 2.8.0 -> 2.8.2. The type assignments could break, if someone has code to assign FileStatus to a Comparable object (but why would someone do that?) - but all Comparable accepting functions commonly used should erase the type to go to Comparable. As you said it used to handle non-FileStatus objects previously as not-equal. was (Author: gopalv): bq. does this retain the binary search functionality? Yes - that's why I suggested it. Also for ABI compat, it means that any existing code compiled with call-site compareTo(FileStatus) will continue to work, without breaking ABI from 2.8.0 -> 2.8.2. The type assignments could break, if someone has code to assign FileStatus to a Comparable object - but all Comparable accepting functions commonly used should erase the type to go to Comparable. As you said it used to handle non-FileStatus objects previously as not-equal. > FileStatus.compareTo binary compat issue between 2.7 and 2.8 > > > Key: HADOOP-14683 > URL: https://issues.apache.org/jira/browse/HADOOP-14683 > Project: Hadoop Common > Issue Type: Bug >Affects Versions: 2.8.0, 2.8.1 >Reporter: Sergey Shelukhin >Assignee: Akira Ajisaka >Priority: Blocker > Attachments: HADOOP-14683-branch-2-01.patch > > > See HIVE-17133. Looks like the signature change is causing issues; according > to [~jnp] this is a public API. > Is it possible to add the old overload back (keeping the new one presumably) > in a point release on 2.8? That way we can avoid creating yet another shim > for this in Hive. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-14683) FileStatus.compareTo binary compat issue between 2.7 and 2.8
[ https://issues.apache.org/jira/browse/HADOOP-14683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16100637#comment-16100637 ] Gopal V commented on HADOOP-14683: -- bq. does this retain the binary search functionality? Yes - that's why I suggested it. Also for ABI compat, it means that any existing code compiled with call-site compareTo(FileStatus) will continue to work, without breaking ABI from 2.8.0 -> 2.8.2. The type assignments could break, if someone has code to assign FileStatus to a Comparable object - but all Comparable accepting functions commonly used should erase the type to go to Comparable. As you said it used to handle non-FileStatus objects previously as not-equal. > FileStatus.compareTo binary compat issue between 2.7 and 2.8 > > > Key: HADOOP-14683 > URL: https://issues.apache.org/jira/browse/HADOOP-14683 > Project: Hadoop Common > Issue Type: Bug >Affects Versions: 2.8.0, 2.8.1 >Reporter: Sergey Shelukhin >Assignee: Akira Ajisaka >Priority: Blocker > Attachments: HADOOP-14683-branch-2-01.patch > > > See HIVE-17133. Looks like the signature change is causing issues; according > to [~jnp] this is a public API. > Is it possible to add the old overload back (keeping the new one presumably) > in a point release on 2.8? That way we can avoid creating yet another shim > for this in Hive. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-14683) FileStatus.compareTo binary compat issue between 2.7 and 2.8
[ https://issues.apache.org/jira/browse/HADOOP-14683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16099608#comment-16099608 ] Gopal V commented on HADOOP-14683: -- [~ajisakaa]: use Comparable ? > FileStatus.compareTo binary compat issue between 2.7 and 2.8 > > > Key: HADOOP-14683 > URL: https://issues.apache.org/jira/browse/HADOOP-14683 > Project: Hadoop Common > Issue Type: Bug >Affects Versions: 2.8.0, 2.8.1 >Reporter: Sergey Shelukhin >Priority: Critical > > See HIVE-17133. Looks like the signature change is causing issues; according > to [~jnp] this is a public API. > Is it possible to add the old overload back (keeping the new one presumably) > in a point release on 2.8? That way we can avoid creating yet another shim > for this in Hive. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-14054) NativeAzureFileSystem has multiple synchronized BufferedStream:read() calls in it
[ https://issues.apache.org/jira/browse/HADOOP-14054?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15852088#comment-15852088 ] Gopal V commented on HADOOP-14054: -- thanks [~steve_l], I will probably link the external issues to this JIRA for discoverability. > NativeAzureFileSystem has multiple synchronized BufferedStream:read() calls > in it > - > > Key: HADOOP-14054 > URL: https://issues.apache.org/jira/browse/HADOOP-14054 > Project: Hadoop Common > Issue Type: Bug > Components: fs/azure >Reporter: Gopal V > > See the lock around the multiple buffered input streams with the read1 > java.io.BufferedInputStream.read() calls are all synchronized blocks. > {code} > IO-Elevator-Thread-9" #288 daemon prio=5 os_prio=0 tid=0x7fbf9c0aa000 > nid=0xa07d runnable [0x7faf3b4b4000] >java.lang.Thread.State: RUNNABLE > sun.security.provider.DigestBase.implCompressMultiBlock(DigestBase.java:140) > at sun.security.provider.DigestBase.engineUpdate(DigestBase.java:127) > at > java.security.MessageDigest$Delegate.engineUpdate(MessageDigest.java:584) > at java.security.MessageDigest.update(MessageDigest.java:325) > at > com.microsoft.azure.storage.core.Utility.writeToOutputStream(Utility.java:1329) > at > com.microsoft.azure.storage.blob.CloudBlob$9.postProcessResponse(CloudBlob.java:1410) > at > com.microsoft.azure.storage.blob.CloudBlob$9.postProcessResponse(CloudBlob.java:1310) > at > com.microsoft.azure.storage.core.ExecutionEngine.executeWithRetry(ExecutionEngine.java:146) > at > com.microsoft.azure.storage.blob.CloudBlob.downloadRangeInternal(CloudBlob.java:1499) > at > com.microsoft.azure.storage.blob.BlobInputStream.dispatchRead(BlobInputStream.java:255) > - eliminated <0x7fbdd5475b68> (a > com.microsoft.azure.storage.blob.BlobInputStream) > at > com.microsoft.azure.storage.blob.BlobInputStream.readInternal(BlobInputStream.java:448) > - locked <0x7fbdd5475b68> (a > com.microsoft.azure.storage.blob.BlobInputStream) > at > com.microsoft.azure.storage.blob.BlobInputStream.read(BlobInputStream.java:420) > at java.io.BufferedInputStream.read1(BufferedInputStream.java:284) > at java.io.BufferedInputStream.read(BufferedInputStream.java:345) > - locked <0x7fbdd5475b38> (a java.io.BufferedInputStream) > at java.io.DataInputStream.read(DataInputStream.java:149) > at > org.apache.hadoop.fs.azure.NativeAzureFileSystem$NativeAzureFsInputStream.read(NativeAzureFileSystem.java:820) > - locked <0x7fbdd5475ac0> (a > org.apache.hadoop.fs.azure.NativeAzureFileSystem$NativeAzureFsInputStream) > at java.io.BufferedInputStream.read1(BufferedInputStream.java:284) > at java.io.BufferedInputStream.read(BufferedInputStream.java:345) > - locked <0x7fbdd5475a90> (a > org.apache.hadoop.fs.BufferedFSInputStream) > at java.io.DataInputStream.read(DataInputStream.java:149) > at > org.apache.hadoop.mapreduce.lib.input.UncompressedSplitLineReader.fillBuffer(UncompressedSplitLineReader.java:62) > at > org.apache.hadoop.util.LineReader.readDefaultLine(LineReader.java:216) > at org.apache.hadoop.util.LineReader.readLine(LineReader.java:174) > at > org.apache.hadoop.mapreduce.lib.input.UncompressedSplitLineReader.readLine(UncompressedSplitLineReader.java:94) > {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Resolved] (HADOOP-14054) NativeAzureFileSystem has multiple synchronized BufferedStream:read() calls in it
[ https://issues.apache.org/jira/browse/HADOOP-14054?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V resolved HADOOP-14054. -- Resolution: Won't Fix Closing as WONTFIX - will file issues on Azure github > NativeAzureFileSystem has multiple synchronized BufferedStream:read() calls > in it > - > > Key: HADOOP-14054 > URL: https://issues.apache.org/jira/browse/HADOOP-14054 > Project: Hadoop Common > Issue Type: Bug > Components: fs/azure >Reporter: Gopal V > > See the lock around the multiple buffered input streams with the read1 > java.io.BufferedInputStream.read() calls are all synchronized blocks. > {code} > IO-Elevator-Thread-9" #288 daemon prio=5 os_prio=0 tid=0x7fbf9c0aa000 > nid=0xa07d runnable [0x7faf3b4b4000] >java.lang.Thread.State: RUNNABLE > sun.security.provider.DigestBase.implCompressMultiBlock(DigestBase.java:140) > at sun.security.provider.DigestBase.engineUpdate(DigestBase.java:127) > at > java.security.MessageDigest$Delegate.engineUpdate(MessageDigest.java:584) > at java.security.MessageDigest.update(MessageDigest.java:325) > at > com.microsoft.azure.storage.core.Utility.writeToOutputStream(Utility.java:1329) > at > com.microsoft.azure.storage.blob.CloudBlob$9.postProcessResponse(CloudBlob.java:1410) > at > com.microsoft.azure.storage.blob.CloudBlob$9.postProcessResponse(CloudBlob.java:1310) > at > com.microsoft.azure.storage.core.ExecutionEngine.executeWithRetry(ExecutionEngine.java:146) > at > com.microsoft.azure.storage.blob.CloudBlob.downloadRangeInternal(CloudBlob.java:1499) > at > com.microsoft.azure.storage.blob.BlobInputStream.dispatchRead(BlobInputStream.java:255) > - eliminated <0x7fbdd5475b68> (a > com.microsoft.azure.storage.blob.BlobInputStream) > at > com.microsoft.azure.storage.blob.BlobInputStream.readInternal(BlobInputStream.java:448) > - locked <0x7fbdd5475b68> (a > com.microsoft.azure.storage.blob.BlobInputStream) > at > com.microsoft.azure.storage.blob.BlobInputStream.read(BlobInputStream.java:420) > at java.io.BufferedInputStream.read1(BufferedInputStream.java:284) > at java.io.BufferedInputStream.read(BufferedInputStream.java:345) > - locked <0x7fbdd5475b38> (a java.io.BufferedInputStream) > at java.io.DataInputStream.read(DataInputStream.java:149) > at > org.apache.hadoop.fs.azure.NativeAzureFileSystem$NativeAzureFsInputStream.read(NativeAzureFileSystem.java:820) > - locked <0x7fbdd5475ac0> (a > org.apache.hadoop.fs.azure.NativeAzureFileSystem$NativeAzureFsInputStream) > at java.io.BufferedInputStream.read1(BufferedInputStream.java:284) > at java.io.BufferedInputStream.read(BufferedInputStream.java:345) > - locked <0x7fbdd5475a90> (a > org.apache.hadoop.fs.BufferedFSInputStream) > at java.io.DataInputStream.read(DataInputStream.java:149) > at > org.apache.hadoop.mapreduce.lib.input.UncompressedSplitLineReader.fillBuffer(UncompressedSplitLineReader.java:62) > at > org.apache.hadoop.util.LineReader.readDefaultLine(LineReader.java:216) > at org.apache.hadoop.util.LineReader.readLine(LineReader.java:174) > at > org.apache.hadoop.mapreduce.lib.input.UncompressedSplitLineReader.readLine(UncompressedSplitLineReader.java:94) > {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Created] (HADOOP-14054) NativeAzureFileSystem has multiple synchronized BufferedStream:read() calls in it
Gopal V created HADOOP-14054: Summary: NativeAzureFileSystem has multiple synchronized BufferedStream:read() calls in it Key: HADOOP-14054 URL: https://issues.apache.org/jira/browse/HADOOP-14054 Project: Hadoop Common Issue Type: Bug Components: fs/azure Reporter: Gopal V See the lock around the multiple buffered input streams with the read1 java.io.BufferedInputStream.read() calls are all synchronized blocks. {code} IO-Elevator-Thread-9" #288 daemon prio=5 os_prio=0 tid=0x7fbf9c0aa000 nid=0xa07d runnable [0x7faf3b4b4000] java.lang.Thread.State: RUNNABLE sun.security.provider.DigestBase.implCompressMultiBlock(DigestBase.java:140) at sun.security.provider.DigestBase.engineUpdate(DigestBase.java:127) at java.security.MessageDigest$Delegate.engineUpdate(MessageDigest.java:584) at java.security.MessageDigest.update(MessageDigest.java:325) at com.microsoft.azure.storage.core.Utility.writeToOutputStream(Utility.java:1329) at com.microsoft.azure.storage.blob.CloudBlob$9.postProcessResponse(CloudBlob.java:1410) at com.microsoft.azure.storage.blob.CloudBlob$9.postProcessResponse(CloudBlob.java:1310) at com.microsoft.azure.storage.core.ExecutionEngine.executeWithRetry(ExecutionEngine.java:146) at com.microsoft.azure.storage.blob.CloudBlob.downloadRangeInternal(CloudBlob.java:1499) at com.microsoft.azure.storage.blob.BlobInputStream.dispatchRead(BlobInputStream.java:255) - eliminated <0x7fbdd5475b68> (a com.microsoft.azure.storage.blob.BlobInputStream) at com.microsoft.azure.storage.blob.BlobInputStream.readInternal(BlobInputStream.java:448) - locked <0x7fbdd5475b68> (a com.microsoft.azure.storage.blob.BlobInputStream) at com.microsoft.azure.storage.blob.BlobInputStream.read(BlobInputStream.java:420) at java.io.BufferedInputStream.read1(BufferedInputStream.java:284) at java.io.BufferedInputStream.read(BufferedInputStream.java:345) - locked <0x7fbdd5475b38> (a java.io.BufferedInputStream) at java.io.DataInputStream.read(DataInputStream.java:149) at org.apache.hadoop.fs.azure.NativeAzureFileSystem$NativeAzureFsInputStream.read(NativeAzureFileSystem.java:820) - locked <0x7fbdd5475ac0> (a org.apache.hadoop.fs.azure.NativeAzureFileSystem$NativeAzureFsInputStream) at java.io.BufferedInputStream.read1(BufferedInputStream.java:284) at java.io.BufferedInputStream.read(BufferedInputStream.java:345) - locked <0x7fbdd5475a90> (a org.apache.hadoop.fs.BufferedFSInputStream) at java.io.DataInputStream.read(DataInputStream.java:149) at org.apache.hadoop.mapreduce.lib.input.UncompressedSplitLineReader.fillBuffer(UncompressedSplitLineReader.java:62) at org.apache.hadoop.util.LineReader.readDefaultLine(LineReader.java:216) at org.apache.hadoop.util.LineReader.readLine(LineReader.java:174) at org.apache.hadoop.mapreduce.lib.input.UncompressedSplitLineReader.readLine(UncompressedSplitLineReader.java:94) {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-12956) Inevitable Log4j2 migration via slf4j
[ https://issues.apache.org/jira/browse/HADOOP-12956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15209207#comment-15209207 ] Gopal V commented on HADOOP-12956: -- bq. if commons-logging supports log4j2.x LLAP moved DFSClient etc logging to log4j2 via the jcl bridge - http://mvnrepository.com/artifact/org.apache.logging.log4j/log4j-jcl The thing that broke however was the .properties syntax & the /logs dynamic configuration servlet, which assumes log4j1.x > Inevitable Log4j2 migration via slf4j > - > > Key: HADOOP-12956 > URL: https://issues.apache.org/jira/browse/HADOOP-12956 > Project: Hadoop Common > Issue Type: Improvement >Reporter: Gopal V > > {{5 August 2015 --The Apache Logging Servicesâ„¢ Project Management Committee > (PMC) has announced that the Log4jâ„¢ 1.x logging framework has reached its end > of life (EOL) and is no longer officially supported.}} > https://blogs.apache.org/foundation/entry/apache_logging_services_project_announces > A whole framework log4j2 upgrade has to be synchronized, partly for improved > performance brought about by log4j2. > https://logging.apache.org/log4j/2.x/manual/async.html#Performance -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (HADOOP-12956) Inevitable Log4j2 migration via slf4j
Gopal V created HADOOP-12956: Summary: Inevitable Log4j2 migration via slf4j Key: HADOOP-12956 URL: https://issues.apache.org/jira/browse/HADOOP-12956 Project: Hadoop Common Issue Type: Improvement Reporter: Gopal V {{5 August 2015 --The Apache Logging Servicesâ„¢ Project Management Committee (PMC) has announced that the Log4jâ„¢ 1.x logging framework has reached its end of life (EOL) and is no longer officially supported.}} https://blogs.apache.org/foundation/entry/apache_logging_services_project_announces A whole framework log4j2 upgrade has to be synchronized, partly for improved performance brought about by log4j2. https://logging.apache.org/log4j/2.x/manual/async.html#Performance -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11771) Configuration::getClassByNameOrNull synchronizes on a static object
[ https://issues.apache.org/jira/browse/HADOOP-11771?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11771: - Description: {code} IPC Client (1970436060) connection to cn106-10.l42scl.hortonworks.com/172.21.128.106:34530 from application_1442254312093_2976 [BLOCKED] [DAEMON] org.apache.hadoop.conf.Configuration.getClassByNameOrNull(String) Configuration.java:2117 org.apache.hadoop.conf.Configuration.getClassByName(String) Configuration.java:2099 org.apache.hadoop.io.ObjectWritable.loadClass(Configuration, String) ObjectWritable.java:373 org.apache.hadoop.io.ObjectWritable.readObject(DataInput, ObjectWritable, Configuration) ObjectWritable.java:282 org.apache.hadoop.io.ObjectWritable.readFields(DataInput) ObjectWritable.java:77 org.apache.hadoop.ipc.Client$Connection.receiveRpcResponse() Client.java:1098 org.apache.hadoop.ipc.Client$Connection.run() Client.java:977 {code} {code} private static final Map>> CACHE_CLASSES = new WeakHashMap >>(); ... synchronized (CACHE_CLASSES) { map = CACHE_CLASSES.get(classLoader); if (map == null) { map = Collections.synchronizedMap( new WeakHashMap >()); CACHE_CLASSES.put(classLoader, map); } } {code} !configuration-sync-cache.png! !configuration-cache-bt.png! was: {code} private static final Map >> CACHE_CLASSES = new WeakHashMap >>(); ... synchronized (CACHE_CLASSES) { map = CACHE_CLASSES.get(classLoader); if (map == null) { map = Collections.synchronizedMap( new WeakHashMap >()); CACHE_CLASSES.put(classLoader, map); } } {code} !configuration-sync-cache.png! !configuration-cache-bt.png! > Configuration::getClassByNameOrNull synchronizes on a static object > --- > > Key: HADOOP-11771 > URL: https://issues.apache.org/jira/browse/HADOOP-11771 > Project: Hadoop Common > Issue Type: Sub-task > Components: conf, io, ipc >Reporter: Gopal V >Priority: Critical > Attachments: configuration-cache-bt.png, configuration-sync-cache.png > > > {code} > IPC Client (1970436060) connection to > cn106-10.l42scl.hortonworks.com/172.21.128.106:34530 from > application_1442254312093_2976 [BLOCKED] [DAEMON] > org.apache.hadoop.conf.Configuration.getClassByNameOrNull(String) > Configuration.java:2117 > org.apache.hadoop.conf.Configuration.getClassByName(String) > Configuration.java:2099 > org.apache.hadoop.io.ObjectWritable.loadClass(Configuration, String) > ObjectWritable.java:373 > org.apache.hadoop.io.ObjectWritable.readObject(DataInput, ObjectWritable, > Configuration) ObjectWritable.java:282 > org.apache.hadoop.io.ObjectWritable.readFields(DataInput) > ObjectWritable.java:77 > org.apache.hadoop.ipc.Client$Connection.receiveRpcResponse() Client.java:1098 > org.apache.hadoop.ipc.Client$Connection.run() Client.java:977 > {code} > {code} > private static final Map >> > CACHE_CLASSES = new WeakHashMap WeakReference >>(); > ... > synchronized (CACHE_CLASSES) { > map = CACHE_CLASSES.get(classLoader); > if (map == null) { > map = Collections.synchronizedMap( > new WeakHashMap >()); > CACHE_CLASSES.put(classLoader, map); > } > } > {code} > !configuration-sync-cache.png! > !configuration-cache-bt.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11771) Configuration::getClassByNameOrNull synchronizes on a static object
[ https://issues.apache.org/jira/browse/HADOOP-11771?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11771: - Priority: Critical (was: Major) > Configuration::getClassByNameOrNull synchronizes on a static object > --- > > Key: HADOOP-11771 > URL: https://issues.apache.org/jira/browse/HADOOP-11771 > Project: Hadoop Common > Issue Type: Sub-task > Components: conf, io, ipc >Reporter: Gopal V >Priority: Critical > Attachments: configuration-cache-bt.png, configuration-sync-cache.png > > > {code} > private static final Map>> > CACHE_CLASSES = new WeakHashMap WeakReference >>(); > ... > synchronized (CACHE_CLASSES) { > map = CACHE_CLASSES.get(classLoader); > if (map == null) { > map = Collections.synchronizedMap( > new WeakHashMap >()); > CACHE_CLASSES.put(classLoader, map); > } > } > {code} > !configuration-sync-cache.png! > !configuration-cache-bt.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-12444) Consider implementing lazy seek in S3AInputStream
[ https://issues.apache.org/jira/browse/HADOOP-12444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14960149#comment-14960149 ] Gopal V commented on HADOOP-12444: -- [~rajesh.balamohan]: Kicking off some 10Tb S3 tests with your side-port (https://github.com/rajeshbalamohan/hadoop-aws) as a new fs.s3a.impl. > Consider implementing lazy seek in S3AInputStream > - > > Key: HADOOP-12444 > URL: https://issues.apache.org/jira/browse/HADOOP-12444 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 >Affects Versions: 2.7.1 >Reporter: Rajesh Balamohan >Assignee: Rajesh Balamohan > Attachments: HADOOP-12444.1.patch, HADOOP-12444.WIP.patch > > > - Currently, "read(long position, byte[] buffer, int offset, int length)" is > not implemented in S3AInputStream (unlike DFSInputStream). So, > "readFully(long position, byte[] buffer, int offset, int length)" in > S3AInputStream goes through the default implementation of seek(), read(), > seek() in FSInputStream. > - However, seek() in S3AInputStream involves re-opening of connection to S3 > everytime > (https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java#L115). > > - It would be good to consider having a lazy seek implementation to reduce > connection overheads to S3. (e.g Presto implements lazy seek. > https://github.com/facebook/presto/blob/master/presto-hive/src/main/java/com/facebook/presto/hive/PrestoS3FileSystem.java#L623) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-12444) Consider implementing lazy seek in S3AInputStream
[ https://issues.apache.org/jira/browse/HADOOP-12444?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-12444: - Assignee: Rajesh Subash > Consider implementing lazy seek in S3AInputStream > - > > Key: HADOOP-12444 > URL: https://issues.apache.org/jira/browse/HADOOP-12444 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 >Affects Versions: 2.7.1 >Reporter: Rajesh Balamohan >Assignee: Rajesh Subash > Attachments: HADOOP-12444.1.patch, HADOOP-12444.WIP.patch > > > - Currently, "read(long position, byte[] buffer, int offset, int length)" is > not implemented in S3AInputStream (unlike DFSInputStream). So, > "readFully(long position, byte[] buffer, int offset, int length)" in > S3AInputStream goes through the default implementation of seek(), read(), > seek() in FSInputStream. > - However, seek() in S3AInputStream involves re-opening of connection to S3 > everytime > (https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java#L115). > > - It would be good to consider having a lazy seek implementation to reduce > connection overheads to S3. (e.g Presto implements lazy seek. > https://github.com/facebook/presto/blob/master/presto-hive/src/main/java/com/facebook/presto/hive/PrestoS3FileSystem.java#L623) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-12444) Consider implementing lazy seek in S3AInputStream
[ https://issues.apache.org/jira/browse/HADOOP-12444?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-12444: - Assignee: (was: Rajesh Subash) > Consider implementing lazy seek in S3AInputStream > - > > Key: HADOOP-12444 > URL: https://issues.apache.org/jira/browse/HADOOP-12444 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 >Affects Versions: 2.7.1 >Reporter: Rajesh Balamohan > Attachments: HADOOP-12444.1.patch, HADOOP-12444.WIP.patch > > > - Currently, "read(long position, byte[] buffer, int offset, int length)" is > not implemented in S3AInputStream (unlike DFSInputStream). So, > "readFully(long position, byte[] buffer, int offset, int length)" in > S3AInputStream goes through the default implementation of seek(), read(), > seek() in FSInputStream. > - However, seek() in S3AInputStream involves re-opening of connection to S3 > everytime > (https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java#L115). > > - It would be good to consider having a lazy seek implementation to reduce > connection overheads to S3. (e.g Presto implements lazy seek. > https://github.com/facebook/presto/blob/master/presto-hive/src/main/java/com/facebook/presto/hive/PrestoS3FileSystem.java#L623) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-12217) hashCode in DoubleWritable returns same value for many numbers
[ https://issues.apache.org/jira/browse/HADOOP-12217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14624087#comment-14624087 ] Gopal V commented on HADOOP-12217: -- bq. I would personally want to find out why fixing hashCode() in Hadoop's DoubleWritable breaks bucketing in Hive - I have some suspicions as to how that might happen, but I would need more info about exactly what you found. Touching the hashcode of any Writable breaks existing distributions in Hive - the hash is used to distribute data to satisfy the BUCKETED BY operations in DDLs. The bucket map-joins and sorted-merge joins will give incorrect results after you do something like this, because data will end up in different buckets for the old and new data, when you upgrade hadoop. Take a look at what happened to Varchar for instance - HIVE-8488 bq. Luckily this operation is not only extremely cheap to perform, The new Optimized hashtable does *NOT* use the Writable::hashCode(), instead uses a post-serialization hashcode (i.e murmur hash of the byte[] formed out of the BinarySortableSerde). This is because allocating objects in the inner loop results in allocator churn and frequent GC pauses - so it is cheaper to never allocate a Double/DoubleWritable, particularly when they're going to be an L1 cache miss (Writable - Double - double). The use of murmur came in as part of the L1 cache optimized hashtable in hive 0.14 (though it was committed the same month that 0.13 came out), which allows us to pack about ~6x the number of k-v pairs in the same amount of memory (DoubleWritable is way bigger than 8 bytes). bq. Once I figure out where/why Hive's behavior changed, I'll file a ticket there, too, if necessary, hopefully with useful patches Please try the same queries in Tez mode and see whether you're hitting the same issues - I suspect the core performance issues in MRv2 mode have mostly no recourse, because they can't readjust during runtime (which is what the L1 cache optimized hash-join does). hashCode in DoubleWritable returns same value for many numbers -- Key: HADOOP-12217 URL: https://issues.apache.org/jira/browse/HADOOP-12217 Project: Hadoop Common Issue Type: Bug Components: io Affects Versions: 0.18.0, 0.18.1, 0.18.2, 0.18.3, 0.19.0, 0.19.1, 0.20.0, 0.20.1, 0.20.2, 0.20.203.0, 0.20.204.0, 0.20.205.0, 1.0.0, 1.0.1, 1.0.2, 1.0.3, 1.0.4, 1.1.0, 1.1.1, 1.2.0, 0.21.0, 0.22.0, 0.23.0, 0.23.1, 0.23.3, 2.0.0-alpha, 2.0.1-alpha, 2.0.2-alpha, 0.23.4, 2.0.3-alpha, 0.23.5, 0.23.6, 1.1.2, 0.23.7, 2.1.0-beta, 2.0.4-alpha, 0.23.8, 1.2.1, 2.0.5-alpha, 0.23.9, 0.23.10, 0.23.11, 2.1.1-beta, 2.0.6-alpha, 2.2.0, 2.3.0, 2.4.0, 2.5.0, 2.4.1, 2.5.1, 2.5.2, 2.6.0, 2.7.0, 2.7.1 Reporter: Steve Scaffidi Labels: easyfix Attachments: HADOOP-12217.1.patch Because DoubleWritable.hashCode() is incorrect, using DoubleWritables as the keys in a HashMap results in abysmal performance, due to hash code collisions. I discovered this when testing the latest version of Hive and certain mapjoin queries were exceedingly slow. Evidently, Hive has its own wrapper/subclass around Hadoop's DoubleWritable that overrode used to override hashCode() with a correct implementation, but for some reason they recently removed that code, so it now uses the incorrect hashCode() method inherited from Hadoop's DoubleWritable. It appears that this bug has been there since DoubleWritable was created(wow!) so I can understand if fixing it is impractical due to the possibility of breaking things down-stream, but I can't think of anything that *should* break, off the top of my head. Searching JIRA, I found several related tickets, which may be useful for some historical perspective: HADOOP-3061, HADOOP-3243, HIVE-511, HIVE-1629, HIVE-7041 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-12217) hashCode in DoubleWritable returns same value for many numbers
[ https://issues.apache.org/jira/browse/HADOOP-12217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14622817#comment-14622817 ] Gopal V commented on HADOOP-12217: -- [~sscaffidi]: I think I hit this earlier and changing this broke the hive bucketing (which uses hash as is to decide filenames). You should probably look into why you're not able to take advantage of HIVE-6924 hashCode in DoubleWritable returns same value for many numbers -- Key: HADOOP-12217 URL: https://issues.apache.org/jira/browse/HADOOP-12217 Project: Hadoop Common Issue Type: Bug Components: io Affects Versions: 0.18.0, 0.18.1, 0.18.2, 0.18.3, 0.19.0, 0.19.1, 0.20.0, 0.20.1, 0.20.2, 0.20.203.0, 0.20.204.0, 0.20.205.0, 1.0.0, 1.0.1, 1.0.2, 1.0.3, 1.0.4, 1.1.0, 1.1.1, 1.2.0, 0.21.0, 0.22.0, 0.23.0, 0.23.1, 0.23.3, 2.0.0-alpha, 2.0.1-alpha, 2.0.2-alpha, 0.23.4, 2.0.3-alpha, 0.23.5, 0.23.6, 1.1.2, 0.23.7, 2.1.0-beta, 2.0.4-alpha, 0.23.8, 1.2.1, 2.0.5-alpha, 0.23.9, 0.23.10, 0.23.11, 2.1.1-beta, 2.0.6-alpha, 2.2.0, 2.3.0, 2.4.0, 2.5.0, 2.4.1, 2.5.1, 2.5.2, 2.6.0, 2.7.0, 2.7.1 Reporter: Steve Scaffidi Labels: easyfix Because DoubleWritable.hashCode() is incorrect, using DoubleWritables as the keys in a HashMap results in abysmal performance, due to hash code collisions. I discovered this when testing the latest version of Hive and certain mapjoin queries were exceedingly slow. Evidently, Hive has its own wrapper/subclass around Hadoop's DoubleWritable that overrode used to override hashCode() with a correct implementation, but for some reason they recently removed that code, so it now uses the incorrect hashCode() method inherited from Hadoop's DoubleWritable. It appears that this bug has been there since DoubleWritable was created(wow!) so I can understand if fixing it is impractical due to the possibility of breaking things down-stream, but I can't think of anything that *should* break, off the top of my head. Searching JIRA, I found several related tickets, which may be useful for some historical perspective: HADOOP-3061, HADOOP-3243, HIVE-511, HIVE-1629, HIVE-7041 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-10694) Remove synchronized input streams from Writable deserialization
[ https://issues.apache.org/jira/browse/HADOOP-10694?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14621740#comment-14621740 ] Gopal V commented on HADOOP-10694: -- [~ozawa]: I think the synchronization is unintentional, since two threads using DataInputBuffer readers without synchronization would cause corruption (something like readInt() would result in interleaved corruption, because each byte read() is independently synchronized). Remove synchronized input streams from Writable deserialization --- Key: HADOOP-10694 URL: https://issues.apache.org/jira/browse/HADOOP-10694 Project: Hadoop Common Issue Type: Bug Components: io Reporter: Gopal V Assignee: Gopal V Labels: BB2015-05-TBR Attachments: HADOOP-10694.1.patch, writable-read-sync.png Writable deserialization is slowing down due to a synchronized block within DataInputBuffer$Buffer. ByteArrayInputStream::read() is synchronized and this shows up as a slow uncontested lock. Hive ships with its own faster thread-unsafe version with hive.common.io.NonSyncByteArrayInputStream. !writable-read-sync.png! The DataInputBuffer and Writable deserialization should not require a lock per readInt()/read(). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (HADOOP-11867) FS API: Add a high-performance vectored Read to FSDataInputStream API
[ https://issues.apache.org/jira/browse/HADOOP-11867?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V reassigned HADOOP-11867: Assignee: Gopal V FS API: Add a high-performance vectored Read to FSDataInputStream API - Key: HADOOP-11867 URL: https://issues.apache.org/jira/browse/HADOOP-11867 Project: Hadoop Common Issue Type: New Feature Affects Versions: 2.8.0 Reporter: Gopal V Assignee: Gopal V The most significant way to read from a filesystem in an efficient way is to let the FileSystem implementation handle the seek behaviour underneath the API to be the most efficient as possible. A better approach to the seek problem is to provide a sequence of read locations as part of a single call, while letting the system schedule/plan the reads ahead of time. This is exceedingly useful for seek-heavy readers on HDFS, since this allows for potentially optimizing away the seek-gaps within the FSDataInputStream implementation. For seek+read systems with even more latency than locally-attached disks, something like a {{readFully(long[] offsets, ByteBuffer[] chunks)}} would take of the seeks internally while reading chunk.remaining() bytes into each chunk (which may be {{slice()}}ed off a bigger buffer). The base implementation can stub in this as a sequence of seeks + read() into ByteBuffers, without forcing each FS implementation to override this in any way. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
[ https://issues.apache.org/jira/browse/HADOOP-11772?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11772: - Attachment: cached-locking.png cached-connections.png The patch looks good, there's no performance degradation due to locking !cached-locking.png! Tried to check total held connections, which seems to also be contained with the guava cache patch. !cached-connections.png! RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc, performance Reporter: Gopal V Assignee: Akira AJISAKA Labels: BB2015-05-RFC Attachments: HADOOP-11772-001.patch, HADOOP-11772-002.patch, HADOOP-11772-003.patch, HADOOP-11772-wip-001.patch, HADOOP-11772-wip-002.patch, HADOOP-11772.004.patch, after-ipc-fix.png, cached-connections.png, cached-locking.png, dfs-sync-ipc.png, sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! !dfs-sync-ipc.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
[ https://issues.apache.org/jira/browse/HADOOP-11772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14547444#comment-14547444 ] Gopal V commented on HADOOP-11772: -- Added patch to next week's build + deploy queue. RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc, performance Reporter: Gopal V Assignee: Akira AJISAKA Labels: BB2015-05-RFC Attachments: HADOOP-11772-001.patch, HADOOP-11772-002.patch, HADOOP-11772-003.patch, HADOOP-11772-wip-001.patch, HADOOP-11772-wip-002.patch, HADOOP-11772.004.patch, after-ipc-fix.png, dfs-sync-ipc.png, sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! !dfs-sync-ipc.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
[ https://issues.apache.org/jira/browse/HADOOP-11772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14546323#comment-14546323 ] Gopal V commented on HADOOP-11772: -- bq. reproduce the problem is to spawn a client that talks to 200 nodes concurrently, but unfortunately I don't have the access of the cluster nor YourKit. The problem was reported as being visible on 1 process when it talks to 1 NameNode. You do not need 200 nodes to reproduce this bug - I reported this as observed using 1 single process and 1 namenode instance (not even HA). I got my yourkit license for use with Apache Hive for free - see section (G) of their license and email their sales folks to get a free license. Those arguments aside, the earlier patch had a unit test - the testClientCacheFromMultiThreads() that [~ajisakaa] wrote, when you run that does that show blocked threads or de-scheduled threads with the new patch? This is an important fix late in the cycle, the new patch should get as much testing as early as possible. RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc, performance Reporter: Gopal V Assignee: Akira AJISAKA Labels: BB2015-05-RFC Attachments: HADOOP-11772-001.patch, HADOOP-11772-002.patch, HADOOP-11772-003.patch, HADOOP-11772-wip-001.patch, HADOOP-11772-wip-002.patch, HADOOP-11772.004.patch, after-ipc-fix.png, dfs-sync-ipc.png, sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! !dfs-sync-ipc.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
[ https://issues.apache.org/jira/browse/HADOOP-11772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14546245#comment-14546245 ] Gopal V commented on HADOOP-11772: -- [~wheat9]: have you got any profiles on this with a multi-threaded test? RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc, performance Reporter: Gopal V Assignee: Akira AJISAKA Labels: BB2015-05-RFC Attachments: HADOOP-11772-001.patch, HADOOP-11772-002.patch, HADOOP-11772-003.patch, HADOOP-11772-wip-001.patch, HADOOP-11772-wip-002.patch, HADOOP-11772.004.patch, after-ipc-fix.png, dfs-sync-ipc.png, sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! !dfs-sync-ipc.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
[ https://issues.apache.org/jira/browse/HADOOP-11772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14545934#comment-14545934 ] Gopal V commented on HADOOP-11772: -- bq. The RPC client will send out the request asynchronously. Asynchronously is what it does - so it does not fail even without this patch. The problem is that it takes 200-300ms to send it out, by which time another IPC update has already queued up for the same connection. See the two threads locked against each other in the bug report, where one is doing a NameNode operation and another is doing an ApplicationMaster update - which need never lock against each other in reality. Because they both use the same {{ipc.Client}} singleton. If you want to revisit this fix, please remove the Client singleton or find another way to remove the synchronization barrier around the getConnection() the way it prevents reopening connections for IPC. The current IPC implementation works asynchronously, but is too slow to keep up with sub-second performance on a multi-threaded daemon which uses a singleton locked object for 24 cores doing everything (namenode lookups, app master heartbeats, data movement events, statistic updates, error recovery). RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc, performance Reporter: Gopal V Assignee: Akira AJISAKA Labels: BB2015-05-RFC Attachments: HADOOP-11772-001.patch, HADOOP-11772-002.patch, HADOOP-11772-003.patch, HADOOP-11772-wip-001.patch, HADOOP-11772-wip-002.patch, after-ipc-fix.png, dfs-sync-ipc.png, sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! !dfs-sync-ipc.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11935) Provide optional native implementation of stat syscall.
[ https://issues.apache.org/jira/browse/HADOOP-11935?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533063#comment-14533063 ] Gopal V commented on HADOOP-11935: -- [~cnauroth]: http://docs.oracle.com/javase/7/docs/api/java/nio/file/LinkOption.html#NOFOLLOW_LINKS? Provide optional native implementation of stat syscall. --- Key: HADOOP-11935 URL: https://issues.apache.org/jira/browse/HADOOP-11935 Project: Hadoop Common Issue Type: Improvement Components: fs, native Reporter: Chris Nauroth Currently, {{RawLocalFileSystem.DeprecatedRawLocalFileStatus#loadPermissionInfo}} is implemented as forking an {{ls}} command and parsing the output. This was observed to be a bottleneck in YARN-3491. This issue proposes an optional native implementation of a {{stat}} syscall through JNI. We would maintain the existing code as a fallback for systems where the native code is not available. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11901) BytesWritable supports only up to ~700MB (instead of 2G) due to integer overflow.
[ https://issues.apache.org/jira/browse/HADOOP-11901?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14525437#comment-14525437 ] Gopal V commented on HADOOP-11901: -- If the only goal is overflow prevention you can skip a multiply and do {{size + size/2}} - the jdk bin-search bug was similar. BytesWritable supports only up to ~700MB (instead of 2G) due to integer overflow. - Key: HADOOP-11901 URL: https://issues.apache.org/jira/browse/HADOOP-11901 Project: Hadoop Common Issue Type: Improvement Reporter: Reynold Xin Assignee: Reynold Xin Attachments: HADOOP-11901.diff BytesWritable.setSize increases the buffer size by 1.5 each time ( * 3 / 2). This is an unsafe operation since it restricts the max size to ~700MB, since 700MB * 3 2GB. I didn't write a test case for this case because in order to trigger this, I'd need to allocate around 700MB, which is pretty expensive to do in a unit test. Note that I didn't throw any exception in the case integer overflow as I didn't want to change that behavior (callers to this might expect a java.lang.NegativeArraySizeException). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (HADOOP-11867) FS API: Add a high-performance vectored Read to FSDataInputStream API
Gopal V created HADOOP-11867: Summary: FS API: Add a high-performance vectored Read to FSDataInputStream API Key: HADOOP-11867 URL: https://issues.apache.org/jira/browse/HADOOP-11867 Project: Hadoop Common Issue Type: New Feature Affects Versions: 2.8.0 Reporter: Gopal V The most significant way to read from a filesystem in an efficient way is to let the FileSystem implementation handle the seek behaviour underneath the API to be the most efficient as possible. A better approach to the seek problem is to provide a sequence of read locations as part of a single call, while letting the system schedule/plan the reads ahead of time. This is exceedingly useful for seek-heavy readers on HDFS, since this allows for potentially optimizing away the seek-gaps within the FSDataInputStream implementation. For seek+read systems with even more latency than locally-attached disks, something like a {{readFully(long[] offsets, ByteBuffer[] chunks)}} would take of the seeks internally while reading chunk.remaining() bytes into each chunk (which may be {{slice()}}ed off a bigger buffer). The base implementation can stub in this as a sequence of seeks + read() into ByteBuffers, without forcing each FS implementation to override this in any way. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11867) FS API: Add a high-performance vectored Read to FSDataInputStream API
[ https://issues.apache.org/jira/browse/HADOOP-11867?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14507432#comment-14507432 ] Gopal V commented on HADOOP-11867: -- [~thodemoor]/[~sanjay.radia]: This is the range-read API that was tossed around during Hadoop Summit Eu FS API: Add a high-performance vectored Read to FSDataInputStream API - Key: HADOOP-11867 URL: https://issues.apache.org/jira/browse/HADOOP-11867 Project: Hadoop Common Issue Type: New Feature Affects Versions: 2.8.0 Reporter: Gopal V The most significant way to read from a filesystem in an efficient way is to let the FileSystem implementation handle the seek behaviour underneath the API to be the most efficient as possible. A better approach to the seek problem is to provide a sequence of read locations as part of a single call, while letting the system schedule/plan the reads ahead of time. This is exceedingly useful for seek-heavy readers on HDFS, since this allows for potentially optimizing away the seek-gaps within the FSDataInputStream implementation. For seek+read systems with even more latency than locally-attached disks, something like a {{readFully(long[] offsets, ByteBuffer[] chunks)}} would take of the seeks internally while reading chunk.remaining() bytes into each chunk (which may be {{slice()}}ed off a bigger buffer). The base implementation can stub in this as a sequence of seeks + read() into ByteBuffers, without forcing each FS implementation to override this in any way. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11867) FS API: Add a high-performance vectored Read to FSDataInputStream API
[ https://issues.apache.org/jira/browse/HADOOP-11867?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14507569#comment-14507569 ] Gopal V commented on HADOOP-11867: -- bq. {{openAt(Path, offset)}} That is a good idea, because in general the seeks are unavoidable in the FileSplitoffset:len - open + seek to offset immediately. bq. is it an error if I ask for overlapping ranges? I think that should be enforced, since that is not only hard to translate into fadvise/equivalent (with page alignment/sloppy reads etc), but is effectively wasted bandwidth and CPU fetching data twice across the wire for the stub implementation. Agree on 2 3, while 1 is a normal IOException - this has the slight disadvantage that the buffers need to be allocated upfront the API does not return till all the buffers are full. But that is in effect the trade-off this represents over the regular seek/read combo, making all the reads ahead of time into a processing buffer vs making them one at a time. FS API: Add a high-performance vectored Read to FSDataInputStream API - Key: HADOOP-11867 URL: https://issues.apache.org/jira/browse/HADOOP-11867 Project: Hadoop Common Issue Type: New Feature Affects Versions: 2.8.0 Reporter: Gopal V The most significant way to read from a filesystem in an efficient way is to let the FileSystem implementation handle the seek behaviour underneath the API to be the most efficient as possible. A better approach to the seek problem is to provide a sequence of read locations as part of a single call, while letting the system schedule/plan the reads ahead of time. This is exceedingly useful for seek-heavy readers on HDFS, since this allows for potentially optimizing away the seek-gaps within the FSDataInputStream implementation. For seek+read systems with even more latency than locally-attached disks, something like a {{readFully(long[] offsets, ByteBuffer[] chunks)}} would take of the seeks internally while reading chunk.remaining() bytes into each chunk (which may be {{slice()}}ed off a bigger buffer). The base implementation can stub in this as a sequence of seeks + read() into ByteBuffers, without forcing each FS implementation to override this in any way. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
[ https://issues.apache.org/jira/browse/HADOOP-11772?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11772: - Attachment: after-ipc-fix.png [~ajisakaa]: looks much better with the patch. !after-ipc-fix.png! I still see the occasional blocked getConnection(), but that's because I'm running 24 threads in parallel with 10 IPC Client instances. RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc, performance Reporter: Gopal V Assignee: Akira AJISAKA Attachments: HADOOP-11772-001.patch, HADOOP-11772-wip-001.patch, HADOOP-11772-wip-002.patch, after-ipc-fix.png, dfs-sync-ipc.png, sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! !dfs-sync-ipc.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
[ https://issues.apache.org/jira/browse/HADOOP-11772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14483923#comment-14483923 ] Gopal V commented on HADOOP-11772: -- [~ajisakaa]: added to today's builds. RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc, performance Reporter: Gopal V Assignee: Akira AJISAKA Attachments: HADOOP-11772-001.patch, HADOOP-11772-wip-001.patch, HADOOP-11772-wip-002.patch, dfs-sync-ipc.png, sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! !dfs-sync-ipc.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
[ https://issues.apache.org/jira/browse/HADOOP-11772?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11772: - Attachment: dfs-sync-ipc.png [~ajisakaa]: Does this patch fix the requirement of needing 1 IPC client per socket-factory? From a quick read, the single factory - single IPC client mapping still exists, so the same hash-bucket will be locked by all processes using regular RPC invoker. RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc, performance Reporter: Gopal V Assignee: Akira AJISAKA Attachments: HADOOP-11772-001.patch, dfs-sync-ipc.png, sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
[ https://issues.apache.org/jira/browse/HADOOP-11772?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11772: - Description: {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! !dfs-sync-ipc.png! was: {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc, performance Reporter: Gopal V Assignee: Akira AJISAKA Attachments: HADOOP-11772-001.patch, dfs-sync-ipc.png, sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! !dfs-sync-ipc.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
[ https://issues.apache.org/jira/browse/HADOOP-11772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14389056#comment-14389056 ] Gopal V commented on HADOOP-11772: -- bq. Do you mean we should create IPC client pool in ClientCache.java? A round-robin pool or at least something better than 1 client object per socket-factory would be nice. Right now, listing block locations in parallel (for compute splits, or opening files etc) produces very bad synchronization within that. RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc, performance Reporter: Gopal V Assignee: Akira AJISAKA Attachments: HADOOP-11772-001.patch, dfs-sync-ipc.png, sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! !dfs-sync-ipc.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11771) Configuration::getClassByNameOrNull synchronizes on a static object
[ https://issues.apache.org/jira/browse/HADOOP-11771?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14385168#comment-14385168 ] Gopal V commented on HADOOP-11771: -- The cache is of dubious value for the default class-loader, is there a way to disable the cache altogether? Configuration::getClassByNameOrNull synchronizes on a static object --- Key: HADOOP-11771 URL: https://issues.apache.org/jira/browse/HADOOP-11771 Project: Hadoop Common Issue Type: Sub-task Components: conf, io, ipc Reporter: Gopal V Attachments: configuration-cache-bt.png, configuration-sync-cache.png {code} private static final MapClassLoader, MapString, WeakReferenceClass? CACHE_CLASSES = new WeakHashMapClassLoader, MapString, WeakReferenceClass?(); ... synchronized (CACHE_CLASSES) { map = CACHE_CLASSES.get(classLoader); if (map == null) { map = Collections.synchronizedMap( new WeakHashMapString, WeakReferenceClass?()); CACHE_CLASSES.put(classLoader, map); } } {code} !configuration-sync-cache.png! !configuration-cache-bt.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11770) [Umbrella] locate static synchronized blocks in hadoop-common
[ https://issues.apache.org/jira/browse/HADOOP-11770?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11770: - Priority: Critical (was: Major) [Umbrella] locate static synchronized blocks in hadoop-common - Key: HADOOP-11770 URL: https://issues.apache.org/jira/browse/HADOOP-11770 Project: Hadoop Common Issue Type: Improvement Components: conf, io, ipc Affects Versions: 2.7.0 Reporter: Gopal V Priority: Critical There are several static synchronized blocks in the hadoop common functionality that hurts any multi-threaded processing system which uses the common APIs. Identify the static synchronized blocks and locate them for potential fixes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (HADOOP-11770) [Umbrella] locate static synchronized blocks in hadoop-common
Gopal V created HADOOP-11770: Summary: [Umbrella] locate static synchronized blocks in hadoop-common Key: HADOOP-11770 URL: https://issues.apache.org/jira/browse/HADOOP-11770 Project: Hadoop Common Issue Type: Improvement Components: conf, io, ipc Affects Versions: 2.7.0 Reporter: Gopal V There are several static synchronized blocks in the hadoop common functionality that hurts any multi-threaded processing system which uses the common APIs. Identify the static synchronized blocks and locate them for potential fixes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11771) Configuration::getClassByNameOrNull synchronizes on a static object
[ https://issues.apache.org/jira/browse/HADOOP-11771?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11771: - Description: {code} private static final MapClassLoader, MapString, WeakReferenceClass? CACHE_CLASSES = new WeakHashMapClassLoader, MapString, WeakReferenceClass?(); ... synchronized (CACHE_CLASSES) { map = CACHE_CLASSES.get(classLoader); if (map == null) { map = Collections.synchronizedMap( new WeakHashMapString, WeakReferenceClass?()); CACHE_CLASSES.put(classLoader, map); } } {code} !configuration-sync-cache.png! !configuration-cache-bt.png! was: {code} private static final MapClassLoader, MapString, WeakReferenceClass? CACHE_CLASSES = new WeakHashMapClassLoader, MapString, WeakReferenceClass?(); ... synchronized (CACHE_CLASSES) { map = CACHE_CLASSES.get(classLoader); if (map == null) { map = Collections.synchronizedMap( new WeakHashMapString, WeakReferenceClass?()); CACHE_CLASSES.put(classLoader, map); } } {code} !configuration-sync-cache.png! Configuration::getClassByNameOrNull synchronizes on a static object --- Key: HADOOP-11771 URL: https://issues.apache.org/jira/browse/HADOOP-11771 Project: Hadoop Common Issue Type: Sub-task Components: conf, io, ipc Reporter: Gopal V Attachments: configuration-cache-bt.png, configuration-sync-cache.png {code} private static final MapClassLoader, MapString, WeakReferenceClass? CACHE_CLASSES = new WeakHashMapClassLoader, MapString, WeakReferenceClass?(); ... synchronized (CACHE_CLASSES) { map = CACHE_CLASSES.get(classLoader); if (map == null) { map = Collections.synchronizedMap( new WeakHashMapString, WeakReferenceClass?()); CACHE_CLASSES.put(classLoader, map); } } {code} !configuration-sync-cache.png! !configuration-cache-bt.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11771) Configuration::getClassByNameOrNull synchronizes on a static object
[ https://issues.apache.org/jira/browse/HADOOP-11771?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11771: - Attachment: configuration-sync-cache.png configuration-cache-bt.png Configuration::getClassByNameOrNull synchronizes on a static object --- Key: HADOOP-11771 URL: https://issues.apache.org/jira/browse/HADOOP-11771 Project: Hadoop Common Issue Type: Sub-task Components: conf, io, ipc Reporter: Gopal V Attachments: configuration-cache-bt.png, configuration-sync-cache.png {code} private static final MapClassLoader, MapString, WeakReferenceClass? CACHE_CLASSES = new WeakHashMapClassLoader, MapString, WeakReferenceClass?(); ... synchronized (CACHE_CLASSES) { map = CACHE_CLASSES.get(classLoader); if (map == null) { map = Collections.synchronizedMap( new WeakHashMapString, WeakReferenceClass?()); CACHE_CLASSES.put(classLoader, map); } } {code} !configuration-sync-cache.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (HADOOP-11771) Configuration::getClassByNameOrNull synchronizes on a static object
Gopal V created HADOOP-11771: Summary: Configuration::getClassByNameOrNull synchronizes on a static object Key: HADOOP-11771 URL: https://issues.apache.org/jira/browse/HADOOP-11771 Project: Hadoop Common Issue Type: Sub-task Reporter: Gopal V {code} private static final MapClassLoader, MapString, WeakReferenceClass? CACHE_CLASSES = new WeakHashMapClassLoader, MapString, WeakReferenceClass?(); ... synchronized (CACHE_CLASSES) { map = CACHE_CLASSES.get(classLoader); if (map == null) { map = Collections.synchronizedMap( new WeakHashMapString, WeakReferenceClass?()); CACHE_CLASSES.put(classLoader, map); } } {code} !configuration-sync-cache.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
Gopal V created HADOOP-11772: Summary: RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc Reporter: Gopal V Attachments: sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11772) RPC Invoker relies on static ClientCache which has synchronized(this) blocks
[ https://issues.apache.org/jira/browse/HADOOP-11772?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11772: - Attachment: sync-client-threads.png sync-client-bt.png RPC Invoker relies on static ClientCache which has synchronized(this) blocks Key: HADOOP-11772 URL: https://issues.apache.org/jira/browse/HADOOP-11772 Project: Hadoop Common Issue Type: Sub-task Components: ipc Reporter: Gopal V Attachments: sync-client-bt.png, sync-client-threads.png {code} private static ClientCache CLIENTS=new ClientCache(); ... this.client = CLIENTS.getClient(conf, factory); {code} Meanwhile in ClientCache {code} public synchronized Client getClient(Configuration conf, SocketFactory factory, Class? extends Writable valueClass) { ... Client client = clients.get(factory); if (client == null) { client = new Client(valueClass, conf, factory); clients.put(factory, client); } else { client.incCount(); } {code} All invokers end up calling these methods, resulting in IPC clients choking up. !sync-client-threads.png! !sync-client-bt.png! -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Status: Patch Available (was: Open) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch, HADOOP-11226.3.patch, HADOOP-11226.4.patch, HADOOP-11226.5.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Attachment: HADOOP-11226.5.patch Updated patch with documentation ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch, HADOOP-11226.3.patch, HADOOP-11226.4.patch, HADOOP-11226.5.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Status: Open (was: Patch Available) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch, HADOOP-11226.3.patch, HADOOP-11226.4.patch, HADOOP-11226.5.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14342523#comment-14342523 ] Gopal V commented on HADOOP-11226: -- Sure, I will add - I grepped to see where tcp.nodelay is mentioned, didn't see any docs. ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch, HADOOP-11226.3.patch, HADOOP-11226.4.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Status: Open (was: Patch Available) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch, HADOOP-11226.3.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Status: Patch Available (was: Open) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch, HADOOP-11226.3.patch, HADOOP-11226.4.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Attachment: HADOOP-11226.4.patch ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch, HADOOP-11226.3.patch, HADOOP-11226.4.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Status: Open (was: Patch Available) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Status: Patch Available (was: Open) No added tests, since this has no easy way to test without a correctly configured system a tcpdump. ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch, HADOOP-11226.3.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Attachment: HADOOP-11226.3.patch Added {{ipc.client.low-latency}} with default=false ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch, HADOOP-11226.3.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14333052#comment-14333052 ] Gopal V commented on HADOOP-11226: -- Sure, I will update the patch in a couple of days with a config option. To correct Colin, this patch was built for 10GigE (ixgbe drivers). I tested this with both H-TCP and Cubic congestion control (2.6 and 3.1 kernels). The RFCs listed in the patch apply only toTCP/IP not Infiniband RDMA (unless JDK7+ translates the hints via SDP, which is something I'd love to be tested by the community). ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11223) Offer a read-only conf alternative to new Configuration()
[ https://issues.apache.org/jira/browse/HADOOP-11223?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14292452#comment-14292452 ] Gopal V commented on HADOOP-11223: -- [~cmccabe]: Agree, the current patch doesn't fix anything. The primary blockage I had trying to implement this is the expected behaviour of addDefaultResource(). The overlays set programmatically is somewhat easier to disable cleanly. Offer a read-only conf alternative to new Configuration() - Key: HADOOP-11223 URL: https://issues.apache.org/jira/browse/HADOOP-11223 Project: Hadoop Common Issue Type: Bug Components: conf Reporter: Gopal V Assignee: Varun Saxena Labels: Performance Attachments: HADOOP-11223.001.patch new Configuration() is called from several static blocks across Hadoop. This is incredibly inefficient, since each one of those involves primarily XML parsing at a point where the JIT won't be triggered interpreter mode is essentially forced on the JVM. The alternate solution would be to offer a {{Configuration::getDefault()}} alternative which disallows any modifications. At the very least, such a method would need to be called from # org.apache.hadoop.io.nativeio.NativeIO::clinit() # org.apache.hadoop.security.SecurityUtil::clinit() # org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider::clinit -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14290067#comment-14290067 ] Gopal V commented on HADOOP-11226: -- Thanks [~cnauroth]. I think this needs testing beyond what I've done - my test-setup is a single rack with has 10G ethernet - using both H-TCP Cubic congestion controls. This definitely needs more testing, should be even better when encountering cross-rack packet loss. In particular, I'd love to see this patch tested with JDK7 Infiniband native support. Hadoop IPC/DataStreamer/Shuffle can already use Infiniband zero-copy mode by adding JVM opt args configuration options {{-Dcom.sun.sdp.conf=/etc/sdp.conf -Dcom.sun.sdp.debug=/tmp/sdp-debug.log}}, provided the OS/Hardware has support. ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14287167#comment-14287167 ] Gopal V commented on HADOOP-11226: -- I was following the line which had TCP keepalive set as a hard-parameter. That was a sane default and on without options. Similarly, I didn't imagine any case where people want slow IPC with dropped packets. ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Labels: Infiniband (was: ) Release Note: Use low latency TCP connections for hadoop IPC Status: Patch Available (was: Open) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Labels: Infiniband Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11449) Cannot build on Windows: error: unexpected end tag: /ul
[ https://issues.apache.org/jira/browse/HADOOP-11449?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14258986#comment-14258986 ] Gopal V commented on HADOOP-11449: -- This is not a windows error - this is bad HTML in the javadoc. Try {{mvn clean package -Pdist -Dtar -Dmaven.javadoc.skip=true -DskipTests -fail-at-end -Pnative}} as a workaround. And if you feel like contributing, the doc HTML patches should be an easy one. Cannot build on Windows: error: unexpected end tag: /ul - Key: HADOOP-11449 URL: https://issues.apache.org/jira/browse/HADOOP-11449 Project: Hadoop Common Issue Type: Bug Components: build Affects Versions: site, trunk-win, 2.6.0 Environment: Windows 8.1 x64 java version 1.8.0_25 Java(TM) SE Runtime Environment (build 1.8.0_25-b18) Java HotSpot(TM) 64-Bit Server VM (build 25.25-b02, mixed mode) Apache Maven 3.2.5 (12a6b3acb947671f09b81f49094c53f426d8cea1; 2014-12-15T04:29:2 3) Reporter: Alec Taylor Labels: build, easyfix, hadoop, windows Original Estimate: 2h Remaining Estimate: 2h Tried on hadoop-2.6.0-src, branch-2.5 and branch-trunk-win. All gave this error: ``` [ERROR] Failed to execute goal org.apache.maven.plugins:maven-javadoc-plugin:2.8.1:jar (module-javadocs) on project hadoop-annotations: MavenReportException: Error while creating archive: [ERROR] Exit code: 1 - E:\Projects\hadoop-common\hadoop-common-project\hadoop-annotations\src\main\java\org\apache\hadoop\classification\InterfaceStability.java:27: error: unexpected end tag: /ul [ERROR] * /ul [ERROR] ^ [ERROR] [ERROR] Command line was: C:\Program Files\Java\jdk1.8.0_25\jre\..\bin\javadoc.exe @options @packages [ERROR] [ERROR] Refer to the generated Javadoc files in 'E:\Projects\hadoop-common\hadoop-common-project\hadoop-annotations\target' dir. [ERROR] - [Help 1] [ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch. [ERROR] Re-run Maven using the -X switch to enable full debug logging. [ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles: [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoExecutionException [ERROR] [ERROR] After correcting the problems, you can resume the build with the command [ERROR] mvn goals -rf :hadoop-annotations ``` -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Attachment: HADOOP-11226.2.patch ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Attachments: HADOOP-11226.1.patch, HADOOP-11226.2.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (HADOOP-11239) [JDK8] azurenative tests fail builds on JDK8
[ https://issues.apache.org/jira/browse/HADOOP-11239?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V resolved HADOOP-11239. -- Resolution: Not a Problem [JDK8] azurenative tests fail builds on JDK8 Key: HADOOP-11239 URL: https://issues.apache.org/jira/browse/HADOOP-11239 Project: Hadoop Common Issue Type: Bug Components: test Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Priority: Trivial java.util.Base64 conflicts with com.microsoft.windowsazure.storage.core.Base64 in Azure unit tests. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11223) Offer a read-only conf alternative to new Configuration()
[ https://issues.apache.org/jira/browse/HADOOP-11223?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14185990#comment-14185990 ] Gopal V commented on HADOOP-11223: -- Yes, that's the whole idea. This is merely from a JVM startup point of view. I'm not sure it is possible to make it immutable at this point, since it is used both during daemon runtime and job configuration. In the former mode, Configuration has more intimate problems with multiple threads mutability, thanks to addDefaultResources and the internal registry. Offer a read-only conf alternative to new Configuration() - Key: HADOOP-11223 URL: https://issues.apache.org/jira/browse/HADOOP-11223 Project: Hadoop Common Issue Type: Bug Components: conf Reporter: Gopal V Labels: Performance new Configuration() is called from several static blocks across Hadoop. This is incredibly inefficient, since each one of those involves primarily XML parsing at a point where the JIT won't be triggered interpreter mode is essentially forced on the JVM. The alternate solution would be to offer a {{Configuration::getDefault()}} alternative which disallows any modifications. At the very least, such a method would need to be called from # org.apache.hadoop.io.nativeio.NativeIO::clinit() # org.apache.hadoop.security.SecurityUtil::clinit() # org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider::clinit -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (HADOOP-11239) [JDK8] azurenative tests fail builds on JDK8
Gopal V created HADOOP-11239: Summary: [JDK8] azurenative tests fail builds on JDK8 Key: HADOOP-11239 URL: https://issues.apache.org/jira/browse/HADOOP-11239 Project: Hadoop Common Issue Type: Bug Components: test Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Priority: Trivial java.util.Base64 conflicts with com.microsoft.windowsazure.storage.core.Base64 in Azure unit tests. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY
Gopal V created HADOOP-11226: Summary: ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Reporter: Gopal V -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Component/s: ipc Description: During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. Affects Version/s: 2.6.0 ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY --- Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V reassigned HADOOP-11226: Assignee: Gopal V ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Summary: ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY (was: ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11226: - Attachment: HADOOP-11226.1.patch ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Attachments: HADOOP-11226.1.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11226) ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY
[ https://issues.apache.org/jira/browse/HADOOP-11226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14183665#comment-14183665 ] Gopal V commented on HADOOP-11226: -- Patch is not complete, as it takes care of only RFC 1349 compat mode. Needs to implement RFC 2474 protocol changes as well. http://docs.oracle.com/javase/1.5.0/docs/api/java/net/Socket.html#setPerformancePreferences%28int,%20int,%20int%29 ipc.Client has to use setTrafficClass() with IPTOS_LOWDELAY|IPTOS_RELIABILITY - Key: HADOOP-11226 URL: https://issues.apache.org/jira/browse/HADOOP-11226 Project: Hadoop Common Issue Type: Bug Components: ipc Affects Versions: 2.6.0 Reporter: Gopal V Assignee: Gopal V Attachments: HADOOP-11226.1.patch During heavy shuffle, packet loss for IPC packets was observed from a machine. Avoid packet-loss and speed up transfer by using 0x14 QOS bits for the packets. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (HADOOP-11223) Offer a read-only conf alternative to new Configuration()
Gopal V created HADOOP-11223: Summary: Offer a read-only conf alternative to new Configuration() Key: HADOOP-11223 URL: https://issues.apache.org/jira/browse/HADOOP-11223 Project: Hadoop Common Issue Type: Bug Reporter: Gopal V new Configuration() is called from several static blocks across Hadoop. This is incredibly inefficient, since each one of those involves primarily XML parsing at a point where the JIT won't be triggered interpreter mode is essentially forced on the JVM. The alternate solution would be to offer a {{Configuration::getDefault()}} alternative which disallows any modifications. At the very least, such a method would need to be called from # org.apache.hadoop.io.nativeio.NativeIO::clinit() # org.apache.hadoop.security.SecurityUtil::clinit() # org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider::clinit -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11223) Offer a read-only conf alternative to new Configuration()
[ https://issues.apache.org/jira/browse/HADOOP-11223?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11223: - Component/s: conf Offer a read-only conf alternative to new Configuration() - Key: HADOOP-11223 URL: https://issues.apache.org/jira/browse/HADOOP-11223 Project: Hadoop Common Issue Type: Bug Components: conf Reporter: Gopal V Labels: Performance new Configuration() is called from several static blocks across Hadoop. This is incredibly inefficient, since each one of those involves primarily XML parsing at a point where the JIT won't be triggered interpreter mode is essentially forced on the JVM. The alternate solution would be to offer a {{Configuration::getDefault()}} alternative which disallows any modifications. At the very least, such a method would need to be called from # org.apache.hadoop.io.nativeio.NativeIO::clinit() # org.apache.hadoop.security.SecurityUtil::clinit() # org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider::clinit -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11144) Update website to reflect that we use git, not svn
[ https://issues.apache.org/jira/browse/HADOOP-11144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14150110#comment-14150110 ] Gopal V commented on HADOOP-11144: -- Also retire broken mirrors - https://github.com/apache/hadoop-common Update website to reflect that we use git, not svn -- Key: HADOOP-11144 URL: https://issues.apache.org/jira/browse/HADOOP-11144 Project: Hadoop Common Issue Type: Bug Reporter: Arun C Murthy Assignee: Arun C Murthy We need to update http://hadoop.apache.org/version_control.html to reflect that we use git, not svn. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11029) FileSystem#Statistics uses volatile variables that must be updated on write or read calls.
[ https://issues.apache.org/jira/browse/HADOOP-11029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11029: - Attachment: fsdatainputstream.png FileSystem#Statistics uses volatile variables that must be updated on write or read calls. -- Key: HADOOP-11029 URL: https://issues.apache.org/jira/browse/HADOOP-11029 Project: Hadoop Common Issue Type: Bug Components: fs Affects Versions: 2.5.0, 2.6.0 Reporter: Gopal V Attachments: local-fs-locking.png This code is there in the hot-path of IFile writer via RawLocalFileSystem. !local-fs-locking.png! From a preliminary glance, the lock prefix calls are coming from a threadlocal.get() within FileSystem.Statistics {code} /** * Get or create the thread-local data associated with the current thread. */ private StatisticsData getThreadData() { StatisticsData data = threadData.get(); if (data == null) { data = new StatisticsData( new WeakReferenceThread(Thread.currentThread())); threadData.set(data); synchronized(this) { if (allData == null) { allData = new LinkedListStatisticsData(); } allData.add(data); } } return data; } /** * Increment the bytes read in the statistics * @param newBytes the additional bytes read */ public void incrementBytesRead(long newBytes) { getThreadData().bytesRead += newBytes; } {code} This is incredibly inefficient when used from FSDataOutputStream {code} public void write(int b) throws IOException { out.write(b); position++; if (statistics != null) { statistics.incrementBytesWritten(1); } } {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11029) FileSystem#Statistics uses volatile variables that must be updated on write or read calls.
[ https://issues.apache.org/jira/browse/HADOOP-11029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11029: - Attachment: (was: fsdatainputstream.png) FileSystem#Statistics uses volatile variables that must be updated on write or read calls. -- Key: HADOOP-11029 URL: https://issues.apache.org/jira/browse/HADOOP-11029 Project: Hadoop Common Issue Type: Bug Components: fs Affects Versions: 2.5.0, 2.6.0 Reporter: Gopal V Attachments: local-fs-locking.png This code is there in the hot-path of IFile writer via RawLocalFileSystem. !local-fs-locking.png! From a preliminary glance, the lock prefix calls are coming from a threadlocal.get() within FileSystem.Statistics {code} /** * Get or create the thread-local data associated with the current thread. */ private StatisticsData getThreadData() { StatisticsData data = threadData.get(); if (data == null) { data = new StatisticsData( new WeakReferenceThread(Thread.currentThread())); threadData.set(data); synchronized(this) { if (allData == null) { allData = new LinkedListStatisticsData(); } allData.add(data); } } return data; } /** * Increment the bytes read in the statistics * @param newBytes the additional bytes read */ public void incrementBytesRead(long newBytes) { getThreadData().bytesRead += newBytes; } {code} This is incredibly inefficient when used from FSDataOutputStream {code} public void write(int b) throws IOException { out.write(b); position++; if (statistics != null) { statistics.incrementBytesWritten(1); } } {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (HADOOP-11029) FileSystem#Statistics uses volatile variables that must be updated on write or read calls.
[ https://issues.apache.org/jira/browse/HADOOP-11029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14126228#comment-14126228 ] Gopal V commented on HADOOP-11029: -- bq. Can you put some numbers behind this? If this is only an issue when you are writing a byte at a time, I'd be inclined to close this, since, as I mentioned, byte-at-a-time is a well-known anti-pattern with Java streams. When running a wordcount example with a Snappy compressed stream (64kb buffers), I find that 57% of cpu samples are split between HADOOP-10694, HADOOP-10681, HADOOP-11029. !fsdatainputstream.png! This perf sample is using the write(byte[], int, int) version. FileSystem#Statistics uses volatile variables that must be updated on write or read calls. -- Key: HADOOP-11029 URL: https://issues.apache.org/jira/browse/HADOOP-11029 Project: Hadoop Common Issue Type: Bug Components: fs Affects Versions: 2.5.0, 2.6.0 Reporter: Gopal V Attachments: local-fs-locking.png This code is there in the hot-path of IFile writer via RawLocalFileSystem. !local-fs-locking.png! From a preliminary glance, the lock prefix calls are coming from a threadlocal.get() within FileSystem.Statistics {code} /** * Get or create the thread-local data associated with the current thread. */ private StatisticsData getThreadData() { StatisticsData data = threadData.get(); if (data == null) { data = new StatisticsData( new WeakReferenceThread(Thread.currentThread())); threadData.set(data); synchronized(this) { if (allData == null) { allData = new LinkedListStatisticsData(); } allData.add(data); } } return data; } /** * Increment the bytes read in the statistics * @param newBytes the additional bytes read */ public void incrementBytesRead(long newBytes) { getThreadData().bytesRead += newBytes; } {code} This is incredibly inefficient when used from FSDataOutputStream {code} public void write(int b) throws IOException { out.write(b); position++; if (statistics != null) { statistics.incrementBytesWritten(1); } } {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11029) FileSystem#Statistics uses volatile variables that must be updated on write or read calls.
[ https://issues.apache.org/jira/browse/HADOOP-11029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11029: - Attachment: fsdatainputstream.png FileSystem#Statistics uses volatile variables that must be updated on write or read calls. -- Key: HADOOP-11029 URL: https://issues.apache.org/jira/browse/HADOOP-11029 Project: Hadoop Common Issue Type: Bug Components: fs Affects Versions: 2.5.0, 2.6.0 Reporter: Gopal V Attachments: fsdatainputstream.png, local-fs-locking.png This code is there in the hot-path of IFile writer via RawLocalFileSystem. !local-fs-locking.png! From a preliminary glance, the lock prefix calls are coming from a threadlocal.get() within FileSystem.Statistics {code} /** * Get or create the thread-local data associated with the current thread. */ private StatisticsData getThreadData() { StatisticsData data = threadData.get(); if (data == null) { data = new StatisticsData( new WeakReferenceThread(Thread.currentThread())); threadData.set(data); synchronized(this) { if (allData == null) { allData = new LinkedListStatisticsData(); } allData.add(data); } } return data; } /** * Increment the bytes read in the statistics * @param newBytes the additional bytes read */ public void incrementBytesRead(long newBytes) { getThreadData().bytesRead += newBytes; } {code} This is incredibly inefficient when used from FSDataOutputStream {code} public void write(int b) throws IOException { out.write(b); position++; if (statistics != null) { statistics.incrementBytesWritten(1); } } {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (HADOOP-11029) LocalFS Statistics performs thread local call per byte written
[ https://issues.apache.org/jira/browse/HADOOP-11029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11029: - Affects Version/s: 2.6.0 2.5.0 LocalFS Statistics performs thread local call per byte written -- Key: HADOOP-11029 URL: https://issues.apache.org/jira/browse/HADOOP-11029 Project: Hadoop Common Issue Type: Bug Affects Versions: 2.5.0, 2.6.0 Reporter: Gopal V Attachments: local-fs-locking.png This code is there in the hot-path of IFile writer via RawLocalFileSystem. !local-fs-locking.png! From a preliminary glance, the lock prefix calls are coming from a threadlocal.get() within FileSystem.Statistics {code} /** * Get or create the thread-local data associated with the current thread. */ private StatisticsData getThreadData() { StatisticsData data = threadData.get(); if (data == null) { data = new StatisticsData( new WeakReferenceThread(Thread.currentThread())); threadData.set(data); synchronized(this) { if (allData == null) { allData = new LinkedListStatisticsData(); } allData.add(data); } } return data; } /** * Increment the bytes read in the statistics * @param newBytes the additional bytes read */ public void incrementBytesRead(long newBytes) { getThreadData().bytesRead += newBytes; } {code} This is incredibly inefficient when used from FSDataOutputStream {code} public void write(int b) throws IOException { out.write(b); position++; if (statistics != null) { statistics.incrementBytesWritten(1); } } {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (HADOOP-11029) LocalFS Statistics performs thread local call per byte written
Gopal V created HADOOP-11029: Summary: LocalFS Statistics performs thread local call per byte written Key: HADOOP-11029 URL: https://issues.apache.org/jira/browse/HADOOP-11029 Project: Hadoop Common Issue Type: Bug Reporter: Gopal V Attachments: local-fs-locking.png This code is there in the hot-path of IFile writer via RawLocalFileSystem. !local-fs-locking.png! From a preliminary glance, the lock prefix calls are coming from a threadlocal.get() within FileSystem.Statistics {code} /** * Get or create the thread-local data associated with the current thread. */ private StatisticsData getThreadData() { StatisticsData data = threadData.get(); if (data == null) { data = new StatisticsData( new WeakReferenceThread(Thread.currentThread())); threadData.set(data); synchronized(this) { if (allData == null) { allData = new LinkedListStatisticsData(); } allData.add(data); } } return data; } /** * Increment the bytes read in the statistics * @param newBytes the additional bytes read */ public void incrementBytesRead(long newBytes) { getThreadData().bytesRead += newBytes; } {code} This is incredibly inefficient when used from FSDataOutputStream {code} public void write(int b) throws IOException { out.write(b); position++; if (statistics != null) { statistics.incrementBytesWritten(1); } } {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (HADOOP-11029) LocalFS Statistics performs thread local call per byte written
[ https://issues.apache.org/jira/browse/HADOOP-11029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11029: - Attachment: local-fs-locking.png LocalFS Statistics performs thread local call per byte written -- Key: HADOOP-11029 URL: https://issues.apache.org/jira/browse/HADOOP-11029 Project: Hadoop Common Issue Type: Bug Affects Versions: 2.5.0, 2.6.0 Reporter: Gopal V Attachments: local-fs-locking.png This code is there in the hot-path of IFile writer via RawLocalFileSystem. !local-fs-locking.png! From a preliminary glance, the lock prefix calls are coming from a threadlocal.get() within FileSystem.Statistics {code} /** * Get or create the thread-local data associated with the current thread. */ private StatisticsData getThreadData() { StatisticsData data = threadData.get(); if (data == null) { data = new StatisticsData( new WeakReferenceThread(Thread.currentThread())); threadData.set(data); synchronized(this) { if (allData == null) { allData = new LinkedListStatisticsData(); } allData.add(data); } } return data; } /** * Increment the bytes read in the statistics * @param newBytes the additional bytes read */ public void incrementBytesRead(long newBytes) { getThreadData().bytesRead += newBytes; } {code} This is incredibly inefficient when used from FSDataOutputStream {code} public void write(int b) throws IOException { out.write(b); position++; if (statistics != null) { statistics.incrementBytesWritten(1); } } {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (HADOOP-11029) LocalFS Statistics performs thread local call per byte written
[ https://issues.apache.org/jira/browse/HADOOP-11029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-11029: - Component/s: fs LocalFS Statistics performs thread local call per byte written -- Key: HADOOP-11029 URL: https://issues.apache.org/jira/browse/HADOOP-11029 Project: Hadoop Common Issue Type: Bug Components: fs Affects Versions: 2.5.0, 2.6.0 Reporter: Gopal V Attachments: local-fs-locking.png This code is there in the hot-path of IFile writer via RawLocalFileSystem. !local-fs-locking.png! From a preliminary glance, the lock prefix calls are coming from a threadlocal.get() within FileSystem.Statistics {code} /** * Get or create the thread-local data associated with the current thread. */ private StatisticsData getThreadData() { StatisticsData data = threadData.get(); if (data == null) { data = new StatisticsData( new WeakReferenceThread(Thread.currentThread())); threadData.set(data); synchronized(this) { if (allData == null) { allData = new LinkedListStatisticsData(); } allData.add(data); } } return data; } /** * Increment the bytes read in the statistics * @param newBytes the additional bytes read */ public void incrementBytesRead(long newBytes) { getThreadData().bytesRead += newBytes; } {code} This is incredibly inefficient when used from FSDataOutputStream {code} public void write(int b) throws IOException { out.write(b); position++; if (statistics != null) { statistics.incrementBytesWritten(1); } } {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (HADOOP-9601) Support native CRC on byte arrays
[ https://issues.apache.org/jira/browse/HADOOP-9601?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gopal V updated HADOOP-9601: Resolution: Incomplete Status: Resolved (was: Patch Available) Closing this issue. btw, I found out bad interaction between between GC getArrayCritical when the memory is fragmented. This is faster until it gets slow all of a sudden. Please pass in the isCopy and run with G1GC to make sure it is doing zero-copy ops for getArrayRegion. Support native CRC on byte arrays - Key: HADOOP-9601 URL: https://issues.apache.org/jira/browse/HADOOP-9601 Project: Hadoop Common Issue Type: Improvement Components: performance, util Affects Versions: 3.0.0 Reporter: Todd Lipcon Assignee: Gopal V Labels: perfomance Attachments: HADOOP-9601-WIP-01.patch, HADOOP-9601-WIP-02.patch, HADOOP-9601-bench.patch, HADOOP-9601-rebase+benchmark.patch, HADOOP-9601-trunk-rebase-2.patch, HADOOP-9601-trunk-rebase.patch When we first implemented the Native CRC code, we only did so for direct byte buffers, because these correspond directly to native heap memory and thus make it easy to access via JNI. We'd generally assumed that accessing byte[] arrays from JNI was not efficient enough, but now that I know more about JNI I don't think that's true -- we just need to make sure that the critical sections where we lock the buffers are short. -- This message was sent by Atlassian JIRA (v6.2#6252)